nsivabalan commented on code in PR #12390:
URL: https://github.com/apache/hudi/pull/12390#discussion_r1866835954
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java:
##########
@@ -261,6 +287,50 @@ public List<WriteStatus> compact(HoodieCompactionHandler
compactionHandler,
}).collect(toList());
}
+ /**
+ * Execute a single compaction operation and report back status.
+ */
+ public List<WriteStatus> compact(HoodieCompactionHandler compactionHandler,
+ HoodieTableMetaClient metaClient,
+ HoodieWriteConfig config,
+ CompactionOperation operation,
Review Comment:
looks like lot of the args are not required here for FG based compaction.
can you clean them up.
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkMergeHandleV2.java:
##########
@@ -0,0 +1,410 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodiePartitionMetadata;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import
org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator;
+import org.apache.hudi.common.table.read.HoodieFileGroupReaderStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCorruptedDataException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileWriter;
+import org.apache.hudi.io.storage.HoodieFileWriterFactory;
+import org.apache.hudi.io.storage.HoodieIOFactory;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+
+@SuppressWarnings("Duplicates")
+/**
+ * Handle to merge incoming records to those in storage.
+ * <p>
+ * Simplified Logic:
+ * For every existing record
+ * Check if there is a new record coming in. If yes, merge two records and
write to file
+ * else write the record as is
+ * For all pending records from incoming batch, write to file.
+ *
+ * Illustration with simple data.
+ * Incoming data:
+ * rec1_2, rec4_2, rec5_1, rec6_1
+ * Existing data:
+ * rec1_1, rec2_1, rec3_1, rec4_1
+ *
+ * For every existing record, merge w/ incoming if required and write to
storage.
+ * => rec1_1 and rec1_2 is merged to write rec1_2 to storage
+ * => rec2_1 is written as is
+ * => rec3_1 is written as is
+ * => rec4_2 and rec4_1 is merged to write rec4_2 to storage
+ * Write all pending records from incoming set to storage
+ * => rec5_1 and rec6_1
+ *
+ * Final snapshot in storage
+ * rec1_2, rec2_1, rec3_1, rec4_2, rec5_1, rec6_1
+ *
+ * </p>
+ */
+@NotThreadSafe
+public class HoodieSparkMergeHandleV2<T, I, K, O> extends HoodieWriteHandle<T,
I, K, O> {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(HoodieMergeHandle.class);
+
+ protected Set<String> writtenRecordKeys;
+ protected HoodieReaderContext readerContext;
+ protected HoodieFileWriter fileWriter;
+ protected boolean preserveMetadata = false;
+
+ protected StoragePath newFilePath;
+ protected Option<StoragePath> oldFilePath = Option.empty();
+ protected long recordsWritten = 0;
+ // TODO(yihua): audit delete stats because file group reader may not return
deletes
+ protected long recordsDeleted = 0;
+ protected long updatedRecordsWritten = 0;
+ protected long insertRecordsWritten = 0;
+ protected Option<BaseKeyGenerator> keyGeneratorOpt;
+ protected FileSlice fileSlice;
+ private Option<HoodieBaseFile> baseFileToMergeOpt;
+
+ protected Option<String[]> partitionFields = Option.empty();
+ protected Object[] partitionValues = new Object[0];
+ protected Configuration conf;
+
+ /**
+ * Called by compactor code path using the file group reader.
+ */
+ public HoodieSparkMergeHandleV2(HoodieWriteConfig config, String
instantTime, HoodieTable<T, I, K, O> hoodieTable,
Review Comment:
Can you name this HoodieSparkFileGroupReaderBasedMergeHandle.
we do have different handles like ConcatHandle, SortedMergeHandle etc.
So, would be better if we name based on the what is does and achieves.
##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java:
##########
@@ -261,6 +287,50 @@ public List<WriteStatus> compact(HoodieCompactionHandler
compactionHandler,
}).collect(toList());
}
+ /**
+ * Execute a single compaction operation and report back status.
+ */
+ public List<WriteStatus> compact(HoodieCompactionHandler compactionHandler,
+ HoodieTableMetaClient metaClient,
+ HoodieWriteConfig config,
+ CompactionOperation operation,
+ String instantTime,
+ String maxInstantTime,
+ Option<InstantRange> instantRange,
+ TaskContextSupplier taskContextSupplier,
+ CompactionExecutionHelper executionHelper,
+ Option<EngineBroadcastManager>
broadcastManagerOpt) throws IOException {
+ List<WriteStatus> writeStatusList =
compactionHandler.runCompactionUsingFileGroupReader(instantTime,
+ operation,
broadcastManagerOpt.get().retrieveFileGroupReaderContext(metaClient.getBasePath()).get(),
+ broadcastManagerOpt.get().retrieveStorageConfig().get());
+ writeStatusList
+ .forEach(s -> {
+ final HoodieWriteStat stat = s.getStat();
+ /*
+ fill in log reading stats
+
stat.setTotalUpdatedRecordsCompacted(scanner.getNumMergedRecordsInLog());
Review Comment:
why commented out?
if not required anymore, we can remove it
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkCopyOnWriteTable.java:
##########
@@ -264,6 +269,24 @@ public Iterator<List<WriteStatus>> handleInsert(
return Collections.singletonList(createHandle.close()).iterator();
}
+ @Override
+ public boolean supportsFileGroupReader() {
+ return true;
+ }
+
+ @Override
+ public List<WriteStatus> runCompactionUsingFileGroupReader(String
instantTime,
+
CompactionOperation operation,
+
HoodieReaderContext readerContext,
+ Configuration
conf) {
+ Option<BaseKeyGenerator> keyGeneratorOpt =
HoodieSparkKeyGeneratorFactory.createBaseKeyGenerator(config);
+ HoodieSparkMergeHandleV2 mergeHandle = new HoodieSparkMergeHandleV2(config,
+ instantTime, this, operation, taskContextSupplier, keyGeneratorOpt,
readerContext, conf);
+ mergeHandle.write();
+ mergeHandle.close();
Review Comment:
close() returns a List<WriteStatus> already. we don't need to make another
call at L 287
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkMergeHandleV2.java:
##########
@@ -0,0 +1,410 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodiePartitionMetadata;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import
org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator;
+import org.apache.hudi.common.table.read.HoodieFileGroupReaderStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCorruptedDataException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileWriter;
+import org.apache.hudi.io.storage.HoodieFileWriterFactory;
+import org.apache.hudi.io.storage.HoodieIOFactory;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+
+@SuppressWarnings("Duplicates")
+/**
+ * Handle to merge incoming records to those in storage.
+ * <p>
+ * Simplified Logic:
+ * For every existing record
+ * Check if there is a new record coming in. If yes, merge two records and
write to file
+ * else write the record as is
+ * For all pending records from incoming batch, write to file.
+ *
+ * Illustration with simple data.
+ * Incoming data:
+ * rec1_2, rec4_2, rec5_1, rec6_1
+ * Existing data:
+ * rec1_1, rec2_1, rec3_1, rec4_1
+ *
+ * For every existing record, merge w/ incoming if required and write to
storage.
+ * => rec1_1 and rec1_2 is merged to write rec1_2 to storage
+ * => rec2_1 is written as is
+ * => rec3_1 is written as is
+ * => rec4_2 and rec4_1 is merged to write rec4_2 to storage
+ * Write all pending records from incoming set to storage
+ * => rec5_1 and rec6_1
+ *
+ * Final snapshot in storage
+ * rec1_2, rec2_1, rec3_1, rec4_2, rec5_1, rec6_1
+ *
+ * </p>
+ */
+@NotThreadSafe
+public class HoodieSparkMergeHandleV2<T, I, K, O> extends HoodieWriteHandle<T,
I, K, O> {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(HoodieMergeHandle.class);
+
+ protected Set<String> writtenRecordKeys;
+ protected HoodieReaderContext readerContext;
+ protected HoodieFileWriter fileWriter;
+ protected boolean preserveMetadata = false;
+
+ protected StoragePath newFilePath;
+ protected Option<StoragePath> oldFilePath = Option.empty();
+ protected long recordsWritten = 0;
+ // TODO(yihua): audit delete stats because file group reader may not return
deletes
+ protected long recordsDeleted = 0;
+ protected long updatedRecordsWritten = 0;
+ protected long insertRecordsWritten = 0;
+ protected Option<BaseKeyGenerator> keyGeneratorOpt;
+ protected FileSlice fileSlice;
+ private Option<HoodieBaseFile> baseFileToMergeOpt;
+
+ protected Option<String[]> partitionFields = Option.empty();
+ protected Object[] partitionValues = new Object[0];
+ protected Configuration conf;
+
+ /**
+ * Called by compactor code path using the file group reader.
+ */
+ public HoodieSparkMergeHandleV2(HoodieWriteConfig config, String
instantTime, HoodieTable<T, I, K, O> hoodieTable,
+ CompactionOperation operation,
TaskContextSupplier taskContextSupplier, Option<BaseKeyGenerator>
keyGeneratorOpt,
+ HoodieReaderContext readerContext,
Configuration conf) {
+ super(config, instantTime, operation.getPartitionPath(),
operation.getFileId(), hoodieTable, taskContextSupplier);
+ this.readerContext = readerContext;
+ this.conf = conf;
+ Option<HoodieBaseFile> baseFileOpt =
+ operation.getBaseFile(config.getBasePath(),
operation.getPartitionPath());
+ List<HoodieLogFile> logFiles =
operation.getDeltaFileNames().stream().map(p ->
+ new HoodieLogFile(new StoragePath(FSUtils.constructAbsolutePath(
+ config.getBasePath(), operation.getPartitionPath()), p)))
+ .collect(Collectors.toList());
+ this.fileSlice = new FileSlice(
+ operation.getFileGroupId(),
+ operation.getBaseInstantTime(),
+ baseFileOpt.isPresent() ? baseFileOpt.get() : null,
+ logFiles);
+ this.preserveMetadata = true;
+ init(fileId, this.partitionPath, baseFileOpt);
+ validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields());
+ }
+
+ private void validateAndSetAndKeyGenProps(Option<BaseKeyGenerator>
keyGeneratorOpt, boolean populateMetaFields) {
+ ValidationUtils.checkArgument(populateMetaFields ==
!keyGeneratorOpt.isPresent());
+ this.keyGeneratorOpt = keyGeneratorOpt;
+ }
+
+ /**
+ * Extract old file path, initialize StorageWriter and WriteStatus.
+ */
+ private void init(String fileId, String partitionPath,
Option<HoodieBaseFile> baseFileToMerge) {
+ LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" +
fileId);
+ this.baseFileToMergeOpt = baseFileToMerge;
+ this.writtenRecordKeys = new HashSet<>();
+ writeStatus.setStat(new HoodieWriteStat());
+ try {
+ Option<String> latestValidFilePath = Option.empty();
+ if (baseFileToMerge.isPresent()) {
+ latestValidFilePath = Option.of(baseFileToMerge.get().getFileName());
+
writeStatus.getStat().setPrevCommit(baseFileToMerge.get().getCommitTime());
+ // At the moment, we only support SI for overwrite with latest
payload. So, we don't need to embed entire file slice here.
+ // HUDI-8518 will be taken up to fix it for any payload during which
we might require entire file slice to be set here.
+ // Already AppendHandle adds all logs file from current file slice to
HoodieDeltaWriteStat.
+ writeStatus.getStat().setPrevBaseFile(latestValidFilePath.get());
+ } else {
+ writeStatus.getStat().setPrevCommit(HoodieWriteStat.NULL_COMMIT);
+ }
+
+ HoodiePartitionMetadata partitionMetadata = new
HoodiePartitionMetadata(storage, instantTime,
+ new StoragePath(config.getBasePath()),
+ FSUtils.constructAbsolutePath(config.getBasePath(), partitionPath),
+ hoodieTable.getPartitionMetafileFormat());
+ partitionMetadata.trySave();
+
+ String newFileName = FSUtils.makeBaseFileName(instantTime, writeToken,
fileId, hoodieTable.getBaseFileExtension());
+ makeOldAndNewFilePaths(partitionPath, latestValidFilePath, newFileName);
+
+ LOG.info(String.format(
+ "Merging data from file group %s, to a new base file %s", fileId,
newFilePath.toString()));
+ // file name is same for all records, in this bunch
+ writeStatus.setFileId(fileId);
+ writeStatus.setPartitionPath(partitionPath);
+ writeStatus.getStat().setPartitionPath(partitionPath);
+ writeStatus.getStat().setFileId(fileId);
+ setWriteStatusPath();
+
+ // Create Marker file,
+ // uses name of `newFilePath` instead of `newFileName`
+ // in case the sub-class may roll over the file handle name.
+ createMarkerFile(partitionPath, newFilePath.getName());
+
+ // Create the writer for writing the new version file
+ fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
newFilePath, hoodieTable.getStorage(),
+ config, writeSchemaWithMetaFields, taskContextSupplier,
HoodieRecord.HoodieRecordType.SPARK);
+ } catch (IOException io) {
+ LOG.error("Error in update task at commit " + instantTime, io);
+ writeStatus.setGlobalError(io);
+ throw new HoodieUpsertException("Failed to initialize HoodieUpdateHandle
for FileId: " + fileId + " on commit "
+ + instantTime + " on path " +
hoodieTable.getMetaClient().getBasePath(), io);
+ }
+ }
+
+ protected void setWriteStatusPath() {
+ writeStatus.getStat().setPath(new StoragePath(config.getBasePath()),
newFilePath);
+ }
+
+ protected void makeOldAndNewFilePaths(String partitionPath, Option<String>
oldFileName, String newFileName) {
+ if (oldFileName.isPresent()) {
+ oldFilePath = Option.of(makeNewFilePath(partitionPath,
oldFileName.get()));
+ }
+ newFilePath = makeNewFilePath(partitionPath, newFileName);
+ }
+
+ // TODO(yihua): is this still needed
+ /*
+ if (needsUpdateLocation()) {
+ record.unseal();
+ record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
+ record.seal();
+ }
+ */
+
+ public void write() {
+ boolean usePosition =
config.getBooleanOrDefault(MERGE_USE_RECORD_POSITIONS);
+ Schema readerSchema;
+ Option<InternalSchema> internalSchemaOption = Option.empty();
+ if (!StringUtils.isNullOrEmpty(config.getInternalSchema())) {
+ readerSchema = HoodieAvroUtils.addMetadataFields(
+ new Schema.Parser().parse(config.getSchema()),
config.allowOperationMetadataField());
+ internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema());
+ } else {
+ readerSchema = HoodieAvroUtils.addMetadataFields(
+ new Schema.Parser().parse(config.getSchema()),
config.allowOperationMetadataField());
Review Comment:
for schema, can we use SeializableSchema rather than pulling schema object
as is.
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/SparkBroadcastManager.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.table;
+
+import org.apache.hudi.SparkAdapterSupport$;
+import org.apache.hudi.SparkFileFormatInternalRowReaderContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.execution.datasources.FileFormat;
+import org.apache.spark.sql.execution.datasources.parquet.SparkParquetReader;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.util.SerializableConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import scala.Tuple2;
+import scala.collection.JavaConverters;
+
+public class SparkBroadcastManager extends EngineBroadcastManager {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(SparkBroadcastManager.class);
+
+ private final transient HoodieEngineContext context;
+
+ protected Option<SparkParquetReader> parquetReaderOpt = Option.empty();
+ protected Broadcast<SQLConf> sqlConfBroadcast;
+ protected Broadcast<SparkParquetReader> parquetReaderBroadcast;
+ protected Broadcast<SerializableConfiguration> configurationBroadcast;
+
+ public SparkBroadcastManager(HoodieEngineContext context) {
+ this.context = context;
+ }
+
+ // Prepare broadcast variables.
+ @Override
+ public void prepareAndBroadcast() {
+ // This needs to be fixed.
+ if (!(context instanceof HoodieSparkEngineContext)) {
+ throw new HoodieIOException("Expected to be called using Engine's
context and not local context");
+ }
+
+ HoodieSparkEngineContext hoodieSparkEngineContext =
(HoodieSparkEngineContext) context;
+ SQLConf sqlConf =
hoodieSparkEngineContext.getSqlContext().sessionState().conf();
+ JavaSparkContext jsc = hoodieSparkEngineContext.jsc();
+
+ // TODO: Confirm what is the correct way to set this config.
+ boolean returningBatch = sqlConf.parquetVectorizedReaderEnabled();
+ scala.collection.immutable.Map<String, String> options =
+ scala.collection.immutable.Map$.MODULE$.<String, String>empty()
+ .$plus(new Tuple2<>(FileFormat.OPTION_RETURNING_BATCH(),
Boolean.toString(returningBatch)));
+
+ // Do broadcast.
+ sqlConfBroadcast = jsc.broadcast(sqlConf);
+ configurationBroadcast = jsc.broadcast(new
SerializableConfiguration(jsc.hadoopConfiguration()));
+ // TODO: Disable vectorization as of now. Assign it based on relevant
settings.
+ // TODO: Verify if we can construct the reader on the executor side if we
has broadcast all necessary variables.
+ parquetReaderOpt =
Option.of(SparkAdapterSupport$.MODULE$.sparkAdapter().createParquetFileReader(
+ false, sqlConfBroadcast.getValue(), options,
configurationBroadcast.getValue().value()));
+ parquetReaderBroadcast = jsc.broadcast(parquetReaderOpt.get());
+ }
+
+ @Override
+ public Option<HoodieReaderContext>
retrieveFileGroupReaderContext(StoragePath basePath) {
+ if (parquetReaderBroadcast == null) {
+ LOG.warn("ParquetReader is not broadcast; cannot create file group
reader");
+ return Option.empty();
+ }
+
+ // Reconstruct metaClient, since underlying properties may not be passed
from driver to executors.
+ HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
+ .setBasePath(basePath)
+ .setConf(new
HadoopStorageConfiguration(configurationBroadcast.getValue().value())).build();
+ SparkParquetReader sparkParquetReader = parquetReaderBroadcast.getValue();
+ if (sparkParquetReader != null) {
+ List<Filter> filters = new ArrayList<>();
+ return Option.of(new SparkFileFormatInternalRowReaderContext(
+ sparkParquetReader,
+ JavaConverters.asScalaBufferConverter(filters).asScala().toSeq(),
+ JavaConverters.asScalaBufferConverter(filters).asScala().toSeq()));
+ } else {
+ LOG.warn("ParquetFileReader is null");
Review Comment:
also, should we throw exception in this case?
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkMergeHandleV2.java:
##########
@@ -0,0 +1,410 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodiePartitionMetadata;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import
org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator;
+import org.apache.hudi.common.table.read.HoodieFileGroupReaderStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCorruptedDataException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileWriter;
+import org.apache.hudi.io.storage.HoodieFileWriterFactory;
+import org.apache.hudi.io.storage.HoodieIOFactory;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+
+@SuppressWarnings("Duplicates")
+/**
+ * Handle to merge incoming records to those in storage.
+ * <p>
+ * Simplified Logic:
+ * For every existing record
+ * Check if there is a new record coming in. If yes, merge two records and
write to file
+ * else write the record as is
+ * For all pending records from incoming batch, write to file.
+ *
+ * Illustration with simple data.
+ * Incoming data:
+ * rec1_2, rec4_2, rec5_1, rec6_1
+ * Existing data:
+ * rec1_1, rec2_1, rec3_1, rec4_1
+ *
+ * For every existing record, merge w/ incoming if required and write to
storage.
+ * => rec1_1 and rec1_2 is merged to write rec1_2 to storage
+ * => rec2_1 is written as is
+ * => rec3_1 is written as is
+ * => rec4_2 and rec4_1 is merged to write rec4_2 to storage
+ * Write all pending records from incoming set to storage
+ * => rec5_1 and rec6_1
+ *
+ * Final snapshot in storage
+ * rec1_2, rec2_1, rec3_1, rec4_2, rec5_1, rec6_1
+ *
+ * </p>
+ */
+@NotThreadSafe
+public class HoodieSparkMergeHandleV2<T, I, K, O> extends HoodieWriteHandle<T,
I, K, O> {
Review Comment:
sorry not sure I understand.
init(String fileId, String partitionPath, Option<HoodieBaseFile>
baseFileToMerge) {
and
validateAndSetAndKeyGenProps(Option<BaseKeyGenerator> keyGeneratorOpt,
boolean populateMetaFields) {
are all duplicated code.
if its worth, you can introduce a BaseMergehandle and extend from both
HoodieMergeHandle and HoodieSparkMergeHandleV2
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkMergeHandleV2.java:
##########
@@ -0,0 +1,410 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodiePartitionMetadata;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import
org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator;
+import org.apache.hudi.common.table.read.HoodieFileGroupReaderStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCorruptedDataException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileWriter;
+import org.apache.hudi.io.storage.HoodieFileWriterFactory;
+import org.apache.hudi.io.storage.HoodieIOFactory;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+
+@SuppressWarnings("Duplicates")
+/**
+ * Handle to merge incoming records to those in storage.
+ * <p>
+ * Simplified Logic:
+ * For every existing record
+ * Check if there is a new record coming in. If yes, merge two records and
write to file
+ * else write the record as is
+ * For all pending records from incoming batch, write to file.
+ *
+ * Illustration with simple data.
+ * Incoming data:
+ * rec1_2, rec4_2, rec5_1, rec6_1
+ * Existing data:
+ * rec1_1, rec2_1, rec3_1, rec4_1
+ *
+ * For every existing record, merge w/ incoming if required and write to
storage.
+ * => rec1_1 and rec1_2 is merged to write rec1_2 to storage
+ * => rec2_1 is written as is
+ * => rec3_1 is written as is
+ * => rec4_2 and rec4_1 is merged to write rec4_2 to storage
+ * Write all pending records from incoming set to storage
+ * => rec5_1 and rec6_1
+ *
+ * Final snapshot in storage
+ * rec1_2, rec2_1, rec3_1, rec4_2, rec5_1, rec6_1
+ *
+ * </p>
+ */
+@NotThreadSafe
+public class HoodieSparkMergeHandleV2<T, I, K, O> extends HoodieWriteHandle<T,
I, K, O> {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(HoodieMergeHandle.class);
+
+ protected Set<String> writtenRecordKeys;
+ protected HoodieReaderContext readerContext;
+ protected HoodieFileWriter fileWriter;
+ protected boolean preserveMetadata = false;
+
+ protected StoragePath newFilePath;
+ protected Option<StoragePath> oldFilePath = Option.empty();
+ protected long recordsWritten = 0;
+ // TODO(yihua): audit delete stats because file group reader may not return
deletes
+ protected long recordsDeleted = 0;
+ protected long updatedRecordsWritten = 0;
+ protected long insertRecordsWritten = 0;
+ protected Option<BaseKeyGenerator> keyGeneratorOpt;
+ protected FileSlice fileSlice;
+ private Option<HoodieBaseFile> baseFileToMergeOpt;
+
+ protected Option<String[]> partitionFields = Option.empty();
+ protected Object[] partitionValues = new Object[0];
+ protected Configuration conf;
+
+ /**
+ * Called by compactor code path using the file group reader.
+ */
+ public HoodieSparkMergeHandleV2(HoodieWriteConfig config, String
instantTime, HoodieTable<T, I, K, O> hoodieTable,
+ CompactionOperation operation,
TaskContextSupplier taskContextSupplier, Option<BaseKeyGenerator>
keyGeneratorOpt,
+ HoodieReaderContext readerContext,
Configuration conf) {
+ super(config, instantTime, operation.getPartitionPath(),
operation.getFileId(), hoodieTable, taskContextSupplier);
+ this.readerContext = readerContext;
+ this.conf = conf;
+ Option<HoodieBaseFile> baseFileOpt =
+ operation.getBaseFile(config.getBasePath(),
operation.getPartitionPath());
+ List<HoodieLogFile> logFiles =
operation.getDeltaFileNames().stream().map(p ->
+ new HoodieLogFile(new StoragePath(FSUtils.constructAbsolutePath(
+ config.getBasePath(), operation.getPartitionPath()), p)))
+ .collect(Collectors.toList());
+ this.fileSlice = new FileSlice(
+ operation.getFileGroupId(),
+ operation.getBaseInstantTime(),
+ baseFileOpt.isPresent() ? baseFileOpt.get() : null,
+ logFiles);
+ this.preserveMetadata = true;
+ init(fileId, this.partitionPath, baseFileOpt);
+ validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields());
+ }
+
+ private void validateAndSetAndKeyGenProps(Option<BaseKeyGenerator>
keyGeneratorOpt, boolean populateMetaFields) {
+ ValidationUtils.checkArgument(populateMetaFields ==
!keyGeneratorOpt.isPresent());
+ this.keyGeneratorOpt = keyGeneratorOpt;
+ }
+
+ /**
+ * Extract old file path, initialize StorageWriter and WriteStatus.
+ */
+ private void init(String fileId, String partitionPath,
Option<HoodieBaseFile> baseFileToMerge) {
+ LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" +
fileId);
+ this.baseFileToMergeOpt = baseFileToMerge;
+ this.writtenRecordKeys = new HashSet<>();
+ writeStatus.setStat(new HoodieWriteStat());
+ try {
+ Option<String> latestValidFilePath = Option.empty();
+ if (baseFileToMerge.isPresent()) {
+ latestValidFilePath = Option.of(baseFileToMerge.get().getFileName());
+
writeStatus.getStat().setPrevCommit(baseFileToMerge.get().getCommitTime());
+ // At the moment, we only support SI for overwrite with latest
payload. So, we don't need to embed entire file slice here.
+ // HUDI-8518 will be taken up to fix it for any payload during which
we might require entire file slice to be set here.
+ // Already AppendHandle adds all logs file from current file slice to
HoodieDeltaWriteStat.
+ writeStatus.getStat().setPrevBaseFile(latestValidFilePath.get());
+ } else {
+ writeStatus.getStat().setPrevCommit(HoodieWriteStat.NULL_COMMIT);
+ }
+
+ HoodiePartitionMetadata partitionMetadata = new
HoodiePartitionMetadata(storage, instantTime,
+ new StoragePath(config.getBasePath()),
+ FSUtils.constructAbsolutePath(config.getBasePath(), partitionPath),
+ hoodieTable.getPartitionMetafileFormat());
+ partitionMetadata.trySave();
+
+ String newFileName = FSUtils.makeBaseFileName(instantTime, writeToken,
fileId, hoodieTable.getBaseFileExtension());
+ makeOldAndNewFilePaths(partitionPath, latestValidFilePath, newFileName);
+
+ LOG.info(String.format(
+ "Merging data from file group %s, to a new base file %s", fileId,
newFilePath.toString()));
+ // file name is same for all records, in this bunch
+ writeStatus.setFileId(fileId);
+ writeStatus.setPartitionPath(partitionPath);
+ writeStatus.getStat().setPartitionPath(partitionPath);
+ writeStatus.getStat().setFileId(fileId);
+ setWriteStatusPath();
+
+ // Create Marker file,
+ // uses name of `newFilePath` instead of `newFileName`
+ // in case the sub-class may roll over the file handle name.
+ createMarkerFile(partitionPath, newFilePath.getName());
+
+ // Create the writer for writing the new version file
+ fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
newFilePath, hoodieTable.getStorage(),
+ config, writeSchemaWithMetaFields, taskContextSupplier,
HoodieRecord.HoodieRecordType.SPARK);
+ } catch (IOException io) {
+ LOG.error("Error in update task at commit " + instantTime, io);
+ writeStatus.setGlobalError(io);
+ throw new HoodieUpsertException("Failed to initialize HoodieUpdateHandle
for FileId: " + fileId + " on commit "
+ + instantTime + " on path " +
hoodieTable.getMetaClient().getBasePath(), io);
+ }
+ }
+
+ protected void setWriteStatusPath() {
+ writeStatus.getStat().setPath(new StoragePath(config.getBasePath()),
newFilePath);
+ }
+
+ protected void makeOldAndNewFilePaths(String partitionPath, Option<String>
oldFileName, String newFileName) {
+ if (oldFileName.isPresent()) {
+ oldFilePath = Option.of(makeNewFilePath(partitionPath,
oldFileName.get()));
+ }
+ newFilePath = makeNewFilePath(partitionPath, newFileName);
+ }
+
+ // TODO(yihua): is this still needed
+ /*
+ if (needsUpdateLocation()) {
+ record.unseal();
+ record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
+ record.seal();
+ }
+ */
+
+ public void write() {
+ boolean usePosition =
config.getBooleanOrDefault(MERGE_USE_RECORD_POSITIONS);
+ Schema readerSchema;
+ Option<InternalSchema> internalSchemaOption = Option.empty();
+ if (!StringUtils.isNullOrEmpty(config.getInternalSchema())) {
+ readerSchema = HoodieAvroUtils.addMetadataFields(
+ new Schema.Parser().parse(config.getSchema()),
config.allowOperationMetadataField());
+ internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema());
+ } else {
+ readerSchema = HoodieAvroUtils.addMetadataFields(
+ new Schema.Parser().parse(config.getSchema()),
config.allowOperationMetadataField());
Review Comment:
also, this "if else" block can be simplified.
```
readerSchema = HoodieAvroUtils.addMetadataFields(
new Schema.Parser().parse(config.getSchema()),
config.allowOperationMetadataField());
if (!StringUtils.isNullOrEmpty(config.getInternalSchema())) {
internalSchemaOption =
SerDeHelper.fromJson(config.getInternalSchema());
}
```
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/SparkBroadcastManager.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.table;
+
+import org.apache.hudi.SparkAdapterSupport$;
+import org.apache.hudi.SparkFileFormatInternalRowReaderContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.execution.datasources.FileFormat;
+import org.apache.spark.sql.execution.datasources.parquet.SparkParquetReader;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.util.SerializableConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import scala.Tuple2;
+import scala.collection.JavaConverters;
+
+public class SparkBroadcastManager extends EngineBroadcastManager {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(SparkBroadcastManager.class);
+
+ private final transient HoodieEngineContext context;
+
+ protected Option<SparkParquetReader> parquetReaderOpt = Option.empty();
+ protected Broadcast<SQLConf> sqlConfBroadcast;
+ protected Broadcast<SparkParquetReader> parquetReaderBroadcast;
+ protected Broadcast<SerializableConfiguration> configurationBroadcast;
+
+ public SparkBroadcastManager(HoodieEngineContext context) {
+ this.context = context;
+ }
+
+ // Prepare broadcast variables.
+ @Override
+ public void prepareAndBroadcast() {
+ // This needs to be fixed.
+ if (!(context instanceof HoodieSparkEngineContext)) {
+ throw new HoodieIOException("Expected to be called using Engine's
context and not local context");
+ }
+
+ HoodieSparkEngineContext hoodieSparkEngineContext =
(HoodieSparkEngineContext) context;
+ SQLConf sqlConf =
hoodieSparkEngineContext.getSqlContext().sessionState().conf();
+ JavaSparkContext jsc = hoodieSparkEngineContext.jsc();
+
+ // TODO: Confirm what is the correct way to set this config.
+ boolean returningBatch = sqlConf.parquetVectorizedReaderEnabled();
+ scala.collection.immutable.Map<String, String> options =
+ scala.collection.immutable.Map$.MODULE$.<String, String>empty()
+ .$plus(new Tuple2<>(FileFormat.OPTION_RETURNING_BATCH(),
Boolean.toString(returningBatch)));
+
+ // Do broadcast.
+ sqlConfBroadcast = jsc.broadcast(sqlConf);
+ configurationBroadcast = jsc.broadcast(new
SerializableConfiguration(jsc.hadoopConfiguration()));
+ // TODO: Disable vectorization as of now. Assign it based on relevant
settings.
+ // TODO: Verify if we can construct the reader on the executor side if we
has broadcast all necessary variables.
+ parquetReaderOpt =
Option.of(SparkAdapterSupport$.MODULE$.sparkAdapter().createParquetFileReader(
+ false, sqlConfBroadcast.getValue(), options,
configurationBroadcast.getValue().value()));
+ parquetReaderBroadcast = jsc.broadcast(parquetReaderOpt.get());
+ }
+
+ @Override
+ public Option<HoodieReaderContext>
retrieveFileGroupReaderContext(StoragePath basePath) {
+ if (parquetReaderBroadcast == null) {
+ LOG.warn("ParquetReader is not broadcast; cannot create file group
reader");
+ return Option.empty();
+ }
+
+ // Reconstruct metaClient, since underlying properties may not be passed
from driver to executors.
+ HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder()
Review Comment:
do you know whats the purpose of this metaClient. we are not using it
anywhere?
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkMergeHandleV2.java:
##########
@@ -0,0 +1,410 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodiePartitionMetadata;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import
org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator;
+import org.apache.hudi.common.table.read.HoodieFileGroupReaderStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCorruptedDataException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileWriter;
+import org.apache.hudi.io.storage.HoodieFileWriterFactory;
+import org.apache.hudi.io.storage.HoodieIOFactory;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+
+@SuppressWarnings("Duplicates")
+/**
+ * Handle to merge incoming records to those in storage.
+ * <p>
+ * Simplified Logic:
+ * For every existing record
+ * Check if there is a new record coming in. If yes, merge two records and
write to file
+ * else write the record as is
+ * For all pending records from incoming batch, write to file.
+ *
+ * Illustration with simple data.
+ * Incoming data:
+ * rec1_2, rec4_2, rec5_1, rec6_1
+ * Existing data:
+ * rec1_1, rec2_1, rec3_1, rec4_1
+ *
+ * For every existing record, merge w/ incoming if required and write to
storage.
+ * => rec1_1 and rec1_2 is merged to write rec1_2 to storage
+ * => rec2_1 is written as is
+ * => rec3_1 is written as is
+ * => rec4_2 and rec4_1 is merged to write rec4_2 to storage
+ * Write all pending records from incoming set to storage
+ * => rec5_1 and rec6_1
+ *
+ * Final snapshot in storage
+ * rec1_2, rec2_1, rec3_1, rec4_2, rec5_1, rec6_1
+ *
+ * </p>
+ */
+@NotThreadSafe
+public class HoodieSparkMergeHandleV2<T, I, K, O> extends HoodieWriteHandle<T,
I, K, O> {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(HoodieMergeHandle.class);
+
+ protected Set<String> writtenRecordKeys;
+ protected HoodieReaderContext readerContext;
+ protected HoodieFileWriter fileWriter;
+ protected boolean preserveMetadata = false;
+
+ protected StoragePath newFilePath;
+ protected Option<StoragePath> oldFilePath = Option.empty();
+ protected long recordsWritten = 0;
+ // TODO(yihua): audit delete stats because file group reader may not return
deletes
+ protected long recordsDeleted = 0;
+ protected long updatedRecordsWritten = 0;
+ protected long insertRecordsWritten = 0;
+ protected Option<BaseKeyGenerator> keyGeneratorOpt;
+ protected FileSlice fileSlice;
+ private Option<HoodieBaseFile> baseFileToMergeOpt;
+
+ protected Option<String[]> partitionFields = Option.empty();
+ protected Object[] partitionValues = new Object[0];
+ protected Configuration conf;
+
+ /**
+ * Called by compactor code path using the file group reader.
+ */
+ public HoodieSparkMergeHandleV2(HoodieWriteConfig config, String
instantTime, HoodieTable<T, I, K, O> hoodieTable,
+ CompactionOperation operation,
TaskContextSupplier taskContextSupplier, Option<BaseKeyGenerator>
keyGeneratorOpt,
+ HoodieReaderContext readerContext,
Configuration conf) {
+ super(config, instantTime, operation.getPartitionPath(),
operation.getFileId(), hoodieTable, taskContextSupplier);
+ this.readerContext = readerContext;
+ this.conf = conf;
+ Option<HoodieBaseFile> baseFileOpt =
+ operation.getBaseFile(config.getBasePath(),
operation.getPartitionPath());
+ List<HoodieLogFile> logFiles =
operation.getDeltaFileNames().stream().map(p ->
+ new HoodieLogFile(new StoragePath(FSUtils.constructAbsolutePath(
+ config.getBasePath(), operation.getPartitionPath()), p)))
+ .collect(Collectors.toList());
+ this.fileSlice = new FileSlice(
+ operation.getFileGroupId(),
+ operation.getBaseInstantTime(),
+ baseFileOpt.isPresent() ? baseFileOpt.get() : null,
+ logFiles);
+ this.preserveMetadata = true;
+ init(fileId, this.partitionPath, baseFileOpt);
+ validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields());
+ }
+
+ private void validateAndSetAndKeyGenProps(Option<BaseKeyGenerator>
keyGeneratorOpt, boolean populateMetaFields) {
+ ValidationUtils.checkArgument(populateMetaFields ==
!keyGeneratorOpt.isPresent());
+ this.keyGeneratorOpt = keyGeneratorOpt;
+ }
+
+ /**
+ * Extract old file path, initialize StorageWriter and WriteStatus.
+ */
+ private void init(String fileId, String partitionPath,
Option<HoodieBaseFile> baseFileToMerge) {
+ LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" +
fileId);
+ this.baseFileToMergeOpt = baseFileToMerge;
+ this.writtenRecordKeys = new HashSet<>();
+ writeStatus.setStat(new HoodieWriteStat());
+ try {
+ Option<String> latestValidFilePath = Option.empty();
+ if (baseFileToMerge.isPresent()) {
+ latestValidFilePath = Option.of(baseFileToMerge.get().getFileName());
+
writeStatus.getStat().setPrevCommit(baseFileToMerge.get().getCommitTime());
+ // At the moment, we only support SI for overwrite with latest
payload. So, we don't need to embed entire file slice here.
+ // HUDI-8518 will be taken up to fix it for any payload during which
we might require entire file slice to be set here.
+ // Already AppendHandle adds all logs file from current file slice to
HoodieDeltaWriteStat.
+ writeStatus.getStat().setPrevBaseFile(latestValidFilePath.get());
+ } else {
+ writeStatus.getStat().setPrevCommit(HoodieWriteStat.NULL_COMMIT);
+ }
+
+ HoodiePartitionMetadata partitionMetadata = new
HoodiePartitionMetadata(storage, instantTime,
+ new StoragePath(config.getBasePath()),
+ FSUtils.constructAbsolutePath(config.getBasePath(), partitionPath),
+ hoodieTable.getPartitionMetafileFormat());
+ partitionMetadata.trySave();
+
+ String newFileName = FSUtils.makeBaseFileName(instantTime, writeToken,
fileId, hoodieTable.getBaseFileExtension());
+ makeOldAndNewFilePaths(partitionPath, latestValidFilePath, newFileName);
+
+ LOG.info(String.format(
+ "Merging data from file group %s, to a new base file %s", fileId,
newFilePath.toString()));
+ // file name is same for all records, in this bunch
+ writeStatus.setFileId(fileId);
+ writeStatus.setPartitionPath(partitionPath);
+ writeStatus.getStat().setPartitionPath(partitionPath);
+ writeStatus.getStat().setFileId(fileId);
+ setWriteStatusPath();
+
+ // Create Marker file,
+ // uses name of `newFilePath` instead of `newFileName`
+ // in case the sub-class may roll over the file handle name.
+ createMarkerFile(partitionPath, newFilePath.getName());
+
+ // Create the writer for writing the new version file
+ fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
newFilePath, hoodieTable.getStorage(),
+ config, writeSchemaWithMetaFields, taskContextSupplier,
HoodieRecord.HoodieRecordType.SPARK);
+ } catch (IOException io) {
+ LOG.error("Error in update task at commit " + instantTime, io);
+ writeStatus.setGlobalError(io);
+ throw new HoodieUpsertException("Failed to initialize HoodieUpdateHandle
for FileId: " + fileId + " on commit "
+ + instantTime + " on path " +
hoodieTable.getMetaClient().getBasePath(), io);
+ }
+ }
+
+ protected void setWriteStatusPath() {
+ writeStatus.getStat().setPath(new StoragePath(config.getBasePath()),
newFilePath);
+ }
+
+ protected void makeOldAndNewFilePaths(String partitionPath, Option<String>
oldFileName, String newFileName) {
+ if (oldFileName.isPresent()) {
+ oldFilePath = Option.of(makeNewFilePath(partitionPath,
oldFileName.get()));
+ }
+ newFilePath = makeNewFilePath(partitionPath, newFileName);
+ }
+
+ // TODO(yihua): is this still needed
+ /*
+ if (needsUpdateLocation()) {
+ record.unseal();
+ record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
+ record.seal();
+ }
+ */
+
+ public void write() {
+ boolean usePosition =
config.getBooleanOrDefault(MERGE_USE_RECORD_POSITIONS);
+ Schema readerSchema;
+ Option<InternalSchema> internalSchemaOption = Option.empty();
+ if (!StringUtils.isNullOrEmpty(config.getInternalSchema())) {
+ readerSchema = HoodieAvroUtils.addMetadataFields(
+ new Schema.Parser().parse(config.getSchema()),
config.allowOperationMetadataField());
+ internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema());
+ } else {
+ readerSchema = HoodieAvroUtils.addMetadataFields(
+ new Schema.Parser().parse(config.getSchema()),
config.allowOperationMetadataField());
+ }
+ // TODO(yihua): reader schema is good enough for writer?
+ HoodieFileGroupReader<T> fileGroupReader = new HoodieFileGroupReader<>(
+ readerContext,
+ storage.newInstance(hoodieTable.getMetaClient().getBasePath(), new
HadoopStorageConfiguration(conf)),
+ hoodieTable.getMetaClient().getBasePath().toString(),
+ instantTime,
+ fileSlice,
+ readerSchema,
+ readerSchema,
+ internalSchemaOption,
+ hoodieTable.getMetaClient(),
+ hoodieTable.getMetaClient().getTableConfig().getProps(),
+ 0,
+ Long.MAX_VALUE,
+ usePosition);
+ try {
+ fileGroupReader.initRecordIterators();
+ HoodieFileGroupReaderIterator<InternalRow> recordIterator
+ = (HoodieFileGroupReaderIterator<InternalRow>)
fileGroupReader.getClosableIterator();
+ StructType sparkSchema =
AvroConversionUtils.convertAvroSchemaToStructType(readerSchema);
+ while (recordIterator.hasNext()) {
+ InternalRow row = recordIterator.next();
+ HoodieKey recordKey = new HoodieKey(
+ row.getString(HoodieRecord.RECORD_KEY_META_FIELD_ORD),
+ row.getString(HoodieRecord.PARTITION_PATH_META_FIELD_ORD));
+ HoodieSparkRecord record = new HoodieSparkRecord(recordKey, row,
sparkSchema, false);
+ Option recordMetadata = record.getMetadata();
+ if (!partitionPath.equals(record.getPartitionPath())) {
+ HoodieUpsertException failureEx = new
HoodieUpsertException("mismatched partition path, record partition: "
+ + record.getPartitionPath() + " but trying to insert into
partition: " + partitionPath);
+ writeStatus.markFailure(record, failureEx, recordMetadata);
+ continue;
+ }
+ try {
+ writeToFile(recordKey, record, readerSchema,
config.getPayloadConfig().getProps(), preserveMetadata);
+ writeStatus.markSuccess(record, recordMetadata);
+ } catch (Exception e) {
+ LOG.error("Error writing record " + record, e);
+ writeStatus.markFailure(record, e, recordMetadata);
+ }
+
+ }
+ // The stats of inserts, updates, and deletes are updated once at the end
Review Comment:
can we close the record iterator
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/SparkBroadcastManager.java:
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.table;
+
+import org.apache.hudi.SparkAdapterSupport$;
+import org.apache.hudi.SparkFileFormatInternalRowReaderContext;
+import org.apache.hudi.client.common.HoodieSparkEngineContext;
+import org.apache.hudi.common.engine.HoodieEngineContext;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.table.HoodieTableMetaClient;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.execution.datasources.FileFormat;
+import org.apache.spark.sql.execution.datasources.parquet.SparkParquetReader;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.sources.Filter;
+import org.apache.spark.util.SerializableConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import scala.Tuple2;
+import scala.collection.JavaConverters;
+
+public class SparkBroadcastManager extends EngineBroadcastManager {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(SparkBroadcastManager.class);
+
+ private final transient HoodieEngineContext context;
+
+ protected Option<SparkParquetReader> parquetReaderOpt = Option.empty();
+ protected Broadcast<SQLConf> sqlConfBroadcast;
+ protected Broadcast<SparkParquetReader> parquetReaderBroadcast;
+ protected Broadcast<SerializableConfiguration> configurationBroadcast;
+
+ public SparkBroadcastManager(HoodieEngineContext context) {
+ this.context = context;
+ }
+
+ // Prepare broadcast variables.
+ @Override
+ public void prepareAndBroadcast() {
+ // This needs to be fixed.
+ if (!(context instanceof HoodieSparkEngineContext)) {
+ throw new HoodieIOException("Expected to be called using Engine's
context and not local context");
+ }
+
+ HoodieSparkEngineContext hoodieSparkEngineContext =
(HoodieSparkEngineContext) context;
+ SQLConf sqlConf =
hoodieSparkEngineContext.getSqlContext().sessionState().conf();
+ JavaSparkContext jsc = hoodieSparkEngineContext.jsc();
+
+ // TODO: Confirm what is the correct way to set this config.
+ boolean returningBatch = sqlConf.parquetVectorizedReaderEnabled();
+ scala.collection.immutable.Map<String, String> options =
+ scala.collection.immutable.Map$.MODULE$.<String, String>empty()
+ .$plus(new Tuple2<>(FileFormat.OPTION_RETURNING_BATCH(),
Boolean.toString(returningBatch)));
+
+ // Do broadcast.
+ sqlConfBroadcast = jsc.broadcast(sqlConf);
+ configurationBroadcast = jsc.broadcast(new
SerializableConfiguration(jsc.hadoopConfiguration()));
+ // TODO: Disable vectorization as of now. Assign it based on relevant
settings.
+ // TODO: Verify if we can construct the reader on the executor side if we
has broadcast all necessary variables.
+ parquetReaderOpt =
Option.of(SparkAdapterSupport$.MODULE$.sparkAdapter().createParquetFileReader(
+ false, sqlConfBroadcast.getValue(), options,
configurationBroadcast.getValue().value()));
+ parquetReaderBroadcast = jsc.broadcast(parquetReaderOpt.get());
+ }
+
+ @Override
+ public Option<HoodieReaderContext>
retrieveFileGroupReaderContext(StoragePath basePath) {
+ if (parquetReaderBroadcast == null) {
+ LOG.warn("ParquetReader is not broadcast; cannot create file group
reader");
+ return Option.empty();
Review Comment:
should we throw exception in this case?
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkMergeHandleV2.java:
##########
@@ -0,0 +1,410 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodiePartitionMetadata;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import
org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator;
+import org.apache.hudi.common.table.read.HoodieFileGroupReaderStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCorruptedDataException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileWriter;
+import org.apache.hudi.io.storage.HoodieFileWriterFactory;
+import org.apache.hudi.io.storage.HoodieIOFactory;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+
+@SuppressWarnings("Duplicates")
+/**
+ * Handle to merge incoming records to those in storage.
+ * <p>
+ * Simplified Logic:
+ * For every existing record
+ * Check if there is a new record coming in. If yes, merge two records and
write to file
+ * else write the record as is
+ * For all pending records from incoming batch, write to file.
+ *
+ * Illustration with simple data.
+ * Incoming data:
+ * rec1_2, rec4_2, rec5_1, rec6_1
+ * Existing data:
+ * rec1_1, rec2_1, rec3_1, rec4_1
+ *
+ * For every existing record, merge w/ incoming if required and write to
storage.
+ * => rec1_1 and rec1_2 is merged to write rec1_2 to storage
+ * => rec2_1 is written as is
+ * => rec3_1 is written as is
+ * => rec4_2 and rec4_1 is merged to write rec4_2 to storage
+ * Write all pending records from incoming set to storage
+ * => rec5_1 and rec6_1
+ *
+ * Final snapshot in storage
+ * rec1_2, rec2_1, rec3_1, rec4_2, rec5_1, rec6_1
+ *
+ * </p>
+ */
+@NotThreadSafe
+public class HoodieSparkMergeHandleV2<T, I, K, O> extends HoodieWriteHandle<T,
I, K, O> {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(HoodieMergeHandle.class);
+
+ protected Set<String> writtenRecordKeys;
+ protected HoodieReaderContext readerContext;
+ protected HoodieFileWriter fileWriter;
+ protected boolean preserveMetadata = false;
+
+ protected StoragePath newFilePath;
+ protected Option<StoragePath> oldFilePath = Option.empty();
+ protected long recordsWritten = 0;
+ // TODO(yihua): audit delete stats because file group reader may not return
deletes
+ protected long recordsDeleted = 0;
+ protected long updatedRecordsWritten = 0;
+ protected long insertRecordsWritten = 0;
+ protected Option<BaseKeyGenerator> keyGeneratorOpt;
+ protected FileSlice fileSlice;
+ private Option<HoodieBaseFile> baseFileToMergeOpt;
+
+ protected Option<String[]> partitionFields = Option.empty();
+ protected Object[] partitionValues = new Object[0];
+ protected Configuration conf;
+
+ /**
+ * Called by compactor code path using the file group reader.
+ */
+ public HoodieSparkMergeHandleV2(HoodieWriteConfig config, String
instantTime, HoodieTable<T, I, K, O> hoodieTable,
+ CompactionOperation operation,
TaskContextSupplier taskContextSupplier, Option<BaseKeyGenerator>
keyGeneratorOpt,
+ HoodieReaderContext readerContext,
Configuration conf) {
+ super(config, instantTime, operation.getPartitionPath(),
operation.getFileId(), hoodieTable, taskContextSupplier);
+ this.readerContext = readerContext;
+ this.conf = conf;
+ Option<HoodieBaseFile> baseFileOpt =
+ operation.getBaseFile(config.getBasePath(),
operation.getPartitionPath());
+ List<HoodieLogFile> logFiles =
operation.getDeltaFileNames().stream().map(p ->
+ new HoodieLogFile(new StoragePath(FSUtils.constructAbsolutePath(
+ config.getBasePath(), operation.getPartitionPath()), p)))
+ .collect(Collectors.toList());
+ this.fileSlice = new FileSlice(
+ operation.getFileGroupId(),
+ operation.getBaseInstantTime(),
+ baseFileOpt.isPresent() ? baseFileOpt.get() : null,
+ logFiles);
+ this.preserveMetadata = true;
+ init(fileId, this.partitionPath, baseFileOpt);
+ validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields());
+ }
+
+ private void validateAndSetAndKeyGenProps(Option<BaseKeyGenerator>
keyGeneratorOpt, boolean populateMetaFields) {
+ ValidationUtils.checkArgument(populateMetaFields ==
!keyGeneratorOpt.isPresent());
+ this.keyGeneratorOpt = keyGeneratorOpt;
+ }
+
+ /**
+ * Extract old file path, initialize StorageWriter and WriteStatus.
+ */
+ private void init(String fileId, String partitionPath,
Option<HoodieBaseFile> baseFileToMerge) {
+ LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" +
fileId);
+ this.baseFileToMergeOpt = baseFileToMerge;
+ this.writtenRecordKeys = new HashSet<>();
+ writeStatus.setStat(new HoodieWriteStat());
+ try {
+ Option<String> latestValidFilePath = Option.empty();
+ if (baseFileToMerge.isPresent()) {
+ latestValidFilePath = Option.of(baseFileToMerge.get().getFileName());
+
writeStatus.getStat().setPrevCommit(baseFileToMerge.get().getCommitTime());
+ // At the moment, we only support SI for overwrite with latest
payload. So, we don't need to embed entire file slice here.
+ // HUDI-8518 will be taken up to fix it for any payload during which
we might require entire file slice to be set here.
+ // Already AppendHandle adds all logs file from current file slice to
HoodieDeltaWriteStat.
+ writeStatus.getStat().setPrevBaseFile(latestValidFilePath.get());
+ } else {
+ writeStatus.getStat().setPrevCommit(HoodieWriteStat.NULL_COMMIT);
+ }
+
+ HoodiePartitionMetadata partitionMetadata = new
HoodiePartitionMetadata(storage, instantTime,
+ new StoragePath(config.getBasePath()),
+ FSUtils.constructAbsolutePath(config.getBasePath(), partitionPath),
+ hoodieTable.getPartitionMetafileFormat());
+ partitionMetadata.trySave();
+
+ String newFileName = FSUtils.makeBaseFileName(instantTime, writeToken,
fileId, hoodieTable.getBaseFileExtension());
+ makeOldAndNewFilePaths(partitionPath, latestValidFilePath, newFileName);
+
+ LOG.info(String.format(
+ "Merging data from file group %s, to a new base file %s", fileId,
newFilePath.toString()));
+ // file name is same for all records, in this bunch
+ writeStatus.setFileId(fileId);
+ writeStatus.setPartitionPath(partitionPath);
+ writeStatus.getStat().setPartitionPath(partitionPath);
+ writeStatus.getStat().setFileId(fileId);
+ setWriteStatusPath();
+
+ // Create Marker file,
+ // uses name of `newFilePath` instead of `newFileName`
+ // in case the sub-class may roll over the file handle name.
+ createMarkerFile(partitionPath, newFilePath.getName());
+
+ // Create the writer for writing the new version file
+ fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
newFilePath, hoodieTable.getStorage(),
+ config, writeSchemaWithMetaFields, taskContextSupplier,
HoodieRecord.HoodieRecordType.SPARK);
+ } catch (IOException io) {
+ LOG.error("Error in update task at commit " + instantTime, io);
+ writeStatus.setGlobalError(io);
+ throw new HoodieUpsertException("Failed to initialize HoodieUpdateHandle
for FileId: " + fileId + " on commit "
+ + instantTime + " on path " +
hoodieTable.getMetaClient().getBasePath(), io);
+ }
+ }
+
+ protected void setWriteStatusPath() {
+ writeStatus.getStat().setPath(new StoragePath(config.getBasePath()),
newFilePath);
+ }
+
+ protected void makeOldAndNewFilePaths(String partitionPath, Option<String>
oldFileName, String newFileName) {
+ if (oldFileName.isPresent()) {
+ oldFilePath = Option.of(makeNewFilePath(partitionPath,
oldFileName.get()));
+ }
+ newFilePath = makeNewFilePath(partitionPath, newFileName);
+ }
+
+ // TODO(yihua): is this still needed
+ /*
+ if (needsUpdateLocation()) {
+ record.unseal();
+ record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
+ record.seal();
+ }
+ */
+
+ public void write() {
Review Comment:
java docs for this method please
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkMergeHandleV2.java:
##########
@@ -0,0 +1,410 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodiePartitionMetadata;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import
org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator;
+import org.apache.hudi.common.table.read.HoodieFileGroupReaderStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCorruptedDataException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileWriter;
+import org.apache.hudi.io.storage.HoodieFileWriterFactory;
+import org.apache.hudi.io.storage.HoodieIOFactory;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+
+@SuppressWarnings("Duplicates")
+/**
+ * Handle to merge incoming records to those in storage.
+ * <p>
+ * Simplified Logic:
+ * For every existing record
+ * Check if there is a new record coming in. If yes, merge two records and
write to file
+ * else write the record as is
+ * For all pending records from incoming batch, write to file.
+ *
+ * Illustration with simple data.
+ * Incoming data:
+ * rec1_2, rec4_2, rec5_1, rec6_1
+ * Existing data:
+ * rec1_1, rec2_1, rec3_1, rec4_1
+ *
+ * For every existing record, merge w/ incoming if required and write to
storage.
+ * => rec1_1 and rec1_2 is merged to write rec1_2 to storage
+ * => rec2_1 is written as is
+ * => rec3_1 is written as is
+ * => rec4_2 and rec4_1 is merged to write rec4_2 to storage
+ * Write all pending records from incoming set to storage
+ * => rec5_1 and rec6_1
+ *
+ * Final snapshot in storage
+ * rec1_2, rec2_1, rec3_1, rec4_2, rec5_1, rec6_1
+ *
+ * </p>
+ */
+@NotThreadSafe
+public class HoodieSparkMergeHandleV2<T, I, K, O> extends HoodieWriteHandle<T,
I, K, O> {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(HoodieMergeHandle.class);
+
+ protected Set<String> writtenRecordKeys;
+ protected HoodieReaderContext readerContext;
+ protected HoodieFileWriter fileWriter;
+ protected boolean preserveMetadata = false;
Review Comment:
preserveMetadata is always true for compaction flows. we can just make this
static final and set to true
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkMergeHandleV2.java:
##########
@@ -0,0 +1,410 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodiePartitionMetadata;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import
org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator;
+import org.apache.hudi.common.table.read.HoodieFileGroupReaderStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCorruptedDataException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileWriter;
+import org.apache.hudi.io.storage.HoodieFileWriterFactory;
+import org.apache.hudi.io.storage.HoodieIOFactory;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+
+@SuppressWarnings("Duplicates")
+/**
+ * Handle to merge incoming records to those in storage.
+ * <p>
+ * Simplified Logic:
+ * For every existing record
+ * Check if there is a new record coming in. If yes, merge two records and
write to file
+ * else write the record as is
+ * For all pending records from incoming batch, write to file.
+ *
+ * Illustration with simple data.
+ * Incoming data:
+ * rec1_2, rec4_2, rec5_1, rec6_1
+ * Existing data:
+ * rec1_1, rec2_1, rec3_1, rec4_1
+ *
+ * For every existing record, merge w/ incoming if required and write to
storage.
+ * => rec1_1 and rec1_2 is merged to write rec1_2 to storage
+ * => rec2_1 is written as is
+ * => rec3_1 is written as is
+ * => rec4_2 and rec4_1 is merged to write rec4_2 to storage
+ * Write all pending records from incoming set to storage
+ * => rec5_1 and rec6_1
+ *
+ * Final snapshot in storage
+ * rec1_2, rec2_1, rec3_1, rec4_2, rec5_1, rec6_1
+ *
+ * </p>
+ */
+@NotThreadSafe
+public class HoodieSparkMergeHandleV2<T, I, K, O> extends HoodieWriteHandle<T,
I, K, O> {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(HoodieMergeHandle.class);
+
+ protected Set<String> writtenRecordKeys;
+ protected HoodieReaderContext readerContext;
+ protected HoodieFileWriter fileWriter;
+ protected boolean preserveMetadata = false;
+
+ protected StoragePath newFilePath;
+ protected Option<StoragePath> oldFilePath = Option.empty();
+ protected long recordsWritten = 0;
+ // TODO(yihua): audit delete stats because file group reader may not return
deletes
+ protected long recordsDeleted = 0;
+ protected long updatedRecordsWritten = 0;
+ protected long insertRecordsWritten = 0;
+ protected Option<BaseKeyGenerator> keyGeneratorOpt;
+ protected FileSlice fileSlice;
+ private Option<HoodieBaseFile> baseFileToMergeOpt;
+
+ protected Option<String[]> partitionFields = Option.empty();
+ protected Object[] partitionValues = new Object[0];
+ protected Configuration conf;
+
+ /**
+ * Called by compactor code path using the file group reader.
+ */
+ public HoodieSparkMergeHandleV2(HoodieWriteConfig config, String
instantTime, HoodieTable<T, I, K, O> hoodieTable,
+ CompactionOperation operation,
TaskContextSupplier taskContextSupplier, Option<BaseKeyGenerator>
keyGeneratorOpt,
+ HoodieReaderContext readerContext,
Configuration conf) {
+ super(config, instantTime, operation.getPartitionPath(),
operation.getFileId(), hoodieTable, taskContextSupplier);
+ this.readerContext = readerContext;
+ this.conf = conf;
+ Option<HoodieBaseFile> baseFileOpt =
+ operation.getBaseFile(config.getBasePath(),
operation.getPartitionPath());
+ List<HoodieLogFile> logFiles =
operation.getDeltaFileNames().stream().map(p ->
+ new HoodieLogFile(new StoragePath(FSUtils.constructAbsolutePath(
+ config.getBasePath(), operation.getPartitionPath()), p)))
+ .collect(Collectors.toList());
+ this.fileSlice = new FileSlice(
+ operation.getFileGroupId(),
+ operation.getBaseInstantTime(),
+ baseFileOpt.isPresent() ? baseFileOpt.get() : null,
+ logFiles);
+ this.preserveMetadata = true;
+ init(fileId, this.partitionPath, baseFileOpt);
+ validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields());
+ }
+
+ private void validateAndSetAndKeyGenProps(Option<BaseKeyGenerator>
keyGeneratorOpt, boolean populateMetaFields) {
+ ValidationUtils.checkArgument(populateMetaFields ==
!keyGeneratorOpt.isPresent());
+ this.keyGeneratorOpt = keyGeneratorOpt;
+ }
+
+ /**
+ * Extract old file path, initialize StorageWriter and WriteStatus.
+ */
+ private void init(String fileId, String partitionPath,
Option<HoodieBaseFile> baseFileToMerge) {
+ LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" +
fileId);
+ this.baseFileToMergeOpt = baseFileToMerge;
+ this.writtenRecordKeys = new HashSet<>();
+ writeStatus.setStat(new HoodieWriteStat());
+ try {
+ Option<String> latestValidFilePath = Option.empty();
+ if (baseFileToMerge.isPresent()) {
+ latestValidFilePath = Option.of(baseFileToMerge.get().getFileName());
+
writeStatus.getStat().setPrevCommit(baseFileToMerge.get().getCommitTime());
+ // At the moment, we only support SI for overwrite with latest
payload. So, we don't need to embed entire file slice here.
+ // HUDI-8518 will be taken up to fix it for any payload during which
we might require entire file slice to be set here.
+ // Already AppendHandle adds all logs file from current file slice to
HoodieDeltaWriteStat.
+ writeStatus.getStat().setPrevBaseFile(latestValidFilePath.get());
+ } else {
+ writeStatus.getStat().setPrevCommit(HoodieWriteStat.NULL_COMMIT);
+ }
+
+ HoodiePartitionMetadata partitionMetadata = new
HoodiePartitionMetadata(storage, instantTime,
+ new StoragePath(config.getBasePath()),
+ FSUtils.constructAbsolutePath(config.getBasePath(), partitionPath),
+ hoodieTable.getPartitionMetafileFormat());
+ partitionMetadata.trySave();
+
+ String newFileName = FSUtils.makeBaseFileName(instantTime, writeToken,
fileId, hoodieTable.getBaseFileExtension());
+ makeOldAndNewFilePaths(partitionPath, latestValidFilePath, newFileName);
+
+ LOG.info(String.format(
+ "Merging data from file group %s, to a new base file %s", fileId,
newFilePath.toString()));
+ // file name is same for all records, in this bunch
+ writeStatus.setFileId(fileId);
+ writeStatus.setPartitionPath(partitionPath);
+ writeStatus.getStat().setPartitionPath(partitionPath);
+ writeStatus.getStat().setFileId(fileId);
+ setWriteStatusPath();
+
+ // Create Marker file,
+ // uses name of `newFilePath` instead of `newFileName`
+ // in case the sub-class may roll over the file handle name.
+ createMarkerFile(partitionPath, newFilePath.getName());
+
+ // Create the writer for writing the new version file
+ fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
newFilePath, hoodieTable.getStorage(),
+ config, writeSchemaWithMetaFields, taskContextSupplier,
HoodieRecord.HoodieRecordType.SPARK);
+ } catch (IOException io) {
+ LOG.error("Error in update task at commit " + instantTime, io);
+ writeStatus.setGlobalError(io);
+ throw new HoodieUpsertException("Failed to initialize HoodieUpdateHandle
for FileId: " + fileId + " on commit "
+ + instantTime + " on path " +
hoodieTable.getMetaClient().getBasePath(), io);
+ }
+ }
+
+ protected void setWriteStatusPath() {
+ writeStatus.getStat().setPath(new StoragePath(config.getBasePath()),
newFilePath);
+ }
+
+ protected void makeOldAndNewFilePaths(String partitionPath, Option<String>
oldFileName, String newFileName) {
+ if (oldFileName.isPresent()) {
+ oldFilePath = Option.of(makeNewFilePath(partitionPath,
oldFileName.get()));
+ }
+ newFilePath = makeNewFilePath(partitionPath, newFileName);
+ }
+
+ // TODO(yihua): is this still needed
+ /*
+ if (needsUpdateLocation()) {
+ record.unseal();
+ record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
+ record.seal();
+ }
+ */
+
+ public void write() {
+ boolean usePosition =
config.getBooleanOrDefault(MERGE_USE_RECORD_POSITIONS);
+ Schema readerSchema;
+ Option<InternalSchema> internalSchemaOption = Option.empty();
+ if (!StringUtils.isNullOrEmpty(config.getInternalSchema())) {
+ readerSchema = HoodieAvroUtils.addMetadataFields(
+ new Schema.Parser().parse(config.getSchema()),
config.allowOperationMetadataField());
+ internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema());
+ } else {
+ readerSchema = HoodieAvroUtils.addMetadataFields(
+ new Schema.Parser().parse(config.getSchema()),
config.allowOperationMetadataField());
+ }
+ // TODO(yihua): reader schema is good enough for writer?
+ HoodieFileGroupReader<T> fileGroupReader = new HoodieFileGroupReader<>(
+ readerContext,
+ storage.newInstance(hoodieTable.getMetaClient().getBasePath(), new
HadoopStorageConfiguration(conf)),
+ hoodieTable.getMetaClient().getBasePath().toString(),
+ instantTime,
+ fileSlice,
+ readerSchema,
+ readerSchema,
+ internalSchemaOption,
+ hoodieTable.getMetaClient(),
+ hoodieTable.getMetaClient().getTableConfig().getProps(),
+ 0,
+ Long.MAX_VALUE,
+ usePosition);
+ try {
+ fileGroupReader.initRecordIterators();
+ HoodieFileGroupReaderIterator<InternalRow> recordIterator
+ = (HoodieFileGroupReaderIterator<InternalRow>)
fileGroupReader.getClosableIterator();
+ StructType sparkSchema =
AvroConversionUtils.convertAvroSchemaToStructType(readerSchema);
+ while (recordIterator.hasNext()) {
+ InternalRow row = recordIterator.next();
+ HoodieKey recordKey = new HoodieKey(
+ row.getString(HoodieRecord.RECORD_KEY_META_FIELD_ORD),
+ row.getString(HoodieRecord.PARTITION_PATH_META_FIELD_ORD));
+ HoodieSparkRecord record = new HoodieSparkRecord(recordKey, row,
sparkSchema, false);
+ Option recordMetadata = record.getMetadata();
+ if (!partitionPath.equals(record.getPartitionPath())) {
+ HoodieUpsertException failureEx = new
HoodieUpsertException("mismatched partition path, record partition: "
+ + record.getPartitionPath() + " but trying to insert into
partition: " + partitionPath);
+ writeStatus.markFailure(record, failureEx, recordMetadata);
+ continue;
+ }
+ try {
+ writeToFile(recordKey, record, readerSchema,
config.getPayloadConfig().getProps(), preserveMetadata);
+ writeStatus.markSuccess(record, recordMetadata);
+ } catch (Exception e) {
+ LOG.error("Error writing record " + record, e);
+ writeStatus.markFailure(record, e, recordMetadata);
+ }
+
+ }
+ // The stats of inserts, updates, and deletes are updated once at the end
+ HoodieFileGroupReaderStats stats = fileGroupReader.getStats();
+ this.insertRecordsWritten = stats.getNumInserts();
+ this.updatedRecordsWritten = stats.getNumUpdates();
+ this.recordsDeleted = stats.getNumDeletes();
+ this.recordsWritten = stats.getNumInserts() + stats.getNumUpdates();
+ } catch (IOException e) {
+ throw new HoodieUpsertException("Failed to compact file slice: " +
fileSlice, e);
+ }
+ }
+
+ protected void writeToFile(HoodieKey key, HoodieSparkRecord record, Schema
schema, Properties prop, boolean shouldPreserveRecordMetadata)
+ throws IOException {
+ // NOTE: `FILENAME_METADATA_FIELD` has to be rewritten to correctly point
to the
+ // file holding this record even in cases when overall metadata is
preserved
+ MetadataValues metadataValues = new
MetadataValues().setFileName(newFilePath.getName());
+ HoodieRecord populatedRecord = record.prependMetaFields(schema,
writeSchemaWithMetaFields, metadataValues, prop);
+
+ if (shouldPreserveRecordMetadata) {
+ fileWriter.write(key.getRecordKey(), populatedRecord,
writeSchemaWithMetaFields);
+ } else {
+ fileWriter.writeWithMetadata(key, populatedRecord,
writeSchemaWithMetaFields);
+ }
+ }
+
+ @Override
+ public List<WriteStatus> close() {
+ try {
+ if (isClosed()) {
+ // Handle has already been closed
+ return Collections.emptyList();
+ }
+
+ markClosed();
+
+ writtenRecordKeys = null;
+
+ fileWriter.close();
+ fileWriter = null;
+
+ long fileSizeInBytes = storage.getPathInfo(newFilePath).getLength();
+ HoodieWriteStat stat = writeStatus.getStat();
+
Review Comment:
I assume you will extend from HoodieMergeHandle and get rid of this
duplicated code
##########
hudi-common/src/main/java/org/apache/hudi/common/table/read/HoodieFileGroupReaderStats.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.common.table.read;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+@NotThreadSafe
+public class HoodieFileGroupReaderStats {
Review Comment:
+1
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/HoodieSparkMergeHandleV2.java:
##########
@@ -0,0 +1,410 @@
+/*
+ * 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.io;
+
+import org.apache.hudi.AvroConversionUtils;
+import org.apache.hudi.avro.HoodieAvroUtils;
+import org.apache.hudi.client.WriteStatus;
+import org.apache.hudi.common.engine.HoodieReaderContext;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.CompactionOperation;
+import org.apache.hudi.common.model.FileSlice;
+import org.apache.hudi.common.model.HoodieBaseFile;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieLogFile;
+import org.apache.hudi.common.model.HoodiePartitionMetadata;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieWriteStat;
+import org.apache.hudi.common.model.HoodieWriteStat.RuntimeStats;
+import org.apache.hudi.common.model.IOType;
+import org.apache.hudi.common.model.MetadataValues;
+import org.apache.hudi.common.table.read.HoodieFileGroupReader;
+import
org.apache.hudi.common.table.read.HoodieFileGroupReader.HoodieFileGroupReaderIterator;
+import org.apache.hudi.common.table.read.HoodieFileGroupReaderStats;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.exception.HoodieCorruptedDataException;
+import org.apache.hudi.exception.HoodieUpsertException;
+import org.apache.hudi.internal.schema.InternalSchema;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+import org.apache.hudi.io.storage.HoodieFileReader;
+import org.apache.hudi.io.storage.HoodieFileWriter;
+import org.apache.hudi.io.storage.HoodieFileWriterFactory;
+import org.apache.hudi.io.storage.HoodieIOFactory;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.storage.StoragePath;
+import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration;
+import org.apache.hudi.table.HoodieTable;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.NotThreadSafe;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import static
org.apache.hudi.common.config.HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS;
+
+@SuppressWarnings("Duplicates")
+/**
+ * Handle to merge incoming records to those in storage.
+ * <p>
+ * Simplified Logic:
+ * For every existing record
+ * Check if there is a new record coming in. If yes, merge two records and
write to file
+ * else write the record as is
+ * For all pending records from incoming batch, write to file.
+ *
+ * Illustration with simple data.
+ * Incoming data:
+ * rec1_2, rec4_2, rec5_1, rec6_1
+ * Existing data:
+ * rec1_1, rec2_1, rec3_1, rec4_1
+ *
+ * For every existing record, merge w/ incoming if required and write to
storage.
+ * => rec1_1 and rec1_2 is merged to write rec1_2 to storage
+ * => rec2_1 is written as is
+ * => rec3_1 is written as is
+ * => rec4_2 and rec4_1 is merged to write rec4_2 to storage
+ * Write all pending records from incoming set to storage
+ * => rec5_1 and rec6_1
+ *
+ * Final snapshot in storage
+ * rec1_2, rec2_1, rec3_1, rec4_2, rec5_1, rec6_1
+ *
+ * </p>
+ */
+@NotThreadSafe
+public class HoodieSparkMergeHandleV2<T, I, K, O> extends HoodieWriteHandle<T,
I, K, O> {
+
+ private static final Logger LOG =
LoggerFactory.getLogger(HoodieMergeHandle.class);
+
+ protected Set<String> writtenRecordKeys;
+ protected HoodieReaderContext readerContext;
+ protected HoodieFileWriter fileWriter;
+ protected boolean preserveMetadata = false;
+
+ protected StoragePath newFilePath;
+ protected Option<StoragePath> oldFilePath = Option.empty();
+ protected long recordsWritten = 0;
+ // TODO(yihua): audit delete stats because file group reader may not return
deletes
+ protected long recordsDeleted = 0;
+ protected long updatedRecordsWritten = 0;
+ protected long insertRecordsWritten = 0;
+ protected Option<BaseKeyGenerator> keyGeneratorOpt;
+ protected FileSlice fileSlice;
+ private Option<HoodieBaseFile> baseFileToMergeOpt;
+
+ protected Option<String[]> partitionFields = Option.empty();
+ protected Object[] partitionValues = new Object[0];
+ protected Configuration conf;
+
+ /**
+ * Called by compactor code path using the file group reader.
+ */
+ public HoodieSparkMergeHandleV2(HoodieWriteConfig config, String
instantTime, HoodieTable<T, I, K, O> hoodieTable,
+ CompactionOperation operation,
TaskContextSupplier taskContextSupplier, Option<BaseKeyGenerator>
keyGeneratorOpt,
+ HoodieReaderContext readerContext,
Configuration conf) {
+ super(config, instantTime, operation.getPartitionPath(),
operation.getFileId(), hoodieTable, taskContextSupplier);
+ this.readerContext = readerContext;
+ this.conf = conf;
+ Option<HoodieBaseFile> baseFileOpt =
+ operation.getBaseFile(config.getBasePath(),
operation.getPartitionPath());
+ List<HoodieLogFile> logFiles =
operation.getDeltaFileNames().stream().map(p ->
+ new HoodieLogFile(new StoragePath(FSUtils.constructAbsolutePath(
+ config.getBasePath(), operation.getPartitionPath()), p)))
+ .collect(Collectors.toList());
+ this.fileSlice = new FileSlice(
+ operation.getFileGroupId(),
+ operation.getBaseInstantTime(),
+ baseFileOpt.isPresent() ? baseFileOpt.get() : null,
+ logFiles);
+ this.preserveMetadata = true;
+ init(fileId, this.partitionPath, baseFileOpt);
+ validateAndSetAndKeyGenProps(keyGeneratorOpt, config.populateMetaFields());
+ }
+
+ private void validateAndSetAndKeyGenProps(Option<BaseKeyGenerator>
keyGeneratorOpt, boolean populateMetaFields) {
+ ValidationUtils.checkArgument(populateMetaFields ==
!keyGeneratorOpt.isPresent());
+ this.keyGeneratorOpt = keyGeneratorOpt;
+ }
+
+ /**
+ * Extract old file path, initialize StorageWriter and WriteStatus.
+ */
+ private void init(String fileId, String partitionPath,
Option<HoodieBaseFile> baseFileToMerge) {
+ LOG.info("partitionPath:" + partitionPath + ", fileId to be merged:" +
fileId);
+ this.baseFileToMergeOpt = baseFileToMerge;
+ this.writtenRecordKeys = new HashSet<>();
+ writeStatus.setStat(new HoodieWriteStat());
+ try {
+ Option<String> latestValidFilePath = Option.empty();
+ if (baseFileToMerge.isPresent()) {
+ latestValidFilePath = Option.of(baseFileToMerge.get().getFileName());
+
writeStatus.getStat().setPrevCommit(baseFileToMerge.get().getCommitTime());
+ // At the moment, we only support SI for overwrite with latest
payload. So, we don't need to embed entire file slice here.
+ // HUDI-8518 will be taken up to fix it for any payload during which
we might require entire file slice to be set here.
+ // Already AppendHandle adds all logs file from current file slice to
HoodieDeltaWriteStat.
+ writeStatus.getStat().setPrevBaseFile(latestValidFilePath.get());
+ } else {
+ writeStatus.getStat().setPrevCommit(HoodieWriteStat.NULL_COMMIT);
+ }
+
+ HoodiePartitionMetadata partitionMetadata = new
HoodiePartitionMetadata(storage, instantTime,
+ new StoragePath(config.getBasePath()),
+ FSUtils.constructAbsolutePath(config.getBasePath(), partitionPath),
+ hoodieTable.getPartitionMetafileFormat());
+ partitionMetadata.trySave();
+
+ String newFileName = FSUtils.makeBaseFileName(instantTime, writeToken,
fileId, hoodieTable.getBaseFileExtension());
+ makeOldAndNewFilePaths(partitionPath, latestValidFilePath, newFileName);
+
+ LOG.info(String.format(
+ "Merging data from file group %s, to a new base file %s", fileId,
newFilePath.toString()));
+ // file name is same for all records, in this bunch
+ writeStatus.setFileId(fileId);
+ writeStatus.setPartitionPath(partitionPath);
+ writeStatus.getStat().setPartitionPath(partitionPath);
+ writeStatus.getStat().setFileId(fileId);
+ setWriteStatusPath();
+
+ // Create Marker file,
+ // uses name of `newFilePath` instead of `newFileName`
+ // in case the sub-class may roll over the file handle name.
+ createMarkerFile(partitionPath, newFilePath.getName());
+
+ // Create the writer for writing the new version file
+ fileWriter = HoodieFileWriterFactory.getFileWriter(instantTime,
newFilePath, hoodieTable.getStorage(),
+ config, writeSchemaWithMetaFields, taskContextSupplier,
HoodieRecord.HoodieRecordType.SPARK);
+ } catch (IOException io) {
+ LOG.error("Error in update task at commit " + instantTime, io);
+ writeStatus.setGlobalError(io);
+ throw new HoodieUpsertException("Failed to initialize HoodieUpdateHandle
for FileId: " + fileId + " on commit "
+ + instantTime + " on path " +
hoodieTable.getMetaClient().getBasePath(), io);
+ }
+ }
+
+ protected void setWriteStatusPath() {
+ writeStatus.getStat().setPath(new StoragePath(config.getBasePath()),
newFilePath);
+ }
+
+ protected void makeOldAndNewFilePaths(String partitionPath, Option<String>
oldFileName, String newFileName) {
+ if (oldFileName.isPresent()) {
+ oldFilePath = Option.of(makeNewFilePath(partitionPath,
oldFileName.get()));
+ }
+ newFilePath = makeNewFilePath(partitionPath, newFileName);
+ }
+
+ // TODO(yihua): is this still needed
+ /*
+ if (needsUpdateLocation()) {
+ record.unseal();
+ record.setNewLocation(new HoodieRecordLocation(instantTime, fileId));
+ record.seal();
+ }
+ */
+
+ public void write() {
+ boolean usePosition =
config.getBooleanOrDefault(MERGE_USE_RECORD_POSITIONS);
+ Schema readerSchema;
+ Option<InternalSchema> internalSchemaOption = Option.empty();
+ if (!StringUtils.isNullOrEmpty(config.getInternalSchema())) {
+ readerSchema = HoodieAvroUtils.addMetadataFields(
+ new Schema.Parser().parse(config.getSchema()),
config.allowOperationMetadataField());
+ internalSchemaOption = SerDeHelper.fromJson(config.getInternalSchema());
+ } else {
+ readerSchema = HoodieAvroUtils.addMetadataFields(
+ new Schema.Parser().parse(config.getSchema()),
config.allowOperationMetadataField());
+ }
+ // TODO(yihua): reader schema is good enough for writer?
+ HoodieFileGroupReader<T> fileGroupReader = new HoodieFileGroupReader<>(
+ readerContext,
+ storage.newInstance(hoodieTable.getMetaClient().getBasePath(), new
HadoopStorageConfiguration(conf)),
+ hoodieTable.getMetaClient().getBasePath().toString(),
+ instantTime,
+ fileSlice,
+ readerSchema,
+ readerSchema,
+ internalSchemaOption,
+ hoodieTable.getMetaClient(),
+ hoodieTable.getMetaClient().getTableConfig().getProps(),
+ 0,
+ Long.MAX_VALUE,
+ usePosition);
+ try {
+ fileGroupReader.initRecordIterators();
+ HoodieFileGroupReaderIterator<InternalRow> recordIterator
+ = (HoodieFileGroupReaderIterator<InternalRow>)
fileGroupReader.getClosableIterator();
+ StructType sparkSchema =
AvroConversionUtils.convertAvroSchemaToStructType(readerSchema);
+ while (recordIterator.hasNext()) {
+ InternalRow row = recordIterator.next();
+ HoodieKey recordKey = new HoodieKey(
+ row.getString(HoodieRecord.RECORD_KEY_META_FIELD_ORD),
+ row.getString(HoodieRecord.PARTITION_PATH_META_FIELD_ORD));
+ HoodieSparkRecord record = new HoodieSparkRecord(recordKey, row,
sparkSchema, false);
+ Option recordMetadata = record.getMetadata();
+ if (!partitionPath.equals(record.getPartitionPath())) {
+ HoodieUpsertException failureEx = new
HoodieUpsertException("mismatched partition path, record partition: "
+ + record.getPartitionPath() + " but trying to insert into
partition: " + partitionPath);
+ writeStatus.markFailure(record, failureEx, recordMetadata);
+ continue;
+ }
+ try {
+ writeToFile(recordKey, record, readerSchema,
config.getPayloadConfig().getProps(), preserveMetadata);
+ writeStatus.markSuccess(record, recordMetadata);
+ } catch (Exception e) {
+ LOG.error("Error writing record " + record, e);
+ writeStatus.markFailure(record, e, recordMetadata);
+ }
+
+ }
+ // The stats of inserts, updates, and deletes are updated once at the end
Review Comment:
and also fileGroupReader as well
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]