-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-14381][table] Partition field names should be got from CatalogTable instead of source/sink #9909
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[FLINK-14381][table] Partition field names should be got from CatalogTable instead of source/sink #9909
Changes from all commits
565f782
6122c86
92a1014
13fb322
6a82d9f
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -143,7 +143,8 @@ private Table convertConnectorTable( | |
return new TableSourceTable<>( | ||
tableSource, | ||
isStreamingMode, | ||
FlinkStatistic.builder().tableStats(tableStats).build()); | ||
FlinkStatistic.builder().tableStats(tableStats).build(), | ||
null); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Will we also have There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Actually, I have some concern to put the whole There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I would say let's use the information from CatalogTable more carefully then... I can imagine partition keys are just a starter, we might need more and more information from CatalogTable in the future. CatalogTable is some kind of meta information about the table. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi @KurtYoung , After temporary table support, every table should have There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Actually, we will retain some expression in source (for rowtime generation), then we have to have another field in There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi Jark, I don't quite understand the expression in this source, but I think we need a good name to clarify it. It's may not just the |
||
} else { | ||
Optional<TableSinkTable> tableSinkTable = table.getTableSink() | ||
.map(tableSink -> new TableSinkTable<>( | ||
|
@@ -180,7 +181,8 @@ private Table convertCatalogTable(ObjectPath tablePath, CatalogTable table) { | |
return new TableSourceTable<>( | ||
tableSource, | ||
!((StreamTableSource<?>) tableSource).isBounded(), | ||
FlinkStatistic.UNKNOWN() | ||
FlinkStatistic.UNKNOWN(), | ||
table | ||
); | ||
} | ||
|
||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -175,12 +175,10 @@ abstract class PlannerBase( | |
case catalogSink: CatalogSinkModifyOperation => | ||
val input = getRelBuilder.queryOperation(modifyOperation.getChild).build() | ||
val identifier = catalogManager.qualifyIdentifier(catalogSink.getTablePath: _*) | ||
getTableSink(identifier).map(sink => { | ||
TableSinkUtils.validateSink(catalogSink, identifier, sink) | ||
getTableSink(identifier).map { case (table, sink) => | ||
TableSinkUtils.validateSink(catalogSink, identifier, sink, table.getPartitionKeys) | ||
sink match { | ||
case partitionableSink: PartitionableTableSink | ||
if partitionableSink.getPartitionFieldNames != null | ||
&& partitionableSink.getPartitionFieldNames.nonEmpty => | ||
case partitionableSink: PartitionableTableSink => | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. see comment on There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, we can. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think we have validateSink above, no need to validate here. |
||
partitionableSink.setStaticPartition(catalogSink.getStaticPartitions) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'm wondering why we set the static partition information here but not during There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Maybe just for the test in |
||
case _ => | ||
} | ||
|
@@ -192,8 +190,8 @@ abstract class PlannerBase( | |
s"${classOf[OverwritableTableSink].getSimpleName} but actually got " + | ||
sink.getClass.getName) | ||
} | ||
LogicalSink.create(input, sink, catalogSink.getTablePath.mkString(".")) | ||
}) match { | ||
LogicalSink.create(input, sink, catalogSink.getTablePath.mkString("."), table) | ||
} match { | ||
case Some(sinkRel) => sinkRel | ||
case None => throw new TableException(s"Sink ${catalogSink.getTablePath} does not exists") | ||
} | ||
|
@@ -254,28 +252,31 @@ abstract class PlannerBase( | |
*/ | ||
protected def translateToPlan(execNodes: util.List[ExecNode[_, _]]): util.List[Transformation[_]] | ||
|
||
private def getTableSink(objectIdentifier: ObjectIdentifier): Option[TableSink[_]] = { | ||
JavaScalaConversionUtil.toScala(catalogManager.getTable(objectIdentifier)) match { | ||
private def getTableSink( | ||
tableIdentifier: ObjectIdentifier): Option[(CatalogTable, TableSink[_])] = { | ||
JavaScalaConversionUtil.toScala(catalogManager.getTable(tableIdentifier)) match { | ||
case Some(s) if s.isInstanceOf[ConnectorCatalogTable[_, _]] => | ||
JavaScalaConversionUtil | ||
.toScala(s.asInstanceOf[ConnectorCatalogTable[_, _]].getTableSink) | ||
val table = s.asInstanceOf[ConnectorCatalogTable[_, _]] | ||
JavaScalaConversionUtil.toScala(table.getTableSink) match { | ||
case Some(sink) => Some(table, sink) | ||
case None => None | ||
} | ||
|
||
case Some(s) if s.isInstanceOf[CatalogTable] => | ||
|
||
val catalog = catalogManager.getCatalog(objectIdentifier.getCatalogName) | ||
val catalogTable = s.asInstanceOf[CatalogTable] | ||
val catalog = catalogManager.getCatalog(tableIdentifier.getCatalogName) | ||
val table = s.asInstanceOf[CatalogTable] | ||
if (catalog.isPresent && catalog.get().getTableFactory.isPresent) { | ||
val objectPath = objectIdentifier.toObjectPath | ||
val objectPath = tableIdentifier.toObjectPath | ||
val sink = TableFactoryUtil.createTableSinkForCatalogTable( | ||
catalog.get(), | ||
catalogTable, | ||
table, | ||
objectPath) | ||
if (sink.isPresent) { | ||
return Option(sink.get()) | ||
return Option(table, sink.get()) | ||
} | ||
} | ||
val sinkProperties = catalogTable.toProperties | ||
Option(TableFactoryService.find(classOf[TableSinkFactory[_]], sinkProperties) | ||
val sinkProperties = table.toProperties | ||
Option(table, TableFactoryService.find(classOf[TableSinkFactory[_]], sinkProperties) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you explain what's the difference between There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. See |
||
.createTableSink(sinkProperties)) | ||
|
||
case _ => None | ||
|
Uh oh!
There was an error while loading. Please reload this page.