From 3d19618627e0d424293306cb6ac666223c091a03 Mon Sep 17 00:00:00 2001 From: manasaks Date: Tue, 6 Jul 2021 20:18:50 +0530 Subject: [PATCH 1/6] HUDI-1827 - Add ORC support to Bootstrap Op --- .../hudi/config/HoodieBootstrapConfig.java | 12 + .../apache/hudi/config/HoodieWriteConfig.java | 22 +- .../HoodieSparkBootstrapSchemaProvider.java | 62 +- .../SparkBootstrapCommitActionExecutor.java | 198 ++++-- .../hudi/common/table/HoodieTableConfig.java | 4 + .../common/table/HoodieTableMetaClient.java | 4 + .../apache/hudi/common/util/AvroOrcUtils.java | 62 ++ ...oodieSparkBootstrapWriteClientExample.java | 146 +++++ .../spark/HoodieSparkBootstrapExample.java | 94 +++ .../spark/HoodieSparkRegBootstrapExample.java | 104 ++++ .../org/apache/hudi/DataSourceOptions.scala | 5 + .../SparkOrcBootstrapDataProvider.java | 84 +++ .../apache/hudi/HoodieSparkSqlWriter.scala | 3 +- .../apache/hudi/client/TestOrcBootstrap.java | 587 ++++++++++++++++++ 14 files changed, 1315 insertions(+), 72 deletions(-) create mode 100644 hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieSparkBootstrapWriteClientExample.java create mode 100644 hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java create mode 100644 hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkRegBootstrapExample.java create mode 100644 hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkOrcBootstrapDataProvider.java create mode 100644 hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java index fb55442ab97f3..420c850f1253e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java @@ -1,3 +1,4 @@ + /* * Licensed to the Apache Software Foundation (ASF) under one * or more contributor license agreements. See the NOTICE file @@ -23,6 +24,7 @@ import org.apache.hudi.client.bootstrap.translator.IdentityBootstrapPartitionPathTranslator; import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex; import org.apache.hudi.common.config.DefaultHoodieConfig; +import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.table.HoodieTableConfig; import java.io.File; @@ -58,6 +60,9 @@ public class HoodieBootstrapConfig extends DefaultHoodieConfig { public static final String BOOTSTRAP_INDEX_CLASS_PROP = "hoodie.bootstrap.index.class"; public static final String DEFAULT_BOOTSTRAP_INDEX_CLASS = HFileBootstrapIndex.class.getName(); + public static final String HOODIE_BASE_FILE_FORMAT_PROP_NAME = "hoodie.table.base.file.format"; + private static final String DEFAULT_TABLE_BASE_FILE_FORMAT = HoodieFileFormat.PARQUET.name(); + public HoodieBootstrapConfig(Properties props) { super(props); } @@ -117,6 +122,11 @@ public Builder withBootstrapModeForRegexMatch(BootstrapMode modeForRegexMatch) { return this; } + public Builder withBootstrapBaseFileFormat(String baseFileFormat) { + props.setProperty(HOODIE_BASE_FILE_FORMAT_PROP_NAME, baseFileFormat); + return this; + } + public Builder fromProperties(Properties props) { this.props.putAll(props); return this; @@ -139,6 +149,8 @@ public HoodieBootstrapConfig build() { HoodieTableConfig.getDefaultBootstrapIndexClass(props)); setDefaultOnCondition(props, !props.containsKey(FULL_BOOTSTRAP_INPUT_PROVIDER), FULL_BOOTSTRAP_INPUT_PROVIDER, DEFAULT_FULL_BOOTSTRAP_INPUT_PROVIDER); + setDefaultOnCondition(props, !props.containsKey(HOODIE_BASE_FILE_FORMAT_PROP_NAME), HOODIE_BASE_FILE_FORMAT_PROP_NAME, + DEFAULT_TABLE_BASE_FILE_FORMAT); return config; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 9e89e0ef7f334..47ae5f8b67e4f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -27,10 +27,7 @@ import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.ConsistencyGuardConfig; -import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; -import org.apache.hudi.common.model.HoodieCleaningPolicy; -import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; -import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.model.*; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.ReflectionUtils; @@ -168,6 +165,10 @@ public class HoodieWriteConfig extends DefaultHoodieConfig { "hoodie.write.meta.key.prefixes"; public static final String DEFAULT_WRITE_META_KEY_PREFIXES = ""; + public static final String HOODIE_BASE_FILE_FORMAT_PROP_NAME = "hoodie.table.base.file.format"; + private static final String DEFAULT_TABLE_BASE_FILE_FORMAT = HoodieFileFormat.PARQUET.name(); + + /** * The specified write schema. In most case, we do not need set this parameter, * but for the case the write schema is not equal to the specified table schema, we can @@ -1073,6 +1074,11 @@ public WriteConcurrencyMode getWriteConcurrencyMode() { return WriteConcurrencyMode.fromValue(props.getProperty(WRITE_CONCURRENCY_MODE_PROP)); } + public String getHoodieBaseFileFormat() { + return props.getProperty(HOODIE_BASE_FILE_FORMAT_PROP_NAME); + } + + public Boolean inlineTableServices() { return inlineClusteringEnabled() || inlineCompactionEnabled() || isAutoClean(); } @@ -1375,6 +1381,11 @@ public Builder withWriteMetaKeyPrefixes(String writeMetaKeyPrefixes) { return this; } + public Builder withBaseFileFormat(String format) { + props.setProperty(HOODIE_BASE_FILE_FORMAT_PROP_NAME, format); + return this; + } + public Builder withProperties(Properties properties) { this.props.putAll(properties); return this; @@ -1470,6 +1481,9 @@ protected void setDefaults() { setDefaultOnCondition(props, !props.containsKey(TIMELINE_LAYOUT_VERSION), TIMELINE_LAYOUT_VERSION, String.valueOf(TimelineLayoutVersion.CURR_VERSION)); + setDefaultOnCondition(props, !props.containsKey(HOODIE_BASE_FILE_FORMAT_PROP_NAME), HOODIE_BASE_FILE_FORMAT_PROP_NAME, DEFAULT_TABLE_BASE_FILE_FORMAT); + + } private void validate() { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java index 6e82f42411e0e..d6ced7697747d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java @@ -34,6 +34,20 @@ import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; +import org.apache.orc.OrcFile; +import org.apache.orc.OrcProto.UserMetadataItem; +import org.apache.orc.Reader; +import org.apache.orc.Reader.Options; +import org.apache.orc.RecordReader; +import org.apache.orc.TypeDescription; +import org.apache.hudi.common.util.AvroOrcUtils; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.util.OrcReaderIterator; +import static org.apache.hudi.common.model.HoodieFileFormat.ORC; +import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; +import org.apache.hudi.avro.model.HoodiePath; + +import java.io.IOException; import java.util.List; import java.util.Objects; @@ -44,20 +58,30 @@ public HoodieSparkBootstrapSchemaProvider(HoodieWriteConfig writeConfig) { @Override protected Schema getBootstrapSourceSchema(HoodieEngineContext context, List>> partitions) { - MessageType parquetSchema = partitions.stream().flatMap(p -> p.getValue().stream()).map(fs -> { - try { - Path filePath = FileStatusUtils.toPath(fs.getPath()); - return new ParquetUtils().readSchema(context.getHadoopConf().get(), filePath); - } catch (Exception ex) { - return null; - } + Path filePath = partitions.stream().flatMap(p -> p.getValue().stream()).map(fs -> { + return FileStatusUtils.toPath(fs.getPath()); }).filter(Objects::nonNull).findAny() - .orElseThrow(() -> new HoodieException("Could not determine schema from the data files.")); + .orElseThrow(() -> new HoodieException("Could not determine schema from the data files.")); + + if(writeConfig.getHoodieBaseFileFormat().equals(PARQUET.toString())) + { + return getBootstrapSourceSchemaParquet(context,filePath); + } + else if(writeConfig.getHoodieBaseFileFormat().equals(ORC.toString())) + { + return getBootstrapSourceSchemaOrc(context,filePath ); + } + else + throw new HoodieException("Could not determine schema from the data files."); + + } + private Schema getBootstrapSourceSchemaParquet(HoodieEngineContext context, Path filePath ) { + MessageType parquetSchema = new ParquetUtils().readSchema(context.getHadoopConf().get(), filePath); ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter( - Boolean.parseBoolean(SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()), - Boolean.parseBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString())); + Boolean.parseBoolean(SQLConf.PARQUET_BINARY_AS_STRING().defaultValueString()), + Boolean.parseBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().defaultValueString())); StructType sparkSchema = converter.convert(parquetSchema); String tableName = HoodieAvroUtils.sanitizeName(writeConfig.getTableName()); String structName = tableName + "_record"; @@ -65,4 +89,22 @@ protected Schema getBootstrapSourceSchema(HoodieEngineContext context, List> - extends BaseCommitActionExecutor>, JavaRDD, JavaRDD, HoodieBootstrapWriteMetadata> { + extends BaseCommitActionExecutor>, JavaRDD, JavaRDD, HoodieBootstrapWriteMetadata> { private static final Logger LOG = LogManager.getLogger(SparkBootstrapCommitActionExecutor.class); protected String bootstrapSchema = null; @@ -111,20 +123,20 @@ public SparkBootstrapCommitActionExecutor(HoodieSparkEngineContext context, HoodieTable>, JavaRDD, JavaRDD> table, Option> extraMetadata) { super(context, new HoodieWriteConfig.Builder().withProps(config.getProps()) - .withAutoCommit(true).withWriteStatusClass(BootstrapWriteStatus.class) - .withBulkInsertParallelism(config.getBootstrapParallelism()) - .build(), table, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, WriteOperationType.BOOTSTRAP, - extraMetadata); + .withAutoCommit(true).withWriteStatusClass(BootstrapWriteStatus.class) + .withBulkInsertParallelism(config.getBootstrapParallelism()) + .build(), table, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, WriteOperationType.BOOTSTRAP, + extraMetadata); bootstrapSourceFileSystem = FSUtils.getFs(config.getBootstrapSourceBasePath(), hadoopConf); } private void validate() { ValidationUtils.checkArgument(config.getBootstrapSourceBasePath() != null, - "Ensure Bootstrap Source Path is set"); + "Ensure Bootstrap Source Path is set"); ValidationUtils.checkArgument(config.getBootstrapModeSelectorClass() != null, - "Ensure Bootstrap Partition Selector is set"); + "Ensure Bootstrap Partition Selector is set"); ValidationUtils.checkArgument(config.getBootstrapKeyGeneratorClass() != null, - "Ensure bootstrap key generator class is set"); + "Ensure bootstrap key generator class is set"); } @Override @@ -133,10 +145,10 @@ public HoodieBootstrapWriteMetadata execute() { try { HoodieTableMetaClient metaClient = table.getMetaClient(); Option completetedInstant = - metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); + metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); ValidationUtils.checkArgument(!completetedInstant.isPresent(), - "Active Timeline is expected to be empty for bootstrap to be performed. " - + "If you want to re-bootstrap, please rollback bootstrap first !!"); + "Active Timeline is expected to be empty for bootstrap to be performed. " + + "If you want to re-bootstrap, please rollback bootstrap first !!"); Map>>> partitionSelections = listAndProcessSourcePartitions(); // First run metadata bootstrap which will auto commit @@ -165,11 +177,11 @@ protected Option metadataBootstrap(List bootstrapWriteStatuses = runMetadataBootstrap(partitionFilesList); @@ -185,7 +197,7 @@ private void updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, H Instant indexStartTime = Instant.now(); // Update the index back JavaRDD statuses = table.getIndex().updateLocation(writeStatusRDD, context, - table); + table); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); commitOnAutoCommit(result); @@ -202,26 +214,26 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta // Perform bootstrap index write and then commit. Make sure both record-key and bootstrap-index // is all done in a single job DAG. Map>> bootstrapSourceAndStats = - result.getWriteStatuses().collect().stream() - .map(w -> { - BootstrapWriteStatus ws = (BootstrapWriteStatus) w; - return Pair.of(ws.getBootstrapSourceFileMapping(), ws.getStat()); - }).collect(Collectors.groupingBy(w -> w.getKey().getPartitionPath())); + result.getWriteStatuses().collect().stream() + .map(w -> { + BootstrapWriteStatus ws = (BootstrapWriteStatus) w; + return Pair.of(ws.getBootstrapSourceFileMapping(), ws.getStat()); + }).collect(Collectors.groupingBy(w -> w.getKey().getPartitionPath())); HoodieTableMetaClient metaClient = table.getMetaClient(); try (BootstrapIndex.IndexWriter indexWriter = BootstrapIndex.getBootstrapIndex(metaClient) - .createWriter(metaClient.getTableConfig().getBootstrapBasePath().get())) { + .createWriter(metaClient.getTableConfig().getBootstrapBasePath().get())) { LOG.info("Starting to write bootstrap index for source " + config.getBootstrapSourceBasePath() + " in table " - + config.getBasePath()); + + config.getBasePath()); indexWriter.begin(); bootstrapSourceAndStats.forEach((key, value) -> indexWriter.appendNextPartition(key, - value.stream().map(Pair::getKey).collect(Collectors.toList()))); + value.stream().map(Pair::getKey).collect(Collectors.toList()))); indexWriter.finish(); LOG.info("Finished writing bootstrap index for source " + config.getBootstrapSourceBasePath() + " in table " - + config.getBasePath()); + + config.getBasePath()); } commit(extraMetadata, result, bootstrapSourceAndStats.values().stream() - .flatMap(f -> f.stream().map(Pair::getValue)).collect(Collectors.toList())); + .flatMap(f -> f.stream().map(Pair::getValue)).collect(Collectors.toList())); LOG.info("Committing metadata bootstrap !!"); } @@ -229,7 +241,7 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta protected void syncTableMetadata() { // Open up the metadata table again, for syncing try (HoodieTableMetadataWriter writer = - SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) { + SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) { LOG.info("Successfully synced to metadata table"); } catch (Exception e) { throw new HoodieMetadataException("Error syncing to metadata table.", e); @@ -261,11 +273,11 @@ protected void commit(Option> extraMetadata, HoodieWriteMeta try { activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime), - Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); LOG.info("Committed " + instantTime); } catch (IOException e) { throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, - e); + e); } result.setCommitMetadata(Option.of(metadata)); } @@ -281,14 +293,14 @@ protected Option fullBootstrap(List inputRecordsRDD = - (JavaRDD) inputProvider.generateInputRecords("bootstrap_source", config.getBootstrapSourceBasePath(), - partitionFilesList); + (JavaRDD) inputProvider.generateInputRecords("bootstrap_source", config.getBootstrapSourceBasePath(), + partitionFilesList); // Start Full Bootstrap final HoodieInstant requested = new HoodieInstant(State.REQUESTED, table.getMetaClient().getCommitActionType(), - HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS); + HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS); table.getActiveTimeline().createNewInstant(requested); // Setup correct schema and run bulk insert. @@ -297,8 +309,8 @@ protected Option fullBootstrap(List getBulkInsertActionExecutor(JavaRDD inputRecordsRDD) { return new SparkBulkInsertCommitActionExecutor((HoodieSparkEngineContext) context, new HoodieWriteConfig.Builder().withProps(config.getProps()) - .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, - inputRecordsRDD, Option.empty(), extraMetadata); + .withSchema(bootstrapSchema).build(), table, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, + inputRecordsRDD, Option.empty(), extraMetadata); } private BootstrapWriteStatus handleMetadataBootstrap(String srcPartitionPath, String partitionPath, @@ -306,24 +318,26 @@ private BootstrapWriteStatus handleMetadataBootstrap(String srcPartitionPath, St Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath()); HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, - table, partitionPath, FSUtils.createNewFileIdPfx(), table.getTaskContextSupplier()); + table, partitionPath, FSUtils.createNewFileIdPfx(), table.getTaskContextSupplier()); Schema avroSchema = null; try { ParquetMetadata readFooter = ParquetFileReader.readFooter(table.getHadoopConf(), sourceFilePath, - ParquetMetadataConverter.NO_FILTER); + ParquetMetadataConverter.NO_FILTER); MessageType parquetSchema = readFooter.getFileMetaData().getSchema(); avroSchema = new AvroSchemaConverter().convert(parquetSchema); Schema recordKeySchema = HoodieAvroUtils.generateProjectionSchema(avroSchema, - keyGenerator.getRecordKeyFieldNames()); + keyGenerator.getRecordKeyFieldNames()); LOG.info("Schema to be used for reading record Keys :" + recordKeySchema); AvroReadSupport.setAvroReadSchema(table.getHadoopConf(), recordKeySchema); AvroReadSupport.setRequestedProjection(table.getHadoopConf(), recordKeySchema); BoundedInMemoryExecutor wrapper = null; + + try (ParquetReader reader = - AvroParquetReader.builder(sourceFilePath).withConf(table.getHadoopConf()).build()) { + AvroParquetReader.builder(sourceFilePath).withConf(table.getHadoopConf()).build()) { wrapper = new SparkBoundedInMemoryExecutor(config, - new ParquetReaderIterator(reader), new BootstrapRecordConsumer(bootstrapHandle), inp -> { + new ParquetReaderIterator(reader), new BootstrapRecordConsumer(bootstrapHandle), inp -> { String recKey = keyGenerator.getKey(inp).getRecordKey(); GenericRecord gr = new GenericData.Record(HoodieAvroUtils.RECORD_KEY_SCHEMA); gr.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recKey); @@ -346,8 +360,62 @@ private BootstrapWriteStatus handleMetadataBootstrap(String srcPartitionPath, St BootstrapWriteStatus writeStatus = (BootstrapWriteStatus) bootstrapHandle.writeStatuses().get(0); BootstrapFileMapping bootstrapFileMapping = new BootstrapFileMapping( - config.getBootstrapSourceBasePath(), srcPartitionPath, partitionPath, - srcFileStatus, writeStatus.getFileId()); + config.getBootstrapSourceBasePath(), srcPartitionPath, partitionPath, + srcFileStatus, writeStatus.getFileId()); + writeStatus.setBootstrapSourceFileMapping(bootstrapFileMapping); + return writeStatus; + } + + private BootstrapWriteStatus handleMetadataBootstrapOrc(String srcPartitionPath, String partitionPath, + HoodieFileStatus srcFileStatus, KeyGeneratorInterface keyGenerator) { + + Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath()); + HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, + HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, table, partitionPath, FSUtils.createNewFileIdPfx(), + table.getTaskContextSupplier()); + Schema avroSchema = null; + try { + Reader orcReader = OrcFile.createReader(sourceFilePath, OrcFile.readerOptions(table.getHadoopConf())); + + TypeDescription orcSchema = orcReader.getSchema(); + + avroSchema= AvroOrcUtils.createAvroSchema(orcSchema); + + + Schema recordKeySchema = HoodieAvroUtils.generateProjectionSchema(avroSchema, + keyGenerator.getRecordKeyFieldNames()); + LOG.info("Schema to be used for reading record Keys :" + recordKeySchema); + AvroReadSupport.setAvroReadSchema(table.getHadoopConf(), recordKeySchema); + AvroReadSupport.setRequestedProjection(table.getHadoopConf(), recordKeySchema); + + + BoundedInMemoryExecutor wrapper = null; + try (RecordReader reader = orcReader.rows(new Options(table.getHadoopConf()).schema(orcSchema))) { + wrapper = new SparkBoundedInMemoryExecutor(config, + new OrcReaderIterator(reader,avroSchema,orcSchema), new BootstrapRecordConsumer(bootstrapHandle), inp -> { + String recKey = keyGenerator.getKey(inp).getRecordKey(); + GenericRecord gr = new GenericData.Record(HoodieAvroUtils.RECORD_KEY_SCHEMA); + gr.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recKey); + BootstrapRecordPayload payload = new BootstrapRecordPayload(gr); + HoodieRecord rec = new HoodieRecord(new HoodieKey(recKey, partitionPath), payload); + return rec; + }); + wrapper.execute(); + } catch (Exception e) { + throw new HoodieException(e); + } finally { + bootstrapHandle.close(); + if (null != wrapper) { + wrapper.shutdownNow(); + } + } + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + + BootstrapWriteStatus writeStatus = (BootstrapWriteStatus) bootstrapHandle.writeStatuses().get(0); + BootstrapFileMapping bootstrapFileMapping = new BootstrapFileMapping(config.getBootstrapSourceBasePath(), + srcPartitionPath, partitionPath, srcFileStatus, writeStatus.getFileId()); writeStatus.setBootstrapSourceFileMapping(bootstrapFileMapping); return writeStatus; } @@ -367,19 +435,19 @@ private Map>>> listAndPr LOG.info("Bootstrap Schema :" + bootstrapSchema); BootstrapModeSelector selector = - (BootstrapModeSelector) ReflectionUtils.loadClass(config.getBootstrapModeSelectorClass(), config); + (BootstrapModeSelector) ReflectionUtils.loadClass(config.getBootstrapModeSelectorClass(), config); Map> result = selector.select(folders); Map> partitionToFiles = folders.stream().collect( - Collectors.toMap(Pair::getKey, Pair::getValue)); + Collectors.toMap(Pair::getKey, Pair::getValue)); // Ensure all partitions are accounted for ValidationUtils.checkArgument(partitionToFiles.keySet().equals( - result.values().stream().flatMap(Collection::stream).collect(Collectors.toSet()))); + result.values().stream().flatMap(Collection::stream).collect(Collectors.toSet()))); return result.entrySet().stream().map(e -> Pair.of(e.getKey(), e.getValue().stream() - .map(p -> Pair.of(p, partitionToFiles.get(p))).collect(Collectors.toList()))) - .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); + .map(p -> Pair.of(p, partitionToFiles.get(p))).collect(Collectors.toList()))) + .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); } private JavaRDD runMetadataBootstrap(List>> partitions) { @@ -391,22 +459,38 @@ private JavaRDD runMetadataBootstrap(List>> bootstrapPaths = partitions.stream() - .flatMap(p -> { - String translatedPartitionPath = translator.getBootstrapTranslatedPath(p.getKey()); - return p.getValue().stream().map(f -> Pair.of(p.getKey(), Pair.of(translatedPartitionPath, f))); - }) - .collect(Collectors.toList()); + .flatMap(p -> { + String translatedPartitionPath = translator.getBootstrapTranslatedPath(p.getKey()); + return p.getValue().stream().map(f -> Pair.of(p.getKey(), Pair.of(translatedPartitionPath, f))); + }) + .collect(Collectors.toList()); return jsc.parallelize(bootstrapPaths, config.getBootstrapParallelism()) - .map(partitionFsPair -> handleMetadataBootstrap(partitionFsPair.getLeft(), partitionFsPair.getRight().getLeft(), - partitionFsPair.getRight().getRight(), keyGenerator)); + .map(partitionFsPair -> fileBasedMetadataHandler(partitionFsPair.getLeft(), partitionFsPair.getRight().getLeft(), + partitionFsPair.getRight().getRight(), keyGenerator)); } + + private BootstrapWriteStatus fileBasedMetadataHandler(String srcPartitionPath, String partitionPath, + HoodieFileStatus srcFileStatus, KeyGeneratorInterface keyGenerator) + { + Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath()); + + String extension = FSUtils.getFileExtension(sourceFilePath.toString()); + if (ORC.getFileExtension().equals(extension)) + return handleMetadataBootstrapOrc(srcPartitionPath,partitionPath,srcFileStatus,keyGenerator); + if (PARQUET.getFileExtension().equals(extension)) + return handleMetadataBootstrap(srcPartitionPath,partitionPath,srcFileStatus,keyGenerator); + else + throw new HoodieIOException("Hoodie InputFormat not implemented for base file format " + extension); + + + } @Override protected Iterator> handleInsert(String idPfx, Iterator> recordItr) { throw new UnsupportedOperationException("Should not called in bootstrap code path"); @@ -416,4 +500,4 @@ protected Iterator> handleInsert(String idPfx, Iterator> handleUpdate(String partitionPath, String fileId, Iterator> recordItr) { throw new UnsupportedOperationException("Should not called in bootstrap code path"); } -} +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index 667073931d8b5..0771dd59aa24f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -154,6 +154,10 @@ public static void createHoodieProperties(FileSystem fs, Path metadataFolder, Pr // Use the default bootstrap index class. properties.setProperty(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, getDefaultBootstrapIndexClass(properties)); } + + if (!properties.containsKey(HOODIE_BASE_FILE_FORMAT_PROP_NAME)) { + properties.setProperty(HOODIE_BASE_FILE_FORMAT_PROP_NAME, DEFAULT_BASE_FILE_FORMAT.name()); + } properties.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis())); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index cf66f16744a68..7b2e18a5ae94e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -725,6 +725,10 @@ public PropertyBuilder fromProperties(Properties properties) { if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)) { setTableCreateSchema(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)); } + if (properties.containsKey(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME)) { + setBaseFileFormat( + properties.getProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME)); + } return this; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java index 0f1f49fedc860..22efffb6ac5a8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java @@ -775,6 +775,68 @@ public static Schema createAvroSchema(TypeDescription orcSchema) { } } + + public static Schema createAvroSchemaWithNamespace(TypeDescription orcSchema, String recordName, String namespace) { + switch (orcSchema.getCategory()) { + case BOOLEAN: + return Schema.create(Schema.Type.BOOLEAN); + case BYTE: + // tinyint (8 bit), use int to hold it + return Schema.create(Schema.Type.INT); + case SHORT: + // smallint (16 bit), use int to hold it + return Schema.create(Schema.Type.INT); + case INT: + // the Avro logical type could be AvroTypeUtil.LOGICAL_TYPE_TIME_MILLIS, but there is no way to distinguish + return Schema.create(Schema.Type.INT); + case LONG: + // the Avro logical type could be AvroTypeUtil.LOGICAL_TYPE_TIME_MICROS, but there is no way to distinguish + return Schema.create(Schema.Type.LONG); + case FLOAT: + return Schema.create(Schema.Type.FLOAT); + case DOUBLE: + return Schema.create(Schema.Type.DOUBLE); + case VARCHAR: + case CHAR: + case STRING: + return Schema.create(Schema.Type.STRING); + case DATE: + Schema date = Schema.create(Schema.Type.INT); + LogicalTypes.date().addToSchema(date); + return date; + case TIMESTAMP: + // Cannot distinguish between TIMESTAMP_MILLIS and TIMESTAMP_MICROS + // Assume TIMESTAMP_MILLIS because Timestamp in ORC is in millis + Schema timestamp = Schema.create(Schema.Type.LONG); + LogicalTypes.timestampMillis().addToSchema(timestamp); + return timestamp; + case BINARY: + return Schema.create(Schema.Type.BYTES); + case DECIMAL: + Schema decimal = Schema.create(Schema.Type.BYTES); + LogicalTypes.decimal(orcSchema.getPrecision(), orcSchema.getScale()).addToSchema(decimal); + return decimal; + case LIST: + return Schema.createArray(createAvroSchema(orcSchema.getChildren().get(0))); + case MAP: + return Schema.createMap(createAvroSchema(orcSchema.getChildren().get(1))); + case STRUCT: + List childFields = new ArrayList<>(); + for (int i = 0; i < orcSchema.getChildren().size(); i++) { + TypeDescription childType = orcSchema.getChildren().get(i); + String childName = orcSchema.getFieldNames().get(i); + childFields.add(new Field(childName, createAvroSchema(childType), "", null)); + } + return Schema.createRecord(recordName,"",namespace ,false ,childFields); + case UNION: + return Schema.createUnion(orcSchema.getChildren().stream() + .map(AvroOrcUtils::createAvroSchema) + .collect(Collectors.toList())); + default: + throw new IllegalStateException(String.format("Unrecognized ORC type: %s", orcSchema.getCategory().getName())); + } + } + /** * Returns the actual schema of a field. * diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieSparkBootstrapWriteClientExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieSparkBootstrapWriteClientExample.java new file mode 100644 index 0000000000000..87b794c4da0f5 --- /dev/null +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieSparkBootstrapWriteClientExample.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.examples.java; + + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.client.HoodieJavaWriteClient; +import org.apache.hudi.client.SparkRDDWriteClient; +import org.apache.hudi.client.bootstrap.HoodieSparkBootstrapSchemaProvider; +import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector; +import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.bootstrap.FileStatusUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.AvroOrcUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieBootstrapConfig; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.examples.common.HoodieExampleDataGenerator; +import org.apache.hudi.examples.common.HoodieExampleSparkUtils; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.keygen.NonpartitionedKeyGenerator; +import org.apache.hudi.keygen.constant.KeyGeneratorOptions; +import org.apache.hudi.table.action.bootstrap.BootstrapUtils; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.schema.MessageType; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.avro.Schema; +import java.util.Properties; + +public class HoodieSparkBootstrapWriteClientExample { + private static String tableType = HoodieTableType.MERGE_ON_READ.name(); + + + public static void main(String[] args) throws Exception { + if (args.length < 3) { + System.err.println("Usage: HoodieWriteClientExample "); + System.exit(1); + } + String tablePath = args[0]; + String tableName = args[1]; + String fileFormat = args[2]; + String tableTy = args[3]; + + if(tableTy.equals("MOR")) + tableType = HoodieTableType.MERGE_ON_READ.name(); + else + tableType = HoodieTableType.COPY_ON_WRITE.name(); + + SparkConf sparkConf = HoodieExampleSparkUtils.defaultSparkConf("hoodie-client-example"); + + try (JavaSparkContext jsc = new JavaSparkContext(sparkConf)) { + + // Generator of some records to be loaded in. + HoodieExampleDataGenerator dataGen = new HoodieExampleDataGenerator<>(); + + // initialize the table, if not done already + Path path = new Path(tablePath); + + + Properties properties = HoodieTableMetaClient.withPropertyBuilder() + .setTableName(tableName) + .setTableType(tableType) + .setPayloadClass(HoodieAvroPayload.class) + .setBootstrapBasePath(tablePath) + .setBaseFileFormat(fileFormat) + .setPreCombineField("sno") + .build(); + HoodieTableMetaClient metaClient= HoodieTableMetaClient.initTableAndGetMetaClient(jsc.hadoopConfiguration(), tablePath, properties); + + + + String filePath = FileStatusUtils.toPath(BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), + tablePath, new HoodieSparkEngineContext(jsc)).stream().findAny().map(p -> p.getValue().stream().findAny()) + .orElse(null).get().getPath()).toString(); + + Schema schema=null; + + TypedProperties prop = new TypedProperties(); + prop.put(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY, "sno"); + + if(fileFormat.equals("orc")) { + String structName = tableName + "_record"; + String recordNamespace = "hoodie." + tableName; + Reader orcReader = OrcFile.createReader(new Path(filePath), OrcFile.readerOptions(jsc.hadoopConfiguration())); + schema = AvroOrcUtils.createAvroSchemaNew(orcReader.getSchema(), structName, recordNamespace); + } + else { + ParquetFileReader reader = ParquetFileReader.open(metaClient.getHadoopConf(), new Path(filePath)); + MessageType schema1 = reader.getFooter().getFileMetaData().getSchema(); + schema= new AvroSchemaConverter().convert(schema1); + } + + + + HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(tablePath) + .withAutoCommit(true) + .forTable("covid_test_orc") + .withSchema(schema.toString()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(1) + .build()) + .withProperties(prop) + .withBootstrapConfig(HoodieBootstrapConfig.newBuilder() + .withBootstrapBasePath(tablePath) + .withBootstrapKeyGenClass(NonpartitionedKeyGenerator.class.getCanonicalName()) + .withFullBootstrapInputProvider(HoodieSparkBootstrapSchemaProvider.class.getName()) + .withBootstrapParallelism(1) + .withBootstrapModeSelector(MetadataOnlyBootstrapModeSelector.class.getCanonicalName()).build()) + .build(); + SparkRDDWriteClient rddClient = new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jsc), config); + rddClient.bootstrap(Option.empty()); + + } + + } + +} diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java new file mode 100644 index 0000000000000..ab87b3f58701e --- /dev/null +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.examples.spark; + +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.AvroOrcUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieBootstrapConfig; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.examples.common.HoodieExampleDataGenerator; +import org.apache.hudi.examples.common.HoodieExampleSparkUtils; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.keygen.NonpartitionedKeyGenerator; +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.Dataset; + +import java.util.HashMap; +import java.util.Map; + +public class HoodieSparkBootstrapExample { + + private static String tableType = HoodieTableType.MERGE_ON_READ.name(); + + + public static void main(String[] args) throws Exception { + if (args.length < 3) { + System.err.println("Usage: HoodieWriteClientExample "); + System.exit(1); + } + String tablePath = args[0]; + String tableName = args[1]; + String fileFormat = args[2]; + String tableTy = args[3]; + + if (tableTy.equals("MOR")) + tableType = HoodieTableType.MERGE_ON_READ.name(); + else + tableType = HoodieTableType.COPY_ON_WRITE.name(); + + SparkConf sparkConf = HoodieExampleSparkUtils.defaultSparkConf("hoodie-client-example"); + + SparkSession spark = SparkSession + .builder() + .appName("Java Spark SQL basic example") + .config("spark.some.config.option", "some-value") + .enableHiveSupport() + .getOrCreate(); + + + Dataset df = spark.emptyDataFrame(); + + Map opts = new HashMap(); + opts.put("hoodie.table.base.file.format","ORC"); + + df.write().format("hudi").option(HoodieWriteConfig.TABLE_NAME, tableName) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL()) + .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "sno") + .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "observationdate") + .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "observationdate") + .option(HoodieBootstrapConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME, HoodieFileFormat.ORC.name()) + .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, "/user/hive/warehouse/"+tableName) + .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, NonpartitionedKeyGenerator.class.getCanonicalName()) + .mode(SaveMode.Overwrite).save("/hudi/"+tableName); + + df.count(); + + + } +} + diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkRegBootstrapExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkRegBootstrapExample.java new file mode 100644 index 0000000000000..911bf112e776e --- /dev/null +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkRegBootstrapExample.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hudi.examples.spark; + +import org.apache.hudi.bootstrap.SparkOrcBootstrapDataProvider; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.AvroOrcUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.config.HoodieBootstrapConfig; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieIndexConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.examples.common.HoodieExampleDataGenerator; +import org.apache.hudi.examples.common.HoodieExampleSparkUtils; +import org.apache.hudi.index.HoodieIndex; +import org.apache.hudi.keygen.NonpartitionedKeyGenerator; +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.spark.SparkConf; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.Dataset; +import org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider; +import org.apache.hudi.client.bootstrap.selector.BootstrapRegexModeSelector; +import org.apache.hudi. HoodieDataSourceHelpers; +import org.apache.hudi.config.HoodieBootstrapConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.spark.sql.SaveMode; + +import java.util.HashMap; +import java.util.Map; + +public class HoodieSparkRegBootstrapExample { + + private static String tableType = HoodieTableType.MERGE_ON_READ.name(); + + + public static void main(String[] args) throws Exception { + if (args.length < 3) { + System.err.println("Usage: HoodieWriteClientExample "); + System.exit(1); + } + String tablePath = args[0]; + String tableName = args[1]; + String fileFormat = args[2]; + String tableTy = args[3]; + + if (tableTy.equals("MOR")) + tableType = HoodieTableType.MERGE_ON_READ.name(); + else + tableType = HoodieTableType.COPY_ON_WRITE.name(); + + SparkConf sparkConf = HoodieExampleSparkUtils.defaultSparkConf("hoodie-client-example"); + + SparkSession spark = SparkSession + .builder() + .appName("Java Spark SQL basic example") + .config("spark.some.config.option", "some-value") + .enableHiveSupport() + .getOrCreate(); + + + Dataset df = spark.emptyDataFrame(); + + Map opts = new HashMap(); + opts.put("hoodie.table.base.file.format","ORC"); + + df.write().format("hudi").option(HoodieWriteConfig.TABLE_NAME, tableName) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL()) + .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "sno") + .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "observationdate") + .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "observationdate") + .option(HoodieBootstrapConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME, HoodieFileFormat.ORC.name()) + .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, "/user/hive/warehouse/"+tableName) + .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR, BootstrapRegexModeSelector.class.getCanonicalName()) + .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX, "2021/04/2[0-9]") + .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, "METADATA_ONLY") + .option(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER, SparkOrcBootstrapDataProvider.class.getCanonicalName()) + .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, SimpleKeyGenerator.class.getCanonicalName()) + .mode(SaveMode.Overwrite).save("/hudi/"+tableName); + + df.count(); + + + } +} diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 89faa3bbbf942..12135be3df6ed 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -81,6 +81,8 @@ object DataSourceReadOptions { @Deprecated val DEFAULT_VIEW_TYPE_OPT_VAL = VIEW_TYPE_READ_OPTIMIZED_OPT_VAL + val HOODIE_BASE_FILE_FORMAT_PROP_NAME = "hoodie.table.base.file.format" + /** * This eases migration from old configs to new configs. */ @@ -187,6 +189,9 @@ object DataSourceWriteOptions { @Deprecated val DEFAULT_STORAGE_TYPE_OPT_VAL = COW_STORAGE_TYPE_OPT_VAL + val HOODIE_BASE_FILE_FORMAT_PROP_NAME = "hoodie.table.base.file.format" + + def translateStorageTypeToTableType(optParams: Map[String, String]) : Map[String, String] = { if (optParams.contains(STORAGE_TYPE_OPT_KEY) && !optParams.contains(TABLE_TYPE_OPT_KEY)) { log.warn(STORAGE_TYPE_OPT_KEY + " is deprecated and will be removed in a later release; Please use " + TABLE_TYPE_OPT_KEY) diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkOrcBootstrapDataProvider.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkOrcBootstrapDataProvider.java new file mode 100644 index 0000000000000..56a92f8b9a109 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkOrcBootstrapDataProvider.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.bootstrap; + +import org.apache.hudi.DataSourceUtils; +import org.apache.hudi.HoodieSparkUtils; +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.bootstrap.FileStatusUtils; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.keygen.KeyGenerator; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.rdd.RDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.SparkSession; + +import java.io.IOException; +import java.util.List; + +/** + * Spark Data frame based bootstrap input provider. + */ +public class SparkOrcBootstrapDataProvider extends FullRecordBootstrapDataProvider> { + + private final transient SparkSession sparkSession; + + public SparkOrcBootstrapDataProvider(TypedProperties props, + HoodieSparkEngineContext context) { + super(props, context); + this.sparkSession = SparkSession.builder().config(context.getJavaSparkContext().getConf()).getOrCreate(); + } + + @Override + public JavaRDD generateInputRecords(String tableName, String sourceBasePath, + List>> partitionPathsWithFiles) { + String[] filePaths = partitionPathsWithFiles.stream().map(Pair::getValue) + .flatMap(f -> f.stream().map(fs -> FileStatusUtils.toPath(fs.getPath()).toString())) + .toArray(String[]::new); + + Dataset inputDataset = sparkSession.read().orc(filePaths); + try { + KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); + String structName = tableName + "_record"; + String namespace = "hoodie." + tableName; + RDD genericRecords = HoodieSparkUtils.createRdd(inputDataset, structName, namespace); + return genericRecords.toJavaRDD().map(gr -> { + String orderingVal = HoodieAvroUtils.getNestedFieldValAsString( + gr, props.getString("hoodie.datasource.write.precombine.field"), false); + try { + return DataSourceUtils.createHoodieRecord(gr, orderingVal, keyGenerator.getKey(gr), + props.getString("hoodie.datasource.write.payload.class")); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + }); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } +} \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 5d6ebd6204e28..7d681bff04db6 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -29,7 +29,7 @@ import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.HoodieWriteResult import org.apache.hudi.client.SparkRDDWriteClient import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties} -import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType} +import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType,HoodieFileFormat} import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.table.timeline.HoodieActiveTimeline import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils} @@ -300,6 +300,7 @@ object HoodieSparkSqlWriter { .setPayloadClassName(parameters(PAYLOAD_CLASS_OPT_KEY)) .setPreCombineField(parameters.getOrDefault(PRECOMBINE_FIELD_OPT_KEY, null)) .setBootstrapIndexClass(bootstrapIndexClass) + .setBaseFileFormat(parameters.getOrDefault(HOODIE_BASE_FILE_FORMAT_PROP_NAME, HoodieFileFormat.PARQUET.name)) .setBootstrapBasePath(bootstrapBasePath) .setPartitionColumns(partitionColumns) .initTable(sparkContext.hadoopConfiguration, path) diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java new file mode 100644 index 0000000000000..3cc887daab06d --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java @@ -0,0 +1,587 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hudi.client; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.LongWritable; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.bootstrap.BootstrapMode; +import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; +import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector; +import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector; +import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.bootstrap.FileStatusUtils; +import org.apache.hudi.common.bootstrap.index.BootstrapIndex; +import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieInstant.State; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.testutils.RawTripTestPayload; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieBootstrapConfig; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; +import org.apache.hudi.index.HoodieIndex.IndexType; +import org.apache.hudi.io.storage.HoodieOrcReader; +import org.apache.hudi.keygen.NonpartitionedKeyGenerator; +import org.apache.hudi.common.util.PartitionPathEncodeUtils; +import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.hudi.table.action.bootstrap.BootstrapUtils; +import org.apache.hudi.testutils.HoodieClientTestBase; +import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; +import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.avro.AvroReadSupport; +import org.apache.parquet.avro.AvroSchemaConverter; +import org.apache.parquet.hadoop.ParquetFileReader; +import org.apache.parquet.schema.MessageType; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.api.java.UDF1; +import org.apache.spark.sql.types.DataTypes; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.Spliterators; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.StreamSupport; + +import static java.util.stream.Collectors.mapping; +import static java.util.stream.Collectors.toList; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.generateGenericRecord; +import static org.apache.spark.sql.functions.callUDF; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests Bootstrap Client functionality. + */ +public class TestOrcBootstrap extends HoodieClientTestBase { + + public static final String TRIP_HIVE_COLUMN_TYPES = "bigint,string,string,string,double,double,double,double," + + "struct,array>,boolean"; + + @TempDir + public java.nio.file.Path tmpFolder; + + protected String bootstrapBasePath = null; + + private JobConf roJobConf; + + private HoodieParquetRealtimeInputFormat rtInputFormat; + private JobConf rtJobConf; + private SparkSession spark; + + @BeforeEach + public void setUp() throws Exception { + bootstrapBasePath = tmpFolder.toAbsolutePath().toString() + "/data"; + initPath(); + initSparkContexts(); + initTestDataGenerator(); + initMetaClient(); + // initialize parquet input format + //reloadInputFormats(); + } + + @AfterEach + public void tearDown() throws IOException { + cleanupSparkContexts(); + cleanupClients(); + cleanupTestDataGenerator(); + } +/* + + private void reloadInputFormats() { + roInputFormat = new HoodieParquetInputFormat(); + roJobConf = new JobConf(jsc.hadoopConfiguration()); + roInputFormat.setConf(roJobConf); + + rtInputFormat = new HoodieParquetRealtimeInputFormat(); + rtJobConf = new JobConf(jsc.hadoopConfiguration()); + rtInputFormat.setConf(rtJobConf); + } +*/ + + public Schema generateNewDataSetAndReturnSchema(long timestamp, int numRecords, List partitionPaths, + String srcPath) throws Exception { + boolean isPartitioned = partitionPaths != null && !partitionPaths.isEmpty(); + Dataset df = generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths, jsc, sqlContext); + df.printSchema(); + if (isPartitioned) { + df.write().partitionBy("datestr").format("orc").mode(SaveMode.Overwrite).save(srcPath); + } else { + df.write().format("orc").mode(SaveMode.Overwrite).save(srcPath); + } + String filePath = FileStatusUtils.toPath(BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), + srcPath, context).stream().findAny().map(p -> p.getValue().stream().findAny()) + .orElse(null).get().getPath()).toString(); + ParquetFileReader reader = ParquetFileReader.open(metaClient.getHadoopConf(), new Path(filePath)); + MessageType schema = reader.getFooter().getFileMetaData().getSchema(); + return new AvroSchemaConverter().convert(schema); + } + + @Test + public void testMetadataBootstrapUnpartitionedCOW() throws Exception { + testBootstrapCommon(false, false, EffectiveMode.METADATA_BOOTSTRAP_MODE); + } + + @Test + public void testMetadataBootstrapWithUpdatesCOW() throws Exception { + testBootstrapCommon(true, false, EffectiveMode.METADATA_BOOTSTRAP_MODE); + } + + private enum EffectiveMode { + FULL_BOOTSTRAP_MODE, + METADATA_BOOTSTRAP_MODE, + MIXED_BOOTSTRAP_MODE + } + + private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, EffectiveMode mode) throws Exception { + + if (deltaCommit) { + metaClient = HoodieTestUtils.init(basePath, HoodieTableType.MERGE_ON_READ, bootstrapBasePath); + } else { + metaClient = HoodieTestUtils.init(basePath, HoodieTableType.COPY_ON_WRITE, bootstrapBasePath); + } + + int totalRecords = 100; + String keyGeneratorClass = partitioned ? SimpleKeyGenerator.class.getCanonicalName() + : NonpartitionedKeyGenerator.class.getCanonicalName(); + final String bootstrapModeSelectorClass; + final String bootstrapCommitInstantTs; + final boolean checkNumRawFiles; + final boolean isBootstrapIndexCreated; + final int numInstantsAfterBootstrap; + final List bootstrapInstants; + switch (mode) { + case FULL_BOOTSTRAP_MODE: + bootstrapModeSelectorClass = FullRecordBootstrapModeSelector.class.getCanonicalName(); + bootstrapCommitInstantTs = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS; + checkNumRawFiles = false; + isBootstrapIndexCreated = false; + numInstantsAfterBootstrap = 1; + bootstrapInstants = Arrays.asList(bootstrapCommitInstantTs); + break; + case METADATA_BOOTSTRAP_MODE: + bootstrapModeSelectorClass = MetadataOnlyBootstrapModeSelector.class.getCanonicalName(); + bootstrapCommitInstantTs = HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS; + checkNumRawFiles = true; + isBootstrapIndexCreated = true; + numInstantsAfterBootstrap = 1; + bootstrapInstants = Arrays.asList(bootstrapCommitInstantTs); + break; + default: + bootstrapModeSelectorClass = TestRandomBootstapModeSelector.class.getName(); + bootstrapCommitInstantTs = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS; + checkNumRawFiles = false; + isBootstrapIndexCreated = true; + numInstantsAfterBootstrap = 2; + bootstrapInstants = Arrays.asList(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, + HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS); + break; + } + List partitions = Arrays.asList("2020/04/01", "2020/04/02", "2020/04/03"); + long timestamp = Instant.now().toEpochMilli(); + Schema schema = generateNewDataSetAndReturnSchema(timestamp, totalRecords, partitions, bootstrapBasePath); + HoodieWriteConfig config = getConfigBuilder(schema.toString()) + .withAutoCommit(true) + .withSchema(schema.toString()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(1) + .build()) + .withBaseFileFormat(HoodieFileFormat.ORC.name()) + .withBootstrapConfig(HoodieBootstrapConfig.newBuilder() + .withBootstrapBasePath(bootstrapBasePath) + .withBootstrapKeyGenClass(keyGeneratorClass) + .withFullBootstrapInputProvider(TestFullBootstrapDataProvider.class.getName()) + .withBootstrapParallelism(3) + .withBootstrapModeSelector(bootstrapModeSelectorClass).build()) + .build(); + SparkRDDWriteClient client = new SparkRDDWriteClient(context, config); + client.bootstrap(Option.empty()); + checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap, + numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); + + // Rollback Bootstrap + FSUtils.deleteInstantFile(metaClient.getFs(), metaClient.getMetaPath(), new HoodieInstant(State.COMPLETED, + deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, bootstrapCommitInstantTs)); + client.rollbackFailedBootstrap(); + metaClient.reloadActiveTimeline(); + assertEquals(0, metaClient.getCommitsTimeline().countInstants()); + assertEquals(0L, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath, context) + .stream().flatMap(f -> f.getValue().stream()).count()); + + BootstrapIndex index = BootstrapIndex.getBootstrapIndex(metaClient); + assertFalse(index.useIndex()); + + // Run bootstrap again + client = new SparkRDDWriteClient(context, config); + client.bootstrap(Option.empty()); + + metaClient.reloadActiveTimeline(); + index = BootstrapIndex.getBootstrapIndex(metaClient); + if (isBootstrapIndexCreated) { + assertTrue(index.useIndex()); + } else { + assertFalse(index.useIndex()); + } + + checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap, + numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); + + // Upsert case + long updateTimestamp = Instant.now().toEpochMilli(); + String updateSPath = tmpFolder.toAbsolutePath().toString() + "/data2"; + generateNewDataSetAndReturnSchema(updateTimestamp, totalRecords, partitions, updateSPath); + JavaRDD updateBatch = + generateInputBatch(jsc, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), updateSPath, context), + schema); + String newInstantTs = client.startCommit(); + client.upsert(updateBatch, newInstantTs); + checkBootstrapResults(totalRecords, schema, newInstantTs, false, numInstantsAfterBootstrap + 1, + updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit); + + if (deltaCommit) { + Option compactionInstant = client.scheduleCompaction(Option.empty()); + assertTrue(compactionInstant.isPresent()); + client.compact(compactionInstant.get()); + checkBootstrapResults(totalRecords, schema, compactionInstant.get(), checkNumRawFiles, + numInstantsAfterBootstrap + 2, 2, updateTimestamp, updateTimestamp, !deltaCommit, + Arrays.asList(compactionInstant.get())); + } + } + + @Test + public void testMetadataBootstrapWithUpdatesMOR() throws Exception { + testBootstrapCommon(true, true, EffectiveMode.METADATA_BOOTSTRAP_MODE); + } + + @Test + public void testFullBootstrapOnlyCOW() throws Exception { + testBootstrapCommon(true, false, EffectiveMode.FULL_BOOTSTRAP_MODE); + } + + @Test + public void testFullBootstrapWithUpdatesMOR() throws Exception { + testBootstrapCommon(true, true, EffectiveMode.FULL_BOOTSTRAP_MODE); + } + + @Test + public void testMetaAndFullBootstrapCOW() throws Exception { + testBootstrapCommon(true, false, EffectiveMode.MIXED_BOOTSTRAP_MODE); + } + + @Test + public void testMetadataAndFullBootstrapWithUpdatesMOR() throws Exception { + testBootstrapCommon(true, true, EffectiveMode.MIXED_BOOTSTRAP_MODE); + } + + private void checkBootstrapResults(int totalRecords, Schema schema, String maxInstant, boolean checkNumRawFiles, + int expNumInstants, long expTimestamp, long expROTimestamp, boolean isDeltaCommit) throws Exception { + checkBootstrapResults(totalRecords, schema, maxInstant, checkNumRawFiles, expNumInstants, expNumInstants, + expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant)); + } + + private void checkBootstrapResults(int totalRecords, Schema schema, String instant, boolean checkNumRawFiles, + int expNumInstants, int numVersions, long expTimestamp, long expROTimestamp, boolean isDeltaCommit, + List instantsWithValidRecords) throws Exception { + metaClient.reloadActiveTimeline(); + assertEquals(expNumInstants, metaClient.getCommitsTimeline().filterCompletedInstants().countInstants()); + assertEquals(instant, metaClient.getActiveTimeline() + .getCommitsTimeline().filterCompletedInstants().lastInstant().get().getTimestamp()); + + Dataset bootstrapped = sqlContext.read().format("orc").load(basePath); + Dataset original = sqlContext.read().format("orc").load(bootstrapBasePath); + bootstrapped.registerTempTable("bootstrapped"); + original.registerTempTable("original"); + if (checkNumRawFiles) { + List files = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), + bootstrapBasePath, context).stream().flatMap(x -> x.getValue().stream()).collect(Collectors.toList()); + assertEquals(files.size() * numVersions, + sqlContext.sql("select distinct _hoodie_file_name from bootstrapped").count()); + } + + if (!isDeltaCommit) { + String predicate = String.join(", ", + instantsWithValidRecords.stream().map(p -> "\"" + p + "\"").collect(Collectors.toList())); + assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN " + + "(" + predicate + ")").count()); + Dataset missingOriginal = sqlContext.sql("select a._row_key from original a where a._row_key not " + + "in (select _hoodie_record_key from bootstrapped)"); + assertEquals(0, missingOriginal.count()); + Dataset missingBootstrapped = sqlContext.sql("select a._hoodie_record_key from bootstrapped a " + + "where a._hoodie_record_key not in (select _row_key from original)"); + assertEquals(0, missingBootstrapped.count()); + //sqlContext.sql("select * from bootstrapped").show(10, false); + } + + // RO Input Format Read + //reloadInputFormats(); + List records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + jsc.hadoopConfiguration(), + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() + .map(f -> basePath + "/" + f).collect(Collectors.toList()), + basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); + assertEquals(totalRecords, records.size()); + Set seenKeys = new HashSet<>(); + for (GenericRecord r : records) { + assertEquals(r.get("_row_key").toString(), r.get("_hoodie_record_key").toString(), "Record :" + r); + assertEquals(expROTimestamp, ((LongWritable)r.get("timestamp")).get(), 0.1, "Record :" + r); + assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString())); + seenKeys.add(r.get("_hoodie_record_key").toString()); + } + assertEquals(totalRecords, seenKeys.size()); + + //RT Input Format Read + //reloadInputFormats(); + seenKeys = new HashSet<>(); + records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + jsc.hadoopConfiguration(), + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() + .map(f -> basePath + "/" + f).collect(Collectors.toList()), + basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); + assertEquals(totalRecords, records.size()); + for (GenericRecord r : records) { + assertEquals(r.get("_row_key").toString(), r.get("_hoodie_record_key").toString(), "Realtime Record :" + r); + assertEquals(expTimestamp, ((LongWritable)r.get("timestamp")).get(),0.1, "Realtime Record :" + r); + assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString())); + seenKeys.add(r.get("_hoodie_record_key").toString()); + } + assertEquals(totalRecords, seenKeys.size()); + + // RO Input Format Read - Project only Hoodie Columns + // reloadInputFormats(); + records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + jsc.hadoopConfiguration(), + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() + .map(f -> basePath + "/" + f).collect(Collectors.toList()), + basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, + true, HoodieRecord.HOODIE_META_COLUMNS); + assertEquals(totalRecords, records.size()); + seenKeys = new HashSet<>(); + for (GenericRecord r : records) { + assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString())); + seenKeys.add(r.get("_hoodie_record_key").toString()); + } + assertEquals(totalRecords, seenKeys.size()); + + //RT Input Format Read - Project only Hoodie Columns + // reloadInputFormats(); + seenKeys = new HashSet<>(); + records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + jsc.hadoopConfiguration(), + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() + .map(f -> basePath + "/" + f).collect(Collectors.toList()), + basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, + HoodieRecord.HOODIE_META_COLUMNS); + assertEquals(totalRecords, records.size()); + for (GenericRecord r : records) { + assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString())); + seenKeys.add(r.get("_hoodie_record_key").toString()); + } + assertEquals(totalRecords, seenKeys.size()); + + // RO Input Format Read - Project only non-hoodie column + //reloadInputFormats(); + records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + jsc.hadoopConfiguration(), + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() + .map(f -> basePath + "/" + f).collect(Collectors.toList()), + basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, + Arrays.asList("_row_key")); + assertEquals(totalRecords, records.size()); + seenKeys = new HashSet<>(); + for (GenericRecord r : records) { + assertFalse(seenKeys.contains(r.get("_row_key").toString())); + seenKeys.add(r.get("_row_key").toString()); + } + assertEquals(totalRecords, seenKeys.size()); + + //RT Input Format Read - Project only non-hoodie column + //reloadInputFormats(); + seenKeys = new HashSet<>(); + records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( + jsc.hadoopConfiguration(), + FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, + HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() + .map(f -> basePath + "/" + f).collect(Collectors.toList()), + basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, + Arrays.asList("_row_key")); + assertEquals(totalRecords, records.size()); + for (GenericRecord r : records) { + assertFalse(seenKeys.contains(r.get("_row_key").toString())); + seenKeys.add(r.get("_row_key").toString()); + } + assertEquals(totalRecords, seenKeys.size()); + } + + public static class TestFullBootstrapDataProvider extends FullRecordBootstrapDataProvider> { + + public TestFullBootstrapDataProvider(TypedProperties props, HoodieSparkEngineContext context) { + super(props, context); + } + + @Override + public JavaRDD generateInputRecords(String tableName, String sourceBasePath, + List>> partitionPaths) { + String filePath = FileStatusUtils.toPath(partitionPaths.stream().flatMap(p -> p.getValue().stream()) + .findAny().get().getPath()).toString(); + ParquetFileReader reader = null; + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + try { + reader = ParquetFileReader.open(jsc.hadoopConfiguration(), new Path(filePath)); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + MessageType parquetSchema = reader.getFooter().getFileMetaData().getSchema(); + Schema schema = new AvroSchemaConverter().convert(parquetSchema); + return generateInputBatch(jsc, partitionPaths, schema); + } + } + + private static JavaRDD generateInputBatch(JavaSparkContext jsc, + List>> partitionPaths, Schema writerSchema) { + List> fullFilePathsWithPartition = partitionPaths.stream().flatMap(p -> p.getValue().stream() + .map(x -> Pair.of(p.getKey(), FileStatusUtils.toPath(x.getPath())))).collect(Collectors.toList()); + return jsc.parallelize(fullFilePathsWithPartition.stream().flatMap(p -> { + try { + Configuration conf = jsc.hadoopConfiguration(); + AvroReadSupport.setAvroReadSchema(conf, writerSchema); + Iterator recIterator = new HoodieOrcReader(conf,p.getValue()).getRecordIterator(); + return StreamSupport.stream(Spliterators.spliteratorUnknownSize(recIterator, 0), false).map(gr -> { + try { + String key = gr.get("_row_key").toString(); + String pPath = p.getKey(); + return new HoodieRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath, + HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + }).collect(Collectors.toList())); + } + + public static class TestRandomBootstapModeSelector extends BootstrapModeSelector { + + private int currIdx = new Random().nextInt(2); + + public TestRandomBootstapModeSelector(HoodieWriteConfig writeConfig) { + super(writeConfig); + } + + @Override + public Map> select(List>> partitions) { + List> selections = new ArrayList<>(); + partitions.stream().forEach(p -> { + final BootstrapMode mode; + if (currIdx == 0) { + mode = BootstrapMode.METADATA_ONLY; + } else { + mode = BootstrapMode.FULL_RECORD; + } + currIdx = (currIdx + 1) % 2; + selections.add(Pair.of(mode, p.getKey())); + }); + return selections.stream().collect(Collectors.groupingBy(Pair::getKey, mapping(Pair::getValue, toList()))); + } + } + + public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) { + HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr, IndexType.BLOOM) + .withExternalSchemaTrasformation(true); + TypedProperties properties = new TypedProperties(); + properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key"); + properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "datestr"); + builder = builder.withProps(properties); + return builder; + } + + public static Dataset generateTestRawTripDataset(long timestamp, int from, int to, List partitionPaths, + JavaSparkContext jsc, SQLContext sqlContext) { + boolean isPartitioned = partitionPaths != null && !partitionPaths.isEmpty(); + final List records = new ArrayList<>(); + IntStream.range(from, to).forEach(i -> { + String id = "" + i; + records.add(generateGenericRecord("trip_" + id, "rider_" + id, "driver_" + id, + timestamp, false, false).toString()); + }); + if (isPartitioned) { + sqlContext.udf().register("partgen", + (UDF1) (val) -> PartitionPathEncodeUtils.escapePathName(partitionPaths.get( + Integer.parseInt(val.split("_")[1]) % partitionPaths.size())), + DataTypes.StringType); + } + JavaRDD rdd = jsc.parallelize(records); + Dataset df = sqlContext.read().json(rdd); + if (isPartitioned) { + df = df.withColumn("datestr", callUDF("partgen", new Column("_row_key"))); + // Order the columns to ensure generated avro schema aligns with Hive schema + df = df.select("timestamp", "_row_key", "rider", "driver", "begin_lat", "begin_lon", + "end_lat", "end_lon", "fare", "tip_history", "_hoodie_is_deleted", "datestr"); + } else { + // Order the columns to ensure generated avro schema aligns with Hive schema + df = df.select("timestamp", "_row_key", "rider", "driver", "begin_lat", "begin_lon", + "end_lat", "end_lon", "fare", "tip_history", "_hoodie_is_deleted"); + } + return df; + } +} \ No newline at end of file From 24891c7ee34741c59f692f3aedb02876641940e0 Mon Sep 17 00:00:00 2001 From: manasaks Date: Wed, 7 Jul 2021 00:29:34 +0530 Subject: [PATCH 2/6] Modified to support orcin bootstrap oper --- .../hudi/config/HoodieBootstrapConfig.java | 12 +++ .../apache/hudi/config/HoodieWriteConfig.java | 18 ++-- .../HoodieSparkBootstrapSchemaProvider.java | 6 +- .../SparkBootstrapCommitActionExecutor.java | 82 ++++++++++++++++++- .../common/table/HoodieTableMetaClient.java | 17 ++-- ...oodieSparkBootstrapWriteClientExample.java | 4 +- .../spark/HoodieSparkBootstrapExample.java | 16 ++-- .../spark/HoodieSparkRegBootstrapExample.java | 25 +++--- .../org/apache/hudi/DataSourceOptions.scala | 2 +- .../apache/hudi/client/TestOrcBootstrap.java | 18 ++-- 10 files changed, 147 insertions(+), 53 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java index 68846d7f34b9d..b2da411fcfc3a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex; import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.HoodieConfig; +import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.keygen.constant.KeyGeneratorType; @@ -99,6 +100,12 @@ public class HoodieBootstrapConfig extends HoodieConfig { .sinceVersion("0.6.0") .withDocumentation(""); + public static final ConfigProperty HOODIE_BASE_FILE_FORMAT_PROP = ConfigProperty + .key("hoodie.table.base.file.format") + .defaultValue(HoodieFileFormat.PARQUET) + .withAlternatives("hoodie.table.ro.file.format") + .withDocumentation(""); + private HoodieBootstrapConfig() { super(); } @@ -164,6 +171,11 @@ public Builder withBootstrapModeForRegexMatch(BootstrapMode modeForRegexMatch) { return this; } + public Builder withBootstrapBaseFileFormat(HoodieFileFormat fileFormat) { + bootstrapConfig.setDefaultValue(HOODIE_BASE_FILE_FORMAT_PROP, fileFormat); + return this; + } + public Builder fromProperties(Properties props) { this.bootstrapConfig.getProps().putAll(props); return this; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index a031d7b5d2679..9a74978e27f81 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -308,8 +308,11 @@ public class HoodieWriteConfig extends HoodieConfig { .withDocumentation("Comma separated metadata key prefixes to override from latest commit " + "during overlapping commits via multi writing"); - public static final String HOODIE_BASE_FILE_FORMAT_PROP_NAME = "hoodie.table.base.file.format"; - private static final String DEFAULT_TABLE_BASE_FILE_FORMAT = HoodieFileFormat.PARQUET.name(); + public static final ConfigProperty HOODIE_BASE_FILE_FORMAT_PROP = ConfigProperty + .key("hoodie.table.base.file.format") + .defaultValue(HoodieFileFormat.PARQUET) + .withAlternatives("hoodie.table.ro.file.format") + .withDocumentation(""); /** @@ -1234,8 +1237,8 @@ public WriteConcurrencyMode getWriteConcurrencyMode() { return WriteConcurrencyMode.fromValue(getString(WRITE_CONCURRENCY_MODE_PROP)); } - public String getHoodieBaseFileFormat() { - return props.getProperty(HOODIE_BASE_FILE_FORMAT_PROP_NAME); + public HoodieFileFormat getBaseFileFormat() { + return HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(); } @@ -1546,8 +1549,8 @@ public Builder withWriteMetaKeyPrefixes(String writeMetaKeyPrefixes) { return this; } - public Builder withBaseFileFormat(String format) { - props.setProperty(HOODIE_BASE_FILE_FORMAT_PROP_NAME, format); + public Builder withBaseFileFormat(HoodieFileFormat fileFormat) { + writeConfig.setDefaultValue(HOODIE_BASE_FILE_FORMAT_PROP, fileFormat); return this; } @@ -1590,9 +1593,6 @@ protected void setDefaults() { writeConfig.setDefaultValue(TIMELINE_LAYOUT_VERSION, String.valueOf(TimelineLayoutVersion.CURR_VERSION)); - setDefaultOnCondition(props, !props.containsKey(HOODIE_BASE_FILE_FORMAT_PROP_NAME), HOODIE_BASE_FILE_FORMAT_PROP_NAME, DEFAULT_TABLE_BASE_FILE_FORMAT); - - } private void validate() { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java index d6ced7697747d..04b82ce5d7402 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java @@ -63,11 +63,11 @@ protected Schema getBootstrapSourceSchema(HoodieEngineContext context, List new HoodieException("Could not determine schema from the data files.")); - if(writeConfig.getHoodieBaseFileFormat().equals(PARQUET.toString())) + if(writeConfig.getBaseFileFormat().equals(PARQUET)) { return getBootstrapSourceSchemaParquet(context,filePath); } - else if(writeConfig.getHoodieBaseFileFormat().equals(ORC.toString())) + else if(writeConfig.getBaseFileFormat().equals(ORC)) { return getBootstrapSourceSchemaOrc(context,filePath ); } @@ -103,7 +103,7 @@ private Schema getBootstrapSourceSchemaOrc(HoodieEngineContext context, Path fil String structName = tableName + "_record"; String recordNamespace = "hoodie." + tableName; - return AvroOrcUtils.createAvroSchemaNew(orcSchema,structName, recordNamespace); + return AvroOrcUtils.createAvroSchemaWithNamespace(orcSchema,structName, recordNamespace); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index 533b5caae9713..17b79db2da0be 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -100,6 +100,17 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import org.apache.orc.OrcFile; +import org.apache.orc.OrcProto.UserMetadataItem; +import org.apache.orc.Reader; +import org.apache.orc.Reader.Options; +import org.apache.orc.RecordReader; +import org.apache.orc.TypeDescription; +import org.apache.hudi.common.util.AvroOrcUtils; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.util.OrcReaderIterator; +import static org.apache.hudi.common.model.HoodieFileFormat.ORC; +import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; public class SparkBootstrapCommitActionExecutor> extends BaseCommitActionExecutor>, JavaRDD, JavaRDD, HoodieBootstrapWriteMetadata> { @@ -409,10 +420,79 @@ private JavaRDD runMetadataBootstrap(List handleMetadataBootstrap(partitionFsPair.getLeft(), partitionFsPair.getRight().getLeft(), + .map(partitionFsPair -> fileBasedMetadataHandler(partitionFsPair.getLeft(), partitionFsPair.getRight().getLeft(), partitionFsPair.getRight().getRight(), keyGenerator)); } + private BootstrapWriteStatus handleMetadataBootstrapOrc(String srcPartitionPath, String partitionPath, + HoodieFileStatus srcFileStatus, KeyGeneratorInterface keyGenerator) { + Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath()); + HoodieBootstrapHandle bootstrapHandle = new HoodieBootstrapHandle(config, + HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, table, partitionPath, FSUtils.createNewFileIdPfx(), + table.getTaskContextSupplier()); + Schema avroSchema = null; + try { + Reader orcReader = OrcFile.createReader(sourceFilePath, OrcFile.readerOptions(table.getHadoopConf())); + + TypeDescription orcSchema = orcReader.getSchema(); + + avroSchema= AvroOrcUtils.createAvroSchema(orcSchema); + + + Schema recordKeySchema = HoodieAvroUtils.generateProjectionSchema(avroSchema, + keyGenerator.getRecordKeyFieldNames()); + LOG.info("Schema to be used for reading record Keys :" + recordKeySchema); + AvroReadSupport.setAvroReadSchema(table.getHadoopConf(), recordKeySchema); + AvroReadSupport.setRequestedProjection(table.getHadoopConf(), recordKeySchema); + + + BoundedInMemoryExecutor wrapper = null; + try (RecordReader reader = orcReader.rows(new Options(table.getHadoopConf()).schema(orcSchema))) { + wrapper = new SparkBoundedInMemoryExecutor(config, + new OrcReaderIterator(reader,avroSchema,orcSchema), new BootstrapRecordConsumer(bootstrapHandle), inp -> { + String recKey = keyGenerator.getKey(inp).getRecordKey(); + GenericRecord gr = new GenericData.Record(HoodieAvroUtils.RECORD_KEY_SCHEMA); + gr.put(HoodieRecord.RECORD_KEY_METADATA_FIELD, recKey); + BootstrapRecordPayload payload = new BootstrapRecordPayload(gr); + HoodieRecord rec = new HoodieRecord(new HoodieKey(recKey, partitionPath), payload); + return rec; + }); + wrapper.execute(); + } catch (Exception e) { + throw new HoodieException(e); + } finally { + bootstrapHandle.close(); + if (null != wrapper) { + wrapper.shutdownNow(); + } + } + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + + BootstrapWriteStatus writeStatus = (BootstrapWriteStatus) bootstrapHandle.writeStatuses().get(0); + BootstrapFileMapping bootstrapFileMapping = new BootstrapFileMapping(config.getBootstrapSourceBasePath(), + srcPartitionPath, partitionPath, srcFileStatus, writeStatus.getFileId()); + writeStatus.setBootstrapSourceFileMapping(bootstrapFileMapping); + return writeStatus; + } + + private BootstrapWriteStatus fileBasedMetadataHandler(String srcPartitionPath, String partitionPath, + HoodieFileStatus srcFileStatus, KeyGeneratorInterface keyGenerator) + { + Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath()); + + String extension = FSUtils.getFileExtension(sourceFilePath.toString()); + if (ORC.getFileExtension().equals(extension)) + return handleMetadataBootstrapOrc(srcPartitionPath,partitionPath,srcFileStatus,keyGenerator); + if (PARQUET.getFileExtension().equals(extension)) + return handleMetadataBootstrap(srcPartitionPath,partitionPath,srcFileStatus,keyGenerator); + else + throw new HoodieIOException("Hoodie InputFormat not implemented for base file format " + extension); + + + } + @Override protected Iterator> handleInsert(String idPfx, Iterator> recordItr) { throw new UnsupportedOperationException("Should not called in bootstrap code path"); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 5e274c89dd434..066d7a38bf5df 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -650,12 +650,7 @@ public PropertyBuilder setTimelineLayoutVersion(Integer timelineLayoutVersion) { return this; } - public PropertyBuilder setBaseFileFormat(String baseFileFormat) { - this.baseFileFormat = baseFileFormat; - return this; - } - - public PropertyBuilder setPreCombineField(String preCombineField) { + public PropertyBuilder setPreCombineField(String preCombineField) { this.preCombineField = preCombineField; return this; } @@ -675,6 +670,11 @@ public PropertyBuilder setBootstrapBasePath(String bootstrapBasePath) { return this; } + public PropertyBuilder setBaseFileFormat(String baseFileFormat) { + this.baseFileFormat = baseFileFormat; + return this; + } + public PropertyBuilder fromMetaClient(HoodieTableMetaClient metaClient) { return setTableType(metaClient.getTableType()) .setTableName(metaClient.getTableConfig().getTableName()) @@ -725,9 +725,8 @@ public PropertyBuilder fromProperties(Properties properties) { if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)) { setTableCreateSchema(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)); } - if (properties.containsKey(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME)) { - setBaseFileFormat( - properties.getProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME)); + if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP)) { + setBaseFileFormat(hoodieConfig.getString(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP)); } return this; } diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieSparkBootstrapWriteClientExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieSparkBootstrapWriteClientExample.java index 87b794c4da0f5..b5a70728d24c0 100644 --- a/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieSparkBootstrapWriteClientExample.java +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieSparkBootstrapWriteClientExample.java @@ -32,6 +32,7 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.AvroOrcUtils; @@ -111,7 +112,7 @@ tablePath, new HoodieSparkEngineContext(jsc)).stream().findAny().map(p -> p.getV String structName = tableName + "_record"; String recordNamespace = "hoodie." + tableName; Reader orcReader = OrcFile.createReader(new Path(filePath), OrcFile.readerOptions(jsc.hadoopConfiguration())); - schema = AvroOrcUtils.createAvroSchemaNew(orcReader.getSchema(), structName, recordNamespace); + schema = AvroOrcUtils.createAvroSchemaWithNamespace(orcReader.getSchema(), structName, recordNamespace); } else { ParquetFileReader reader = ParquetFileReader.open(metaClient.getHadoopConf(), new Path(filePath)); @@ -129,6 +130,7 @@ tablePath, new HoodieSparkEngineContext(jsc)).stream().findAny().map(p -> p.getV .withMaxNumDeltaCommitsBeforeCompaction(1) .build()) .withProperties(prop) + .withBaseFileFormat(HoodieFileFormat.PARQUET) .withBootstrapConfig(HoodieBootstrapConfig.newBuilder() .withBootstrapBasePath(tablePath) .withBootstrapKeyGenClass(NonpartitionedKeyGenerator.class.getCanonicalName()) diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java index ab87b3f58701e..3c7a2b3db73e8 100644 --- a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkBootstrapExample.java @@ -76,14 +76,14 @@ public static void main(String[] args) throws Exception { Map opts = new HashMap(); opts.put("hoodie.table.base.file.format","ORC"); - df.write().format("hudi").option(HoodieWriteConfig.TABLE_NAME, tableName) - .option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL()) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "sno") - .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "observationdate") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "observationdate") - .option(HoodieBootstrapConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME, HoodieFileFormat.ORC.name()) - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, "/user/hive/warehouse/"+tableName) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, NonpartitionedKeyGenerator.class.getCanonicalName()) + df.write().format("hudi").option(HoodieWriteConfig.TABLE_NAME.key(), tableName) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY().key(), DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL()) + .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "sno") + .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "observationdate") + .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "observationdate") + .option(HoodieBootstrapConfig.HOODIE_BASE_FILE_FORMAT_PROP.key(), HoodieFileFormat.ORC.name()) + .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP.key(), "/user/hive/warehouse/"+tableName) + .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key(), NonpartitionedKeyGenerator.class.getCanonicalName()) .mode(SaveMode.Overwrite).save("/hudi/"+tableName); df.count(); diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkRegBootstrapExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkRegBootstrapExample.java index 911bf112e776e..9ebd7775dca0f 100644 --- a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkRegBootstrapExample.java +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkRegBootstrapExample.java @@ -18,6 +18,7 @@ package org.apache.hudi.examples.spark; import org.apache.hudi.bootstrap.SparkOrcBootstrapDataProvider; +import org.apache.hudi.client.bootstrap.BootstrapMode; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -83,18 +84,18 @@ public static void main(String[] args) throws Exception { Map opts = new HashMap(); opts.put("hoodie.table.base.file.format","ORC"); - df.write().format("hudi").option(HoodieWriteConfig.TABLE_NAME, tableName) - .option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL()) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "sno") - .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "observationdate") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "observationdate") - .option(HoodieBootstrapConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME, HoodieFileFormat.ORC.name()) - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, "/user/hive/warehouse/"+tableName) - .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR, BootstrapRegexModeSelector.class.getCanonicalName()) - .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX, "2021/04/2[0-9]") - .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, "METADATA_ONLY") - .option(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER, SparkOrcBootstrapDataProvider.class.getCanonicalName()) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, SimpleKeyGenerator.class.getCanonicalName()) + df.write().format("hudi").option(HoodieWriteConfig.TABLE_NAME.key(), tableName) + .option(DataSourceWriteOptions.OPERATION_OPT_KEY().key(), DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL()) + .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "sno") + .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "observationdate") + .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "observationdate") + .option(HoodieBootstrapConfig.HOODIE_BASE_FILE_FORMAT_PROP.key(), HoodieFileFormat.ORC.name()) + .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP.key(), "/user/hive/warehouse/"+tableName) + .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR.key(), BootstrapRegexModeSelector.class.getCanonicalName()) + .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX.key(), "2021/04/2[0-9]") + .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX_MODE.key(), BootstrapMode.METADATA_ONLY.name()) + .option(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER.key(), SparkOrcBootstrapDataProvider.class.getCanonicalName()) + .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key(), SimpleKeyGenerator.class.getCanonicalName()) .mode(SaveMode.Overwrite).save("/hudi/"+tableName); df.count(); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 437c5d2bcd8da..57cd48edc62ab 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -531,4 +531,4 @@ object DataSourceOptionsHelper { }) translatedOpt.toMap } -} \ No newline at end of file +} diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java index 3cc887daab06d..f8aaa37f8d4bc 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java @@ -240,7 +240,7 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec .withCompactionConfig(HoodieCompactionConfig.newBuilder() .withMaxNumDeltaCommitsBeforeCompaction(1) .build()) - .withBaseFileFormat(HoodieFileFormat.ORC.name()) + .withBaseFileFormat(HoodieFileFormat.ORC) .withBootstrapConfig(HoodieBootstrapConfig.newBuilder() .withBootstrapBasePath(bootstrapBasePath) .withBootstrapKeyGenClass(keyGeneratorClass) @@ -371,7 +371,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta List records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() + HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); assertEquals(totalRecords, records.size()); @@ -390,7 +390,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() + HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); assertEquals(totalRecords, records.size()); @@ -407,7 +407,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() + HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, HoodieRecord.HOODIE_META_COLUMNS); @@ -425,7 +425,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() + HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, HoodieRecord.HOODIE_META_COLUMNS); @@ -441,7 +441,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() + HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, Arrays.asList("_row_key")); @@ -459,7 +459,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( jsc.hadoopConfiguration(), FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream() + HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() .map(f -> basePath + "/" + f).collect(Collectors.toList()), basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, Arrays.asList("_row_key")); @@ -549,8 +549,8 @@ public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) { HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr, IndexType.BLOOM) .withExternalSchemaTrasformation(true); TypedProperties properties = new TypedProperties(); - properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key"); - properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "datestr"); + properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key"); + properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "datestr"); builder = builder.withProps(properties); return builder; } From 1ef7dcc95ba16a8c9f296335b3f313563090bffc Mon Sep 17 00:00:00 2001 From: manasaks Date: Sat, 10 Jul 2021 22:34:21 +0530 Subject: [PATCH 3/6] Support bootstrap oper --- .../java/org/apache/hudi/config/HoodieBootstrapConfig.java | 2 +- .../bootstrap/HoodieSparkBootstrapSchemaProvider.java | 6 +++--- .../org/apache/hudi/common/table/HoodieTableConfig.java | 1 + .../apache/hudi/common/table/HoodieTableMetaClient.java | 4 +--- .../src/main/scala/org/apache/hudi/DataSourceOptions.scala | 7 ++++++- .../main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala | 1 + 6 files changed, 13 insertions(+), 8 deletions(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java index b2da411fcfc3a..ea1fd566b7fa1 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieBootstrapConfig.java @@ -171,7 +171,7 @@ public Builder withBootstrapModeForRegexMatch(BootstrapMode modeForRegexMatch) { return this; } - public Builder withBootstrapBaseFileFormat(HoodieFileFormat fileFormat) { + public Builder withBaseFileFormat(HoodieFileFormat fileFormat) { bootstrapConfig.setDefaultValue(HOODIE_BASE_FILE_FORMAT_PROP, fileFormat); return this; } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java index 04b82ce5d7402..040d005348b1b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java @@ -62,12 +62,12 @@ protected Schema getBootstrapSourceSchema(HoodieEngineContext context, List new HoodieException("Could not determine schema from the data files.")); - - if(writeConfig.getBaseFileFormat().equals(PARQUET)) + String fileExtension = FSUtils.getFileExtension(filePath.getName()); + if(fileExtension.equals(PARQUET.getFileExtension())) { return getBootstrapSourceSchemaParquet(context,filePath); } - else if(writeConfig.getBaseFileFormat().equals(ORC)) + else if(fileExtension.equals(ORC.getFileExtension())) { return getBootstrapSourceSchemaOrc(context,filePath ); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index 9f3046f8812f5..934d36568ea75 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -266,6 +266,7 @@ public static String getDefaultBootstrapIndexClass(Properties props) { return defaultClass; } + public Option getBootstrapBasePath() { return Option.ofNullable(getString(HOODIE_BOOTSTRAP_BASE_PATH_PROP)); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 066d7a38bf5df..e94af058c74bf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -725,9 +725,7 @@ public PropertyBuilder fromProperties(Properties properties) { if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)) { setTableCreateSchema(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)); } - if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP)) { - setBaseFileFormat(hoodieConfig.getString(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP)); - } + return this; } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index 57cd48edc62ab..c856d559479b3 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -19,7 +19,7 @@ package org.apache.hudi import org.apache.hudi.common.config.ConfigProperty import org.apache.hudi.common.fs.ConsistencyGuardConfig -import org.apache.hudi.common.model.{HoodieTableType, WriteOperationType} +import org.apache.hudi.common.model.{HoodieTableType, WriteOperationType , HoodieFileFormat} import org.apache.hudi.common.table.HoodieTableConfig import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.hive.{HiveSyncTool, SlashEncodedDayPartitionValueExtractor} @@ -82,6 +82,8 @@ object DataSourceReadOptions { .defaultValue(true) .withDocumentation("") + val HOODIE_BASE_FILE_FORMAT_PROP = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP + @Deprecated val VIEW_TYPE_OPT_KEY = "hoodie.datasource.view.type" @Deprecated @@ -201,6 +203,9 @@ object DataSourceWriteOptions { @Deprecated val DEFAULT_STORAGE_TYPE_OPT_VAL = COW_STORAGE_TYPE_OPT_VAL + val HOODIE_BASE_FILE_FORMAT_PROP = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP + + /** * Translate spark parameters to hudi parameters * diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index ef18ed80c5151..50b9db4f9ba99 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -298,6 +298,7 @@ object HoodieSparkSqlWriter { .setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD_OPT_KEY, null)) .setBootstrapIndexClass(bootstrapIndexClass) .setBootstrapBasePath(bootstrapBasePath) + .setBaseFileFormat(hoodieConfig.getStringOrDefault(HOODIE_BASE_FILE_FORMAT_PROP)) .setPartitionColumns(partitionColumns) .initTable(sparkContext.hadoopConfiguration, path) } From ce5e820c64950df9009bf9eb501a3a18dbc1ba8b Mon Sep 17 00:00:00 2001 From: manasaks Date: Sun, 11 Jul 2021 00:06:04 +0530 Subject: [PATCH 4/6] Delete Test Files --- .../spark/HoodieSparkRegBootstrapExample.java | 105 ---- .../apache/hudi/client/TestOrcBootstrap.java | 587 ------------------ 2 files changed, 692 deletions(-) delete mode 100644 hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkRegBootstrapExample.java delete mode 100644 hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkRegBootstrapExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkRegBootstrapExample.java deleted file mode 100644 index 9ebd7775dca0f..0000000000000 --- a/hudi-examples/src/main/java/org/apache/hudi/examples/spark/HoodieSparkRegBootstrapExample.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hudi.examples.spark; - -import org.apache.hudi.bootstrap.SparkOrcBootstrapDataProvider; -import org.apache.hudi.client.bootstrap.BootstrapMode; -import org.apache.hudi.common.model.HoodieFileFormat; -import org.apache.hudi.common.model.HoodieTableType; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.util.AvroOrcUtils; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.config.HoodieBootstrapConfig; -import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieIndexConfig; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.examples.common.HoodieExampleDataGenerator; -import org.apache.hudi.examples.common.HoodieExampleSparkUtils; -import org.apache.hudi.index.HoodieIndex; -import org.apache.hudi.keygen.NonpartitionedKeyGenerator; -import org.apache.hudi.DataSourceWriteOptions; -import org.apache.spark.SparkConf; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.Dataset; -import org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider; -import org.apache.hudi.client.bootstrap.selector.BootstrapRegexModeSelector; -import org.apache.hudi. HoodieDataSourceHelpers; -import org.apache.hudi.config.HoodieBootstrapConfig; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.keygen.SimpleKeyGenerator; -import org.apache.spark.sql.SaveMode; - -import java.util.HashMap; -import java.util.Map; - -public class HoodieSparkRegBootstrapExample { - - private static String tableType = HoodieTableType.MERGE_ON_READ.name(); - - - public static void main(String[] args) throws Exception { - if (args.length < 3) { - System.err.println("Usage: HoodieWriteClientExample "); - System.exit(1); - } - String tablePath = args[0]; - String tableName = args[1]; - String fileFormat = args[2]; - String tableTy = args[3]; - - if (tableTy.equals("MOR")) - tableType = HoodieTableType.MERGE_ON_READ.name(); - else - tableType = HoodieTableType.COPY_ON_WRITE.name(); - - SparkConf sparkConf = HoodieExampleSparkUtils.defaultSparkConf("hoodie-client-example"); - - SparkSession spark = SparkSession - .builder() - .appName("Java Spark SQL basic example") - .config("spark.some.config.option", "some-value") - .enableHiveSupport() - .getOrCreate(); - - - Dataset df = spark.emptyDataFrame(); - - Map opts = new HashMap(); - opts.put("hoodie.table.base.file.format","ORC"); - - df.write().format("hudi").option(HoodieWriteConfig.TABLE_NAME.key(), tableName) - .option(DataSourceWriteOptions.OPERATION_OPT_KEY().key(), DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL()) - .option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "sno") - .option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "observationdate") - .option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "observationdate") - .option(HoodieBootstrapConfig.HOODIE_BASE_FILE_FORMAT_PROP.key(), HoodieFileFormat.ORC.name()) - .option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP.key(), "/user/hive/warehouse/"+tableName) - .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR.key(), BootstrapRegexModeSelector.class.getCanonicalName()) - .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX.key(), "2021/04/2[0-9]") - .option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR_REGEX_MODE.key(), BootstrapMode.METADATA_ONLY.name()) - .option(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER.key(), SparkOrcBootstrapDataProvider.class.getCanonicalName()) - .option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key(), SimpleKeyGenerator.class.getCanonicalName()) - .mode(SaveMode.Overwrite).save("/hudi/"+tableName); - - df.count(); - - - } -} diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java deleted file mode 100644 index f8aaa37f8d4bc..0000000000000 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java +++ /dev/null @@ -1,587 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.hudi.client; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.io.LongWritable; -import org.apache.hadoop.mapred.JobConf; -import org.apache.hudi.DataSourceWriteOptions; -import org.apache.hudi.avro.model.HoodieFileStatus; -import org.apache.hudi.client.bootstrap.BootstrapMode; -import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; -import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector; -import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector; -import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector; -import org.apache.hudi.client.common.HoodieSparkEngineContext; -import org.apache.hudi.common.bootstrap.FileStatusUtils; -import org.apache.hudi.common.bootstrap.index.BootstrapIndex; -import org.apache.hudi.common.config.HoodieMetadataConfig; -import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.model.HoodieFileFormat; -import org.apache.hudi.common.model.HoodieKey; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieTableType; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieInstant.State; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.testutils.HoodieTestDataGenerator; -import org.apache.hudi.common.testutils.HoodieTestUtils; -import org.apache.hudi.common.testutils.RawTripTestPayload; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieBootstrapConfig; -import org.apache.hudi.config.HoodieCompactionConfig; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; -import org.apache.hudi.index.HoodieIndex.IndexType; -import org.apache.hudi.io.storage.HoodieOrcReader; -import org.apache.hudi.keygen.NonpartitionedKeyGenerator; -import org.apache.hudi.common.util.PartitionPathEncodeUtils; -import org.apache.hudi.keygen.SimpleKeyGenerator; -import org.apache.hudi.table.action.bootstrap.BootstrapUtils; -import org.apache.hudi.testutils.HoodieClientTestBase; -import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; -import org.apache.parquet.avro.AvroParquetReader; -import org.apache.parquet.avro.AvroReadSupport; -import org.apache.parquet.avro.AvroSchemaConverter; -import org.apache.parquet.hadoop.ParquetFileReader; -import org.apache.parquet.schema.MessageType; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Column; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SQLContext; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.api.java.UDF1; -import org.apache.spark.sql.types.DataTypes; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -import java.io.IOException; -import java.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.Spliterators; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import java.util.stream.StreamSupport; - -import static java.util.stream.Collectors.mapping; -import static java.util.stream.Collectors.toList; -import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.generateGenericRecord; -import static org.apache.spark.sql.functions.callUDF; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; - -/** - * Tests Bootstrap Client functionality. - */ -public class TestOrcBootstrap extends HoodieClientTestBase { - - public static final String TRIP_HIVE_COLUMN_TYPES = "bigint,string,string,string,double,double,double,double," - + "struct,array>,boolean"; - - @TempDir - public java.nio.file.Path tmpFolder; - - protected String bootstrapBasePath = null; - - private JobConf roJobConf; - - private HoodieParquetRealtimeInputFormat rtInputFormat; - private JobConf rtJobConf; - private SparkSession spark; - - @BeforeEach - public void setUp() throws Exception { - bootstrapBasePath = tmpFolder.toAbsolutePath().toString() + "/data"; - initPath(); - initSparkContexts(); - initTestDataGenerator(); - initMetaClient(); - // initialize parquet input format - //reloadInputFormats(); - } - - @AfterEach - public void tearDown() throws IOException { - cleanupSparkContexts(); - cleanupClients(); - cleanupTestDataGenerator(); - } -/* - - private void reloadInputFormats() { - roInputFormat = new HoodieParquetInputFormat(); - roJobConf = new JobConf(jsc.hadoopConfiguration()); - roInputFormat.setConf(roJobConf); - - rtInputFormat = new HoodieParquetRealtimeInputFormat(); - rtJobConf = new JobConf(jsc.hadoopConfiguration()); - rtInputFormat.setConf(rtJobConf); - } -*/ - - public Schema generateNewDataSetAndReturnSchema(long timestamp, int numRecords, List partitionPaths, - String srcPath) throws Exception { - boolean isPartitioned = partitionPaths != null && !partitionPaths.isEmpty(); - Dataset df = generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths, jsc, sqlContext); - df.printSchema(); - if (isPartitioned) { - df.write().partitionBy("datestr").format("orc").mode(SaveMode.Overwrite).save(srcPath); - } else { - df.write().format("orc").mode(SaveMode.Overwrite).save(srcPath); - } - String filePath = FileStatusUtils.toPath(BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), - srcPath, context).stream().findAny().map(p -> p.getValue().stream().findAny()) - .orElse(null).get().getPath()).toString(); - ParquetFileReader reader = ParquetFileReader.open(metaClient.getHadoopConf(), new Path(filePath)); - MessageType schema = reader.getFooter().getFileMetaData().getSchema(); - return new AvroSchemaConverter().convert(schema); - } - - @Test - public void testMetadataBootstrapUnpartitionedCOW() throws Exception { - testBootstrapCommon(false, false, EffectiveMode.METADATA_BOOTSTRAP_MODE); - } - - @Test - public void testMetadataBootstrapWithUpdatesCOW() throws Exception { - testBootstrapCommon(true, false, EffectiveMode.METADATA_BOOTSTRAP_MODE); - } - - private enum EffectiveMode { - FULL_BOOTSTRAP_MODE, - METADATA_BOOTSTRAP_MODE, - MIXED_BOOTSTRAP_MODE - } - - private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, EffectiveMode mode) throws Exception { - - if (deltaCommit) { - metaClient = HoodieTestUtils.init(basePath, HoodieTableType.MERGE_ON_READ, bootstrapBasePath); - } else { - metaClient = HoodieTestUtils.init(basePath, HoodieTableType.COPY_ON_WRITE, bootstrapBasePath); - } - - int totalRecords = 100; - String keyGeneratorClass = partitioned ? SimpleKeyGenerator.class.getCanonicalName() - : NonpartitionedKeyGenerator.class.getCanonicalName(); - final String bootstrapModeSelectorClass; - final String bootstrapCommitInstantTs; - final boolean checkNumRawFiles; - final boolean isBootstrapIndexCreated; - final int numInstantsAfterBootstrap; - final List bootstrapInstants; - switch (mode) { - case FULL_BOOTSTRAP_MODE: - bootstrapModeSelectorClass = FullRecordBootstrapModeSelector.class.getCanonicalName(); - bootstrapCommitInstantTs = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS; - checkNumRawFiles = false; - isBootstrapIndexCreated = false; - numInstantsAfterBootstrap = 1; - bootstrapInstants = Arrays.asList(bootstrapCommitInstantTs); - break; - case METADATA_BOOTSTRAP_MODE: - bootstrapModeSelectorClass = MetadataOnlyBootstrapModeSelector.class.getCanonicalName(); - bootstrapCommitInstantTs = HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS; - checkNumRawFiles = true; - isBootstrapIndexCreated = true; - numInstantsAfterBootstrap = 1; - bootstrapInstants = Arrays.asList(bootstrapCommitInstantTs); - break; - default: - bootstrapModeSelectorClass = TestRandomBootstapModeSelector.class.getName(); - bootstrapCommitInstantTs = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS; - checkNumRawFiles = false; - isBootstrapIndexCreated = true; - numInstantsAfterBootstrap = 2; - bootstrapInstants = Arrays.asList(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, - HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS); - break; - } - List partitions = Arrays.asList("2020/04/01", "2020/04/02", "2020/04/03"); - long timestamp = Instant.now().toEpochMilli(); - Schema schema = generateNewDataSetAndReturnSchema(timestamp, totalRecords, partitions, bootstrapBasePath); - HoodieWriteConfig config = getConfigBuilder(schema.toString()) - .withAutoCommit(true) - .withSchema(schema.toString()) - .withCompactionConfig(HoodieCompactionConfig.newBuilder() - .withMaxNumDeltaCommitsBeforeCompaction(1) - .build()) - .withBaseFileFormat(HoodieFileFormat.ORC) - .withBootstrapConfig(HoodieBootstrapConfig.newBuilder() - .withBootstrapBasePath(bootstrapBasePath) - .withBootstrapKeyGenClass(keyGeneratorClass) - .withFullBootstrapInputProvider(TestFullBootstrapDataProvider.class.getName()) - .withBootstrapParallelism(3) - .withBootstrapModeSelector(bootstrapModeSelectorClass).build()) - .build(); - SparkRDDWriteClient client = new SparkRDDWriteClient(context, config); - client.bootstrap(Option.empty()); - checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap, - numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); - - // Rollback Bootstrap - FSUtils.deleteInstantFile(metaClient.getFs(), metaClient.getMetaPath(), new HoodieInstant(State.COMPLETED, - deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, bootstrapCommitInstantTs)); - client.rollbackFailedBootstrap(); - metaClient.reloadActiveTimeline(); - assertEquals(0, metaClient.getCommitsTimeline().countInstants()); - assertEquals(0L, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath, context) - .stream().flatMap(f -> f.getValue().stream()).count()); - - BootstrapIndex index = BootstrapIndex.getBootstrapIndex(metaClient); - assertFalse(index.useIndex()); - - // Run bootstrap again - client = new SparkRDDWriteClient(context, config); - client.bootstrap(Option.empty()); - - metaClient.reloadActiveTimeline(); - index = BootstrapIndex.getBootstrapIndex(metaClient); - if (isBootstrapIndexCreated) { - assertTrue(index.useIndex()); - } else { - assertFalse(index.useIndex()); - } - - checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap, - numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); - - // Upsert case - long updateTimestamp = Instant.now().toEpochMilli(); - String updateSPath = tmpFolder.toAbsolutePath().toString() + "/data2"; - generateNewDataSetAndReturnSchema(updateTimestamp, totalRecords, partitions, updateSPath); - JavaRDD updateBatch = - generateInputBatch(jsc, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), updateSPath, context), - schema); - String newInstantTs = client.startCommit(); - client.upsert(updateBatch, newInstantTs); - checkBootstrapResults(totalRecords, schema, newInstantTs, false, numInstantsAfterBootstrap + 1, - updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit); - - if (deltaCommit) { - Option compactionInstant = client.scheduleCompaction(Option.empty()); - assertTrue(compactionInstant.isPresent()); - client.compact(compactionInstant.get()); - checkBootstrapResults(totalRecords, schema, compactionInstant.get(), checkNumRawFiles, - numInstantsAfterBootstrap + 2, 2, updateTimestamp, updateTimestamp, !deltaCommit, - Arrays.asList(compactionInstant.get())); - } - } - - @Test - public void testMetadataBootstrapWithUpdatesMOR() throws Exception { - testBootstrapCommon(true, true, EffectiveMode.METADATA_BOOTSTRAP_MODE); - } - - @Test - public void testFullBootstrapOnlyCOW() throws Exception { - testBootstrapCommon(true, false, EffectiveMode.FULL_BOOTSTRAP_MODE); - } - - @Test - public void testFullBootstrapWithUpdatesMOR() throws Exception { - testBootstrapCommon(true, true, EffectiveMode.FULL_BOOTSTRAP_MODE); - } - - @Test - public void testMetaAndFullBootstrapCOW() throws Exception { - testBootstrapCommon(true, false, EffectiveMode.MIXED_BOOTSTRAP_MODE); - } - - @Test - public void testMetadataAndFullBootstrapWithUpdatesMOR() throws Exception { - testBootstrapCommon(true, true, EffectiveMode.MIXED_BOOTSTRAP_MODE); - } - - private void checkBootstrapResults(int totalRecords, Schema schema, String maxInstant, boolean checkNumRawFiles, - int expNumInstants, long expTimestamp, long expROTimestamp, boolean isDeltaCommit) throws Exception { - checkBootstrapResults(totalRecords, schema, maxInstant, checkNumRawFiles, expNumInstants, expNumInstants, - expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant)); - } - - private void checkBootstrapResults(int totalRecords, Schema schema, String instant, boolean checkNumRawFiles, - int expNumInstants, int numVersions, long expTimestamp, long expROTimestamp, boolean isDeltaCommit, - List instantsWithValidRecords) throws Exception { - metaClient.reloadActiveTimeline(); - assertEquals(expNumInstants, metaClient.getCommitsTimeline().filterCompletedInstants().countInstants()); - assertEquals(instant, metaClient.getActiveTimeline() - .getCommitsTimeline().filterCompletedInstants().lastInstant().get().getTimestamp()); - - Dataset bootstrapped = sqlContext.read().format("orc").load(basePath); - Dataset original = sqlContext.read().format("orc").load(bootstrapBasePath); - bootstrapped.registerTempTable("bootstrapped"); - original.registerTempTable("original"); - if (checkNumRawFiles) { - List files = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), - bootstrapBasePath, context).stream().flatMap(x -> x.getValue().stream()).collect(Collectors.toList()); - assertEquals(files.size() * numVersions, - sqlContext.sql("select distinct _hoodie_file_name from bootstrapped").count()); - } - - if (!isDeltaCommit) { - String predicate = String.join(", ", - instantsWithValidRecords.stream().map(p -> "\"" + p + "\"").collect(Collectors.toList())); - assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN " - + "(" + predicate + ")").count()); - Dataset missingOriginal = sqlContext.sql("select a._row_key from original a where a._row_key not " - + "in (select _hoodie_record_key from bootstrapped)"); - assertEquals(0, missingOriginal.count()); - Dataset missingBootstrapped = sqlContext.sql("select a._hoodie_record_key from bootstrapped a " - + "where a._hoodie_record_key not in (select _row_key from original)"); - assertEquals(0, missingBootstrapped.count()); - //sqlContext.sql("select * from bootstrapped").show(10, false); - } - - // RO Input Format Read - //reloadInputFormats(); - List records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( - jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() - .map(f -> basePath + "/" + f).collect(Collectors.toList()), - basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); - assertEquals(totalRecords, records.size()); - Set seenKeys = new HashSet<>(); - for (GenericRecord r : records) { - assertEquals(r.get("_row_key").toString(), r.get("_hoodie_record_key").toString(), "Record :" + r); - assertEquals(expROTimestamp, ((LongWritable)r.get("timestamp")).get(), 0.1, "Record :" + r); - assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString())); - seenKeys.add(r.get("_hoodie_record_key").toString()); - } - assertEquals(totalRecords, seenKeys.size()); - - //RT Input Format Read - //reloadInputFormats(); - seenKeys = new HashSet<>(); - records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( - jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() - .map(f -> basePath + "/" + f).collect(Collectors.toList()), - basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>()); - assertEquals(totalRecords, records.size()); - for (GenericRecord r : records) { - assertEquals(r.get("_row_key").toString(), r.get("_hoodie_record_key").toString(), "Realtime Record :" + r); - assertEquals(expTimestamp, ((LongWritable)r.get("timestamp")).get(),0.1, "Realtime Record :" + r); - assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString())); - seenKeys.add(r.get("_hoodie_record_key").toString()); - } - assertEquals(totalRecords, seenKeys.size()); - - // RO Input Format Read - Project only Hoodie Columns - // reloadInputFormats(); - records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( - jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() - .map(f -> basePath + "/" + f).collect(Collectors.toList()), - basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, - true, HoodieRecord.HOODIE_META_COLUMNS); - assertEquals(totalRecords, records.size()); - seenKeys = new HashSet<>(); - for (GenericRecord r : records) { - assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString())); - seenKeys.add(r.get("_hoodie_record_key").toString()); - } - assertEquals(totalRecords, seenKeys.size()); - - //RT Input Format Read - Project only Hoodie Columns - // reloadInputFormats(); - seenKeys = new HashSet<>(); - records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( - jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() - .map(f -> basePath + "/" + f).collect(Collectors.toList()), - basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, - HoodieRecord.HOODIE_META_COLUMNS); - assertEquals(totalRecords, records.size()); - for (GenericRecord r : records) { - assertFalse(seenKeys.contains(r.get("_hoodie_record_key").toString())); - seenKeys.add(r.get("_hoodie_record_key").toString()); - } - assertEquals(totalRecords, seenKeys.size()); - - // RO Input Format Read - Project only non-hoodie column - //reloadInputFormats(); - records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( - jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() - .map(f -> basePath + "/" + f).collect(Collectors.toList()), - basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true, - Arrays.asList("_row_key")); - assertEquals(totalRecords, records.size()); - seenKeys = new HashSet<>(); - for (GenericRecord r : records) { - assertFalse(seenKeys.contains(r.get("_row_key").toString())); - seenKeys.add(r.get("_row_key").toString()); - } - assertEquals(totalRecords, seenKeys.size()); - - //RT Input Format Read - Project only non-hoodie column - //reloadInputFormats(); - seenKeys = new HashSet<>(); - records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat( - jsc.hadoopConfiguration(), - FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS, - HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream() - .map(f -> basePath + "/" + f).collect(Collectors.toList()), - basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true, - Arrays.asList("_row_key")); - assertEquals(totalRecords, records.size()); - for (GenericRecord r : records) { - assertFalse(seenKeys.contains(r.get("_row_key").toString())); - seenKeys.add(r.get("_row_key").toString()); - } - assertEquals(totalRecords, seenKeys.size()); - } - - public static class TestFullBootstrapDataProvider extends FullRecordBootstrapDataProvider> { - - public TestFullBootstrapDataProvider(TypedProperties props, HoodieSparkEngineContext context) { - super(props, context); - } - - @Override - public JavaRDD generateInputRecords(String tableName, String sourceBasePath, - List>> partitionPaths) { - String filePath = FileStatusUtils.toPath(partitionPaths.stream().flatMap(p -> p.getValue().stream()) - .findAny().get().getPath()).toString(); - ParquetFileReader reader = null; - JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); - try { - reader = ParquetFileReader.open(jsc.hadoopConfiguration(), new Path(filePath)); - } catch (IOException e) { - throw new HoodieIOException(e.getMessage(), e); - } - MessageType parquetSchema = reader.getFooter().getFileMetaData().getSchema(); - Schema schema = new AvroSchemaConverter().convert(parquetSchema); - return generateInputBatch(jsc, partitionPaths, schema); - } - } - - private static JavaRDD generateInputBatch(JavaSparkContext jsc, - List>> partitionPaths, Schema writerSchema) { - List> fullFilePathsWithPartition = partitionPaths.stream().flatMap(p -> p.getValue().stream() - .map(x -> Pair.of(p.getKey(), FileStatusUtils.toPath(x.getPath())))).collect(Collectors.toList()); - return jsc.parallelize(fullFilePathsWithPartition.stream().flatMap(p -> { - try { - Configuration conf = jsc.hadoopConfiguration(); - AvroReadSupport.setAvroReadSchema(conf, writerSchema); - Iterator recIterator = new HoodieOrcReader(conf,p.getValue()).getRecordIterator(); - return StreamSupport.stream(Spliterators.spliteratorUnknownSize(recIterator, 0), false).map(gr -> { - try { - String key = gr.get("_row_key").toString(); - String pPath = p.getKey(); - return new HoodieRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath, - HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)); - } catch (IOException e) { - throw new HoodieIOException(e.getMessage(), e); - } - }); - } catch (IOException ioe) { - throw new HoodieIOException(ioe.getMessage(), ioe); - } - }).collect(Collectors.toList())); - } - - public static class TestRandomBootstapModeSelector extends BootstrapModeSelector { - - private int currIdx = new Random().nextInt(2); - - public TestRandomBootstapModeSelector(HoodieWriteConfig writeConfig) { - super(writeConfig); - } - - @Override - public Map> select(List>> partitions) { - List> selections = new ArrayList<>(); - partitions.stream().forEach(p -> { - final BootstrapMode mode; - if (currIdx == 0) { - mode = BootstrapMode.METADATA_ONLY; - } else { - mode = BootstrapMode.FULL_RECORD; - } - currIdx = (currIdx + 1) % 2; - selections.add(Pair.of(mode, p.getKey())); - }); - return selections.stream().collect(Collectors.groupingBy(Pair::getKey, mapping(Pair::getValue, toList()))); - } - } - - public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) { - HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr, IndexType.BLOOM) - .withExternalSchemaTrasformation(true); - TypedProperties properties = new TypedProperties(); - properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key"); - properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "datestr"); - builder = builder.withProps(properties); - return builder; - } - - public static Dataset generateTestRawTripDataset(long timestamp, int from, int to, List partitionPaths, - JavaSparkContext jsc, SQLContext sqlContext) { - boolean isPartitioned = partitionPaths != null && !partitionPaths.isEmpty(); - final List records = new ArrayList<>(); - IntStream.range(from, to).forEach(i -> { - String id = "" + i; - records.add(generateGenericRecord("trip_" + id, "rider_" + id, "driver_" + id, - timestamp, false, false).toString()); - }); - if (isPartitioned) { - sqlContext.udf().register("partgen", - (UDF1) (val) -> PartitionPathEncodeUtils.escapePathName(partitionPaths.get( - Integer.parseInt(val.split("_")[1]) % partitionPaths.size())), - DataTypes.StringType); - } - JavaRDD rdd = jsc.parallelize(records); - Dataset df = sqlContext.read().json(rdd); - if (isPartitioned) { - df = df.withColumn("datestr", callUDF("partgen", new Column("_row_key"))); - // Order the columns to ensure generated avro schema aligns with Hive schema - df = df.select("timestamp", "_row_key", "rider", "driver", "begin_lat", "begin_lon", - "end_lat", "end_lon", "fare", "tip_history", "_hoodie_is_deleted", "datestr"); - } else { - // Order the columns to ensure generated avro schema aligns with Hive schema - df = df.select("timestamp", "_row_key", "rider", "driver", "begin_lat", "begin_lon", - "end_lat", "end_lon", "fare", "tip_history", "_hoodie_is_deleted"); - } - return df; - } -} \ No newline at end of file From f533ba25ae842833e89084aacb439ef44877d7e2 Mon Sep 17 00:00:00 2001 From: manasaks Date: Sun, 11 Jul 2021 00:13:53 +0530 Subject: [PATCH 5/6] Remove unused imports --- .../main/java/org/apache/hudi/config/HoodieWriteConfig.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 57c82f5ef0e1a..1e92decba901b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -27,7 +27,11 @@ import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.EngineType; import org.apache.hudi.common.fs.ConsistencyGuardConfig; -import org.apache.hudi.common.model.*; +import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieCleaningPolicy; +import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload; +import org.apache.hudi.common.model.WriteConcurrencyMode; +import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.util.ReflectionUtils; From 5ebb41172b7cc92c63c98c6a51ed1b8a7cebf96f Mon Sep 17 00:00:00 2001 From: manasaks Date: Wed, 14 Jul 2021 22:27:01 +0530 Subject: [PATCH 6/6] Added test cases to test ORC bootstrap functionality --- .../apache/hudi/config/HoodieWriteConfig.java | 3 +- .../HoodieSparkBootstrapSchemaProvider.java | 28 +- .../SparkBootstrapCommitActionExecutor.java | 23 +- .../hudi/common/table/HoodieTableConfig.java | 1 - .../common/table/HoodieTableMetaClient.java | 2 +- .../apache/hudi/common/util/AvroOrcUtils.java | 36 +- .../common/testutils/HoodieTestUtils.java | 7 + ...oodieSparkBootstrapWriteClientExample.java | 2 +- .../SparkOrcBootstrapDataProvider.java | 49 +- .../apache/hudi/client/TestOrcBootstrap.java | 497 ++++++++++++++++++ 10 files changed, 576 insertions(+), 72 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index f64e9fb9f37f7..6d966e8adfbae 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1249,8 +1249,7 @@ public WriteConcurrencyMode getWriteConcurrencyMode() { public HoodieFileFormat getBaseFileFormat() { return HOODIE_BASE_FILE_FORMAT_PROP.defaultValue(); } - - + public Boolean inlineTableServices() { return inlineClusteringEnabled() || inlineCompactionEnabled() || isAutoClean(); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java index 040d005348b1b..eddde4939875a 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/bootstrap/HoodieSparkBootstrapSchemaProvider.java @@ -35,17 +35,12 @@ import org.apache.spark.sql.types.StructType; import org.apache.orc.OrcFile; -import org.apache.orc.OrcProto.UserMetadataItem; import org.apache.orc.Reader; -import org.apache.orc.Reader.Options; -import org.apache.orc.RecordReader; import org.apache.orc.TypeDescription; import org.apache.hudi.common.util.AvroOrcUtils; import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.util.OrcReaderIterator; import static org.apache.hudi.common.model.HoodieFileFormat.ORC; import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; -import org.apache.hudi.avro.model.HoodiePath; import java.io.IOException; import java.util.List; @@ -63,20 +58,17 @@ protected Schema getBootstrapSourceSchema(HoodieEngineContext context, List new HoodieException("Could not determine schema from the data files.")); String fileExtension = FSUtils.getFileExtension(filePath.getName()); - if(fileExtension.equals(PARQUET.getFileExtension())) - { + if (fileExtension.equals(PARQUET.getFileExtension())) { return getBootstrapSourceSchemaParquet(context,filePath); - } - else if(fileExtension.equals(ORC.getFileExtension())) - { - return getBootstrapSourceSchemaOrc(context,filePath ); - } - else + } else if (fileExtension.equals(ORC.getFileExtension())) { + return getBootstrapSourceSchemaOrc(context,filePath); + } else { throw new HoodieException("Could not determine schema from the data files."); + } } - private Schema getBootstrapSourceSchemaParquet(HoodieEngineContext context, Path filePath ) { + private Schema getBootstrapSourceSchemaParquet(HoodieEngineContext context, Path filePath) { MessageType parquetSchema = new ParquetUtils().readSchema(context.getHadoopConf().get(), filePath); ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter( @@ -90,21 +82,19 @@ private Schema getBootstrapSourceSchemaParquet(HoodieEngineContext context, Path return AvroConversionUtils.convertStructTypeToAvroSchema(sparkSchema, structName, recordNamespace); } - - private Schema getBootstrapSourceSchemaOrc(HoodieEngineContext context, Path filePath ) { + private Schema getBootstrapSourceSchemaOrc(HoodieEngineContext context, Path filePath) { Reader orcReader = null; try { orcReader = OrcFile.createReader(filePath, OrcFile.readerOptions(context.getHadoopConf().get())); } catch (IOException e) { throw new HoodieException("Could not determine schema from the data files."); } - TypeDescription orcSchema= orcReader.getSchema(); + TypeDescription orcSchema = orcReader.getSchema(); String tableName = HoodieAvroUtils.sanitizeName(writeConfig.getTableName()); String structName = tableName + "_record"; String recordNamespace = "hoodie." + tableName; - return AvroOrcUtils.createAvroSchemaWithNamespace(orcSchema,structName, recordNamespace); + return AvroOrcUtils.createAvroSchemaWithDefaultValue(orcSchema,structName, recordNamespace,true); } - } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index 17b79db2da0be..071b9a0796999 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -21,8 +21,8 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieFileStatus; import org.apache.hudi.client.WriteStatus; -import org.apache.hudi.client.bootstrap.HoodieBootstrapSchemaProvider; import org.apache.hudi.client.bootstrap.BootstrapMode; +import org.apache.hudi.client.bootstrap.HoodieBootstrapSchemaProvider; import org.apache.hudi.client.bootstrap.BootstrapRecordPayload; import org.apache.hudi.client.bootstrap.BootstrapWriteStatus; import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; @@ -68,8 +68,8 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; -import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; import org.apache.hudi.table.action.commit.BaseCommitActionExecutor; +import org.apache.hudi.table.action.commit.BaseSparkCommitActionExecutor; import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor; import org.apache.avro.Schema; @@ -101,13 +101,11 @@ import java.util.Map; import java.util.stream.Collectors; import org.apache.orc.OrcFile; -import org.apache.orc.OrcProto.UserMetadataItem; import org.apache.orc.Reader; import org.apache.orc.Reader.Options; import org.apache.orc.RecordReader; import org.apache.orc.TypeDescription; import org.apache.hudi.common.util.AvroOrcUtils; -import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.util.OrcReaderIterator; import static org.apache.hudi.common.model.HoodieFileFormat.ORC; import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; @@ -436,7 +434,7 @@ private BootstrapWriteStatus handleMetadataBootstrapOrc(String srcPartitionPath, TypeDescription orcSchema = orcReader.getSchema(); - avroSchema= AvroOrcUtils.createAvroSchema(orcSchema); + avroSchema = AvroOrcUtils.createAvroSchema(orcSchema); Schema recordKeySchema = HoodieAvroUtils.generateProjectionSchema(avroSchema, @@ -478,17 +476,18 @@ private BootstrapWriteStatus handleMetadataBootstrapOrc(String srcPartitionPath, } private BootstrapWriteStatus fileBasedMetadataHandler(String srcPartitionPath, String partitionPath, - HoodieFileStatus srcFileStatus, KeyGeneratorInterface keyGenerator) - { + HoodieFileStatus srcFileStatus, KeyGeneratorInterface keyGenerator) { Path sourceFilePath = FileStatusUtils.toPath(srcFileStatus.getPath()); String extension = FSUtils.getFileExtension(sourceFilePath.toString()); - if (ORC.getFileExtension().equals(extension)) - return handleMetadataBootstrapOrc(srcPartitionPath,partitionPath,srcFileStatus,keyGenerator); - if (PARQUET.getFileExtension().equals(extension)) - return handleMetadataBootstrap(srcPartitionPath,partitionPath,srcFileStatus,keyGenerator); - else + if (ORC.getFileExtension().equals(extension)) { + return handleMetadataBootstrapOrc(srcPartitionPath, partitionPath, srcFileStatus, keyGenerator); + } + if (PARQUET.getFileExtension().equals(extension)) { + return handleMetadataBootstrap(srcPartitionPath, partitionPath, srcFileStatus, keyGenerator); + } else { throw new HoodieIOException("Hoodie InputFormat not implemented for base file format " + extension); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index 934d36568ea75..9f3046f8812f5 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -266,7 +266,6 @@ public static String getDefaultBootstrapIndexClass(Properties props) { return defaultClass; } - public Option getBootstrapBasePath() { return Option.ofNullable(getString(HOODIE_BOOTSTRAP_BASE_PATH_PROP)); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index e94af058c74bf..7c7074c1730c0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -650,7 +650,7 @@ public PropertyBuilder setTimelineLayoutVersion(Integer timelineLayoutVersion) { return this; } - public PropertyBuilder setPreCombineField(String preCombineField) { + public PropertyBuilder setPreCombineField(String preCombineField) { this.preCombineField = preCombineField; return this; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java index 22efffb6ac5a8..fbd8c1993bc16 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/AvroOrcUtils.java @@ -53,6 +53,8 @@ import org.apache.hudi.exception.HoodieIOException; import org.apache.orc.TypeDescription; +import static org.apache.avro.JsonProperties.NULL_VALUE; + /** * Methods including addToVector, addUnionValue, createOrcSchema are originally from * https://github.com/streamsets/datacollector. @@ -775,8 +777,25 @@ public static Schema createAvroSchema(TypeDescription orcSchema) { } } + public static Schema createAvroSchemaWithDefaultValue(TypeDescription orcSchema, String recordName, String namespace, boolean nullable) { + Schema avroSchema = createAvroSchemaWithNamespace(orcSchema,recordName,namespace); + List fields = new ArrayList(); + List fieldList = avroSchema.getFields(); + for (Field field : fieldList) { + Schema fieldSchema = field.schema(); + Schema nullableSchema = Schema.createUnion(Schema.create(Schema.Type.NULL),fieldSchema); + if (nullable) { + fields.add(new Schema.Field(field.name(), nullableSchema, null, NULL_VALUE)); + } else { + fields.add(new Schema.Field(field.name(), fieldSchema, null, (Object) null)); + } + } + Schema schema = Schema.createRecord(recordName, null, null, false); + schema.setFields(fields); + return schema; + } - public static Schema createAvroSchemaWithNamespace(TypeDescription orcSchema, String recordName, String namespace) { + private static Schema createAvroSchemaWithNamespace(TypeDescription orcSchema, String recordName, String namespace) { switch (orcSchema.getCategory()) { case BOOLEAN: return Schema.create(Schema.Type.BOOLEAN); @@ -805,8 +824,6 @@ public static Schema createAvroSchemaWithNamespace(TypeDescription orcSchema, St LogicalTypes.date().addToSchema(date); return date; case TIMESTAMP: - // Cannot distinguish between TIMESTAMP_MILLIS and TIMESTAMP_MICROS - // Assume TIMESTAMP_MILLIS because Timestamp in ORC is in millis Schema timestamp = Schema.create(Schema.Type.LONG); LogicalTypes.timestampMillis().addToSchema(timestamp); return timestamp; @@ -817,23 +834,20 @@ public static Schema createAvroSchemaWithNamespace(TypeDescription orcSchema, St LogicalTypes.decimal(orcSchema.getPrecision(), orcSchema.getScale()).addToSchema(decimal); return decimal; case LIST: - return Schema.createArray(createAvroSchema(orcSchema.getChildren().get(0))); + return Schema.createArray(createAvroSchemaWithNamespace(orcSchema.getChildren().get(0),recordName,"")); case MAP: - return Schema.createMap(createAvroSchema(orcSchema.getChildren().get(1))); + return Schema.createMap(createAvroSchemaWithNamespace(orcSchema.getChildren().get(1),recordName,"")); case STRUCT: List childFields = new ArrayList<>(); for (int i = 0; i < orcSchema.getChildren().size(); i++) { TypeDescription childType = orcSchema.getChildren().get(i); String childName = orcSchema.getFieldNames().get(i); - childFields.add(new Field(childName, createAvroSchema(childType), "", null)); + childFields.add(new Field(childName, createAvroSchemaWithNamespace(childType,childName,""), null, null)); } - return Schema.createRecord(recordName,"",namespace ,false ,childFields); - case UNION: - return Schema.createUnion(orcSchema.getChildren().stream() - .map(AvroOrcUtils::createAvroSchema) - .collect(Collectors.toList())); + return Schema.createRecord(recordName,null, namespace,false,childFields); default: throw new IllegalStateException(String.format("Unrecognized ORC type: %s", orcSchema.getCategory().getName())); + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java index cf83eaca7c59e..27c4f2f8c81b9 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java @@ -69,6 +69,13 @@ public static HoodieTableMetaClient init(String basePath, HoodieTableType tableT return init(getDefaultHadoopConf(), basePath, tableType, props); } + public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType, String bootstrapBasePath, HoodieFileFormat baseFileFormat) throws IOException { + Properties props = new Properties(); + props.setProperty(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP.key(), bootstrapBasePath); + props.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.key(),baseFileFormat.name()); + return init(getDefaultHadoopConf(), basePath, tableType, props); + } + public static HoodieTableMetaClient init(String basePath, HoodieFileFormat baseFileFormat) throws IOException { return init(getDefaultHadoopConf(), basePath, HoodieTableType.COPY_ON_WRITE, baseFileFormat); } diff --git a/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieSparkBootstrapWriteClientExample.java b/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieSparkBootstrapWriteClientExample.java index b5a70728d24c0..8ee98e0ac30b9 100644 --- a/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieSparkBootstrapWriteClientExample.java +++ b/hudi-examples/src/main/java/org/apache/hudi/examples/java/HoodieSparkBootstrapWriteClientExample.java @@ -112,7 +112,7 @@ tablePath, new HoodieSparkEngineContext(jsc)).stream().findAny().map(p -> p.getV String structName = tableName + "_record"; String recordNamespace = "hoodie." + tableName; Reader orcReader = OrcFile.createReader(new Path(filePath), OrcFile.readerOptions(jsc.hadoopConfiguration())); - schema = AvroOrcUtils.createAvroSchemaWithNamespace(orcReader.getSchema(), structName, recordNamespace); + schema = AvroOrcUtils.createAvroSchemaWithDefaultValue(orcReader.getSchema(), structName, recordNamespace,true); } else { ParquetFileReader reader = ParquetFileReader.open(metaClient.getHadoopConf(), new Path(filePath)); diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkOrcBootstrapDataProvider.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkOrcBootstrapDataProvider.java index 56a92f8b9a109..1635436671551 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkOrcBootstrapDataProvider.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/bootstrap/SparkOrcBootstrapDataProvider.java @@ -45,40 +45,39 @@ * Spark Data frame based bootstrap input provider. */ public class SparkOrcBootstrapDataProvider extends FullRecordBootstrapDataProvider> { + private final transient SparkSession sparkSession; - private final transient SparkSession sparkSession; - - public SparkOrcBootstrapDataProvider(TypedProperties props, + public SparkOrcBootstrapDataProvider(TypedProperties props, HoodieSparkEngineContext context) { - super(props, context); - this.sparkSession = SparkSession.builder().config(context.getJavaSparkContext().getConf()).getOrCreate(); - } + super(props, context); + this.sparkSession = SparkSession.builder().config(context.getJavaSparkContext().getConf()).getOrCreate(); + } - @Override - public JavaRDD generateInputRecords(String tableName, String sourceBasePath, + @Override + public JavaRDD generateInputRecords(String tableName, String sourceBasePath, List>> partitionPathsWithFiles) { - String[] filePaths = partitionPathsWithFiles.stream().map(Pair::getValue) + String[] filePaths = partitionPathsWithFiles.stream().map(Pair::getValue) .flatMap(f -> f.stream().map(fs -> FileStatusUtils.toPath(fs.getPath()).toString())) .toArray(String[]::new); - Dataset inputDataset = sparkSession.read().orc(filePaths); + Dataset inputDataset = sparkSession.read().orc(filePaths); + try { + KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); + String structName = tableName + "_record"; + String namespace = "hoodie." + tableName; + RDD genericRecords = HoodieSparkUtils.createRdd(inputDataset, structName, namespace); + return genericRecords.toJavaRDD().map(gr -> { + String orderingVal = HoodieAvroUtils.getNestedFieldValAsString( + gr, props.getString("hoodie.datasource.write.precombine.field"), false); try { - KeyGenerator keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props); - String structName = tableName + "_record"; - String namespace = "hoodie." + tableName; - RDD genericRecords = HoodieSparkUtils.createRdd(inputDataset, structName, namespace); - return genericRecords.toJavaRDD().map(gr -> { - String orderingVal = HoodieAvroUtils.getNestedFieldValAsString( - gr, props.getString("hoodie.datasource.write.precombine.field"), false); - try { - return DataSourceUtils.createHoodieRecord(gr, orderingVal, keyGenerator.getKey(gr), - props.getString("hoodie.datasource.write.payload.class")); - } catch (IOException ioe) { - throw new HoodieIOException(ioe.getMessage(), ioe); - } - }); + return DataSourceUtils.createHoodieRecord(gr, orderingVal, keyGenerator.getKey(gr), + props.getString("hoodie.datasource.write.payload.class")); } catch (IOException ioe) { - throw new HoodieIOException(ioe.getMessage(), ioe); + throw new HoodieIOException(ioe.getMessage(), ioe); } + }); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); } + } } \ No newline at end of file diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java new file mode 100644 index 0000000000000..d1639122c81a5 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/TestOrcBootstrap.java @@ -0,0 +1,497 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.mapred.JobConf; +import org.apache.hudi.DataSourceWriteOptions; +import org.apache.hudi.avro.model.HoodieFileStatus; +import org.apache.hudi.client.bootstrap.BootstrapMode; +import org.apache.hudi.client.bootstrap.FullRecordBootstrapDataProvider; +import org.apache.hudi.client.bootstrap.selector.BootstrapModeSelector; +import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector; +import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector; +import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.common.bootstrap.FileStatusUtils; +import org.apache.hudi.common.bootstrap.index.BootstrapIndex; +import org.apache.hudi.common.config.TypedProperties; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieInstant.State; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.testutils.RawTripTestPayload; +import org.apache.hudi.common.util.AvroOrcUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.OrcReaderIterator; +import org.apache.hudi.common.util.PartitionPathEncodeUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieBootstrapConfig; +import org.apache.hudi.config.HoodieCompactionConfig; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.hadoop.HoodieParquetInputFormat; +import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat; +import org.apache.hudi.index.HoodieIndex.IndexType; +import org.apache.hudi.keygen.NonpartitionedKeyGenerator; +import org.apache.hudi.keygen.SimpleKeyGenerator; +import org.apache.hudi.table.action.bootstrap.BootstrapUtils; +import org.apache.hudi.testutils.HoodieClientTestBase; +import org.apache.orc.OrcFile; +import org.apache.orc.Reader; +import org.apache.orc.RecordReader; +import org.apache.orc.TypeDescription; +import org.apache.parquet.avro.AvroReadSupport; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SQLContext; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.api.java.UDF1; +import org.apache.spark.sql.types.DataTypes; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.IOException; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Spliterators; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.StreamSupport; + +import static java.util.stream.Collectors.mapping; +import static java.util.stream.Collectors.toList; +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.generateGenericRecord; +import static org.apache.spark.sql.functions.callUDF; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests Bootstrap Client functionality. + */ +public class TestOrcBootstrap extends HoodieClientTestBase { + + + public static final String TRIP_HIVE_COLUMN_TYPES = "bigint,string,string,string,double,double,double,double," + + "struct,array>,boolean"; + @TempDir + public java.nio.file.Path tmpFolder; + + protected String bootstrapBasePath = null; + + private HoodieParquetInputFormat roInputFormat; + private JobConf roJobConf; + + private HoodieParquetRealtimeInputFormat rtInputFormat; + private JobConf rtJobConf; + private SparkSession spark; + + @BeforeEach + public void setUp() throws Exception { + bootstrapBasePath = tmpFolder.toAbsolutePath().toString() + "/data"; + initPath(); + initSparkContexts(); + initTestDataGenerator(); + initMetaClient(); + reloadInputFormats(); + } + + @AfterEach + public void tearDown() throws IOException { + cleanupSparkContexts(); + cleanupClients(); + cleanupTestDataGenerator(); + } + + private void reloadInputFormats() { + // initialize parquet input format + roInputFormat = new HoodieParquetInputFormat(); + roJobConf = new JobConf(jsc.hadoopConfiguration()); + roInputFormat.setConf(roJobConf); + + } + + public Schema generateNewDataSetAndReturnSchema(long timestamp, int numRecords, List partitionPaths, + String srcPath) throws Exception { + boolean isPartitioned = partitionPaths != null && !partitionPaths.isEmpty(); + Dataset df = generateTestRawTripDataset(timestamp, 0, numRecords, partitionPaths, jsc, sqlContext); + df.printSchema(); + if (isPartitioned) { + df.write().partitionBy("datestr").format("orc").mode(SaveMode.Overwrite).save(srcPath); + } else { + df.write().format("orc").mode(SaveMode.Overwrite).save(srcPath); + } + String filePath = FileStatusUtils.toPath(BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), + srcPath, context).stream().findAny().map(p -> p.getValue().stream().findAny()) + .orElse(null).get().getPath()).toString(); + Reader orcReader = OrcFile.createReader(new Path(filePath), OrcFile.readerOptions(metaClient.getHadoopConf())); + + TypeDescription orcSchema = orcReader.getSchema(); + + return AvroOrcUtils.createAvroSchemaWithDefaultValue(orcSchema,"test_orc_record",null,true); + } + + @Test + public void testMetadataBootstrapUnpartitionedCOW() throws Exception { + testBootstrapCommon(false, false, EffectiveMode.METADATA_BOOTSTRAP_MODE); + } + + @Test + public void testMetadataBootstrapWithUpdatesCOW() throws Exception { + testBootstrapCommon(true, false, EffectiveMode.METADATA_BOOTSTRAP_MODE); + } + + private enum EffectiveMode { + FULL_BOOTSTRAP_MODE, + METADATA_BOOTSTRAP_MODE, + MIXED_BOOTSTRAP_MODE + } + + private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, EffectiveMode mode) throws Exception { + + if (deltaCommit) { + metaClient = HoodieTestUtils.init(basePath, HoodieTableType.MERGE_ON_READ, bootstrapBasePath,HoodieFileFormat.ORC); + } else { + metaClient = HoodieTestUtils.init(basePath, HoodieTableType.COPY_ON_WRITE, bootstrapBasePath,HoodieFileFormat.ORC); + } + + int totalRecords = 100; + String keyGeneratorClass = partitioned ? SimpleKeyGenerator.class.getCanonicalName() + : NonpartitionedKeyGenerator.class.getCanonicalName(); + final String bootstrapModeSelectorClass; + final String bootstrapCommitInstantTs; + final boolean checkNumRawFiles; + final boolean isBootstrapIndexCreated; + final int numInstantsAfterBootstrap; + final List bootstrapInstants; + switch (mode) { + case FULL_BOOTSTRAP_MODE: + bootstrapModeSelectorClass = FullRecordBootstrapModeSelector.class.getCanonicalName(); + bootstrapCommitInstantTs = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS; + checkNumRawFiles = false; + isBootstrapIndexCreated = false; + numInstantsAfterBootstrap = 1; + bootstrapInstants = Arrays.asList(bootstrapCommitInstantTs); + break; + case METADATA_BOOTSTRAP_MODE: + bootstrapModeSelectorClass = MetadataOnlyBootstrapModeSelector.class.getCanonicalName(); + bootstrapCommitInstantTs = HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS; + checkNumRawFiles = true; + isBootstrapIndexCreated = true; + numInstantsAfterBootstrap = 1; + bootstrapInstants = Arrays.asList(bootstrapCommitInstantTs); + break; + default: + bootstrapModeSelectorClass = TestRandomBootstapModeSelector.class.getName(); + bootstrapCommitInstantTs = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS; + checkNumRawFiles = false; + isBootstrapIndexCreated = true; + numInstantsAfterBootstrap = 2; + bootstrapInstants = Arrays.asList(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS, + HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS); + break; + } + List partitions = Arrays.asList("2020/04/01", "2020/04/02", "2020/04/03"); + long timestamp = Instant.now().toEpochMilli(); + Schema schema = generateNewDataSetAndReturnSchema(timestamp, totalRecords, partitions, bootstrapBasePath); + HoodieWriteConfig config = getConfigBuilder(schema.toString()) + .withAutoCommit(true) + .withSchema(schema.toString()) + .withCompactionConfig(HoodieCompactionConfig.newBuilder() + .withMaxNumDeltaCommitsBeforeCompaction(1) + .build()) + .withBootstrapConfig(HoodieBootstrapConfig.newBuilder() + .withBootstrapBasePath(bootstrapBasePath) + .withBaseFileFormat(HoodieFileFormat.ORC) + .withBootstrapKeyGenClass(keyGeneratorClass) + .withFullBootstrapInputProvider(TestFullBootstrapDataProvider.class.getName()) + .withBootstrapParallelism(3) + .withBootstrapModeSelector(bootstrapModeSelectorClass).build()) + .build(); + SparkRDDWriteClient client = new SparkRDDWriteClient(context, config); + client.bootstrap(Option.empty()); + checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap, + numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); + + // Rollback Bootstrap + FSUtils.deleteInstantFile(metaClient.getFs(), metaClient.getMetaPath(), new HoodieInstant(State.COMPLETED, + deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, bootstrapCommitInstantTs)); + client.rollbackFailedBootstrap(); + metaClient.reloadActiveTimeline(); + assertEquals(0, metaClient.getCommitsTimeline().countInstants()); + assertEquals(0L, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), basePath, context) + .stream().flatMap(f -> f.getValue().stream()).count()); + + BootstrapIndex index = BootstrapIndex.getBootstrapIndex(metaClient); + assertFalse(index.useIndex()); + + // Run bootstrap again + client = new SparkRDDWriteClient(context, config); + client.bootstrap(Option.empty()); + + metaClient.reloadActiveTimeline(); + index = BootstrapIndex.getBootstrapIndex(metaClient); + if (isBootstrapIndexCreated) { + assertTrue(index.useIndex()); + } else { + assertFalse(index.useIndex()); + } + + checkBootstrapResults(totalRecords, schema, bootstrapCommitInstantTs, checkNumRawFiles, numInstantsAfterBootstrap, + numInstantsAfterBootstrap, timestamp, timestamp, deltaCommit, bootstrapInstants); + + // Upsert case + long updateTimestamp = Instant.now().toEpochMilli(); + String updateSPath = tmpFolder.toAbsolutePath().toString() + "/data2"; + generateNewDataSetAndReturnSchema(updateTimestamp, totalRecords, partitions, updateSPath); + JavaRDD updateBatch = + generateInputBatch(jsc, BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), updateSPath, context), + schema); + String newInstantTs = client.startCommit(); + client.upsert(updateBatch, newInstantTs); + checkBootstrapResults(totalRecords, schema, newInstantTs, false, numInstantsAfterBootstrap + 1, + updateTimestamp, deltaCommit ? timestamp : updateTimestamp, deltaCommit); + + if (deltaCommit) { + Option compactionInstant = client.scheduleCompaction(Option.empty()); + assertTrue(compactionInstant.isPresent()); + client.compact(compactionInstant.get()); + checkBootstrapResults(totalRecords, schema, compactionInstant.get(), checkNumRawFiles, + numInstantsAfterBootstrap + 2, 2, updateTimestamp, updateTimestamp, !deltaCommit, + Arrays.asList(compactionInstant.get())); + } + } + + @Test + public void testMetadataBootstrapWithUpdatesMOR() throws Exception { + testBootstrapCommon(true, true, EffectiveMode.METADATA_BOOTSTRAP_MODE); + } + + @Test + public void testFullBootstrapOnlyCOW() throws Exception { + testBootstrapCommon(true, false, EffectiveMode.FULL_BOOTSTRAP_MODE); + } + + @Test + public void testFullBootstrapWithUpdatesMOR() throws Exception { + testBootstrapCommon(true, true, EffectiveMode.FULL_BOOTSTRAP_MODE); + } + + @Test + public void testMetaAndFullBootstrapCOW() throws Exception { + testBootstrapCommon(true, false, EffectiveMode.MIXED_BOOTSTRAP_MODE); + } + + @Test + public void testMetadataAndFullBootstrapWithUpdatesMOR() throws Exception { + testBootstrapCommon(true, true, EffectiveMode.MIXED_BOOTSTRAP_MODE); + } + + private void checkBootstrapResults(int totalRecords, Schema schema, String maxInstant, boolean checkNumRawFiles, + int expNumInstants, long expTimestamp, long expROTimestamp, boolean isDeltaCommit) throws Exception { + checkBootstrapResults(totalRecords, schema, maxInstant, checkNumRawFiles, expNumInstants, expNumInstants, + expTimestamp, expROTimestamp, isDeltaCommit, Arrays.asList(maxInstant)); + } + + private void checkBootstrapResults(int totalRecords, Schema schema, String instant, boolean checkNumRawFiles, + int expNumInstants, int numVersions, long expTimestamp, long expROTimestamp, boolean isDeltaCommit, + List instantsWithValidRecords) throws Exception { + metaClient.reloadActiveTimeline(); + assertEquals(expNumInstants, metaClient.getCommitsTimeline().filterCompletedInstants().countInstants()); + assertEquals(instant, metaClient.getActiveTimeline() + .getCommitsTimeline().filterCompletedInstants().lastInstant().get().getTimestamp()); + + Dataset bootstrapped = sqlContext.read().format("orc").load(basePath); + Dataset original = sqlContext.read().format("orc").load(bootstrapBasePath); + bootstrapped.registerTempTable("bootstrapped"); + original.registerTempTable("original"); + if (checkNumRawFiles) { + List files = BootstrapUtils.getAllLeafFoldersWithFiles(metaClient, metaClient.getFs(), + bootstrapBasePath, context).stream().flatMap(x -> x.getValue().stream()).collect(Collectors.toList()); + assertEquals(files.size() * numVersions, + sqlContext.sql("select distinct _hoodie_file_name from bootstrapped").count()); + } + + if (!isDeltaCommit) { + String predicate = String.join(", ", + instantsWithValidRecords.stream().map(p -> "\"" + p + "\"").collect(Collectors.toList())); + assertEquals(totalRecords, sqlContext.sql("select * from bootstrapped where _hoodie_commit_time IN " + + "(" + predicate + ")").count()); + Dataset missingOriginal = sqlContext.sql("select a._row_key from original a where a._row_key not " + + "in (select _hoodie_record_key from bootstrapped)"); + assertEquals(0, missingOriginal.count()); + Dataset missingBootstrapped = sqlContext.sql("select a._hoodie_record_key from bootstrapped a " + + "where a._hoodie_record_key not in (select _row_key from original)"); + assertEquals(0, missingBootstrapped.count()); + //sqlContext.sql("select * from bootstrapped").show(10, false); + } + + + } + + public static class TestFullBootstrapDataProvider extends FullRecordBootstrapDataProvider> { + + public TestFullBootstrapDataProvider(TypedProperties props, HoodieSparkEngineContext context) { + super(props, context); + } + + @Override + public JavaRDD generateInputRecords(String tableName, String sourceBasePath, + List>> partitionPaths) { + String[] filePaths = partitionPaths.stream().map(Pair::getValue) + .flatMap(f -> f.stream().map(fs -> FileStatusUtils.toPath(fs.getPath()).toString())) + .toArray(String[]::new); + + JavaSparkContext jsc = HoodieSparkEngineContext.getSparkContext(context); + + String filePath = FileStatusUtils.toPath(partitionPaths.stream().flatMap(p -> p.getValue().stream()) + .findAny().get().getPath()).toString(); + try { + Reader orcReader = OrcFile.createReader( + new Path(filePath), new OrcFile.ReaderOptions(jsc.hadoopConfiguration())); + + TypeDescription orcSchema = orcReader.getSchema(); + Schema avroSchema = AvroOrcUtils.createAvroSchemaWithDefaultValue(orcSchema,"test_orc_record",null,true); + return generateInputBatch(jsc, partitionPaths, avroSchema); + + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + } + } + + private static JavaRDD generateInputBatch(JavaSparkContext jsc, + List>> partitionPaths, Schema writerSchema) { + List> fullFilePathsWithPartition = partitionPaths.stream().flatMap(p -> p.getValue().stream() + .map(x -> Pair.of(p.getKey(), FileStatusUtils.toPath(x.getPath())))).collect(Collectors.toList()); + return jsc.parallelize(fullFilePathsWithPartition.stream().flatMap(p -> { + try { + Configuration conf = jsc.hadoopConfiguration(); + AvroReadSupport.setAvroReadSchema(conf, writerSchema); + Reader orcReader = OrcFile.createReader( + p.getValue(), + new OrcFile.ReaderOptions(jsc.hadoopConfiguration())); + RecordReader recordReader = orcReader.rows(); + + TypeDescription orcSchema = orcReader.getSchema(); + + Schema avroSchema = AvroOrcUtils.createAvroSchemaWithDefaultValue(orcSchema,"test_orc_record",null,true); + + Iterator recIterator = new OrcReaderIterator(recordReader, avroSchema, orcSchema); + + return StreamSupport.stream(Spliterators.spliteratorUnknownSize(recIterator, 0), false).map(gr -> { + try { + String key = gr.get("_row_key").toString(); + String pPath = p.getKey(); + return new HoodieRecord<>(new HoodieKey(key, pPath), new RawTripTestPayload(gr.toString(), key, pPath, + HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + }); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } + }).collect(Collectors.toList())); + } + + public static class TestRandomBootstapModeSelector extends BootstrapModeSelector { + private int currIdx = new Random().nextInt(2); + public TestRandomBootstapModeSelector(HoodieWriteConfig writeConfig) { + super(writeConfig); + } + + @Override + public Map> select(List>> partitions) { + List> selections = new ArrayList<>(); + partitions.stream().forEach(p -> { + final BootstrapMode mode; + if (currIdx == 0) { + mode = BootstrapMode.METADATA_ONLY; + } else { + mode = BootstrapMode.FULL_RECORD; + } + currIdx = (currIdx + 1) % 2; + selections.add(Pair.of(mode, p.getKey())); + }); + return selections.stream().collect(Collectors.groupingBy(Pair::getKey, mapping(Pair::getValue, toList()))); + } + } + + public HoodieWriteConfig.Builder getConfigBuilder(String schemaStr) { + HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr, IndexType.BLOOM) + .withExternalSchemaTrasformation(true); + TypedProperties properties = new TypedProperties(); + properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key"); + properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "datestr"); + builder = builder.withProps(properties); + return builder; + } + + public static Dataset generateTestRawTripDataset(long timestamp, int from, int to, List partitionPaths, + JavaSparkContext jsc, SQLContext sqlContext) { + boolean isPartitioned = partitionPaths != null && !partitionPaths.isEmpty(); + final List records = new ArrayList<>(); + IntStream.range(from, to).forEach(i -> { + String id = "" + i; + records.add(generateGenericRecord("trip_" + id, "rider_" + id, "driver_" + id, + timestamp, false, false).toString()); + }); + if (isPartitioned) { + sqlContext.udf().register("partgen", + (UDF1) (val) -> PartitionPathEncodeUtils.escapePathName(partitionPaths.get( + Integer.parseInt(val.split("_")[1]) % partitionPaths.size())), + DataTypes.StringType); + } + JavaRDD rdd = jsc.parallelize(records); + Dataset df = sqlContext.read().json(rdd); + if (isPartitioned) { + df = df.withColumn("datestr", callUDF("partgen", new Column("_row_key"))); + // Order the columns to ensure generated avro schema aligns with Hive schema + df = df.select("timestamp", "_row_key", "rider", "driver", "begin_lat", "begin_lon", + "end_lat", "end_lon", "fare", "tip_history", "_hoodie_is_deleted", "datestr"); + } else { + // Order the columns to ensure generated avro schema aligns with Hive schema + df = df.select("timestamp", "_row_key", "rider", "driver", "begin_lat", "begin_lon", + "end_lat", "end_lon", "fare", "tip_history", "_hoodie_is_deleted"); + } + return df; + } +} +