-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-3838] Implemented drop partition column feature for delta streamer code path #5294
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
Changes from all commits
7505141
afc2f67
a6f16be
e9164c1
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 |
|---|---|---|
|
|
@@ -105,6 +105,7 @@ | |
| import scala.collection.JavaConversions; | ||
|
|
||
| import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; | ||
| import static org.apache.hudi.common.table.HoodieTableConfig.DROP_PARTITION_COLUMNS; | ||
| import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE; | ||
| import static org.apache.hudi.config.HoodieClusteringConfig.INLINE_CLUSTERING; | ||
| import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT; | ||
|
|
@@ -280,6 +281,7 @@ public void refreshTimeline() throws IOException { | |
| .setPreCombineField(cfg.sourceOrderingField) | ||
| .setPartitionMetafileUseBaseFormat(props.getBoolean(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.key(), | ||
| HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.defaultValue())) | ||
| .setDropPartitionColumnsWhenWrite(isDropPartitionColumns()) | ||
| .initTable(new Configuration(jssc.hadoopConfiguration()), | ||
| cfg.targetBasePath); | ||
| } | ||
|
|
@@ -375,6 +377,7 @@ public Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> readFromSource( | |
| SimpleKeyGenerator.class.getName())) | ||
| .setPartitionMetafileUseBaseFormat(props.getBoolean(HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.key(), | ||
| HoodieTableConfig.PARTITION_METAFILE_USE_BASE_FORMAT.defaultValue())) | ||
| .setDropPartitionColumnsWhenWrite(isDropPartitionColumns()) | ||
| .initTable(new Configuration(jssc.hadoopConfiguration()), cfg.targetBasePath); | ||
| } | ||
|
|
||
|
|
@@ -478,13 +481,14 @@ private Pair<SchemaProvider, Pair<String, JavaRDD<HoodieRecord>>> fetchFromSourc | |
|
|
||
| boolean shouldCombine = cfg.filterDupes || cfg.operation.equals(WriteOperationType.UPSERT); | ||
| JavaRDD<GenericRecord> avroRDD = avroRDDOptional.get(); | ||
| JavaRDD<HoodieRecord> records = avroRDD.map(gr -> { | ||
| JavaRDD<HoodieRecord> records = avroRDD.map(record -> { | ||
| GenericRecord gr = isDropPartitionColumns() ? HoodieAvroUtils.removeFields(record, getPartitionColumns(keyGenerator, props)) : record; | ||
|
Contributor
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. getPartitionColumns(keyGenerator, props) could have been done once in the driver.
Member
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. makes sense.. please land #5302 if it looks good.
Author
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 have created the same PR #5303 as well. |
||
| HoodieRecordPayload payload = shouldCombine ? DataSourceUtils.createPayload(cfg.payloadClassName, gr, | ||
| (Comparable) HoodieAvroUtils.getNestedFieldVal(gr, cfg.sourceOrderingField, false, props.getBoolean( | ||
| KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.key(), | ||
| Boolean.parseBoolean(KeyGeneratorOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue())))) | ||
| : DataSourceUtils.createPayload(cfg.payloadClassName, gr); | ||
| return new HoodieAvroRecord<>(keyGenerator.getKey(gr), payload); | ||
| return new HoodieAvroRecord<>(keyGenerator.getKey(record), payload); | ||
| }); | ||
|
|
||
| return Pair.of(schemaProvider, Pair.of(checkpointStr, records)); | ||
|
|
@@ -727,6 +731,9 @@ public void setupWriteClient() throws IOException { | |
|
|
||
| private void reInitWriteClient(Schema sourceSchema, Schema targetSchema) throws IOException { | ||
| LOG.info("Setting up new Hoodie Write Client"); | ||
| if (isDropPartitionColumns()) { | ||
| targetSchema = HoodieAvroUtils.removeFields(targetSchema, getPartitionColumns(keyGenerator, props)); | ||
| } | ||
| registerAvroSchemas(sourceSchema, targetSchema); | ||
| HoodieWriteConfig hoodieCfg = getHoodieClientConfig(targetSchema); | ||
| if (hoodieCfg.isEmbeddedTimelineServerEnabled()) { | ||
|
|
@@ -898,4 +905,24 @@ public Option<String> getClusteringInstantOpt() { | |
| return Option.empty(); | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * Set based on hoodie.datasource.write.drop.partition.columns config. | ||
| * When set to true, will not write the partition columns into the table. | ||
| */ | ||
| private Boolean isDropPartitionColumns() { | ||
| return props.getBoolean(DROP_PARTITION_COLUMNS.key(), DROP_PARTITION_COLUMNS.defaultValue()); | ||
| } | ||
|
|
||
| /** | ||
| * Get the list of partition columns as a list of strings. | ||
| * | ||
| * @param keyGenerator KeyGenerator | ||
| * @param props TypedProperties | ||
| * @return List of partition columns. | ||
| */ | ||
| private List<String> getPartitionColumns(KeyGenerator keyGenerator, TypedProperties props) { | ||
| String partitionColumns = HoodieSparkUtils.getPartitionColumns(keyGenerator, props); | ||
| return Arrays.asList(partitionColumns.split(",")); | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.