[HUDI-1598] Write as minor batches during one checkpoint interval for the new writer (#2553)
This commit is contained in:
@@ -53,11 +53,11 @@ public class HoodieCreateHandle<T extends HoodieRecordPayload, I, K, O> extends
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieCreateHandle.class);
|
||||
|
||||
private final HoodieFileWriter<IndexedRecord> fileWriter;
|
||||
private final Path path;
|
||||
private long recordsWritten = 0;
|
||||
private long insertRecordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
protected final HoodieFileWriter<IndexedRecord> fileWriter;
|
||||
protected final Path path;
|
||||
protected long recordsWritten = 0;
|
||||
protected long insertRecordsWritten = 0;
|
||||
protected long recordsDeleted = 0;
|
||||
private Map<String, HoodieRecord<T>> recordMap;
|
||||
private boolean useWriterSchema = false;
|
||||
|
||||
|
||||
@@ -95,10 +95,10 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
protected HoodieFileWriter<IndexedRecord> fileWriter;
|
||||
|
||||
protected Path newFilePath;
|
||||
private Path oldFilePath;
|
||||
protected Path oldFilePath;
|
||||
protected long recordsWritten = 0;
|
||||
private long recordsDeleted = 0;
|
||||
private long updatedRecordsWritten = 0;
|
||||
protected long recordsDeleted = 0;
|
||||
protected long updatedRecordsWritten = 0;
|
||||
protected long insertRecordsWritten = 0;
|
||||
protected boolean useWriterSchema;
|
||||
private HoodieBaseFile baseFileToMerge;
|
||||
@@ -132,6 +132,13 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
return writerSchema;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the data file name.
|
||||
*/
|
||||
protected String generatesDataFileName() {
|
||||
return FSUtils.makeDataFileName(instantTime, writeToken, fileId, hoodieTable.getBaseFileExtension());
|
||||
}
|
||||
|
||||
/**
|
||||
* Extract old file path, initialize StorageWriter and WriteStatus.
|
||||
*/
|
||||
@@ -149,7 +156,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
partitionMetadata.trySave(getPartitionId());
|
||||
|
||||
oldFilePath = new Path(config.getBasePath() + "/" + partitionPath + "/" + latestValidFilePath);
|
||||
String newFileName = FSUtils.makeDataFileName(instantTime, writeToken, fileId, hoodieTable.getBaseFileExtension());
|
||||
String newFileName = generatesDataFileName();
|
||||
String relativePath = new Path((partitionPath.isEmpty() ? "" : partitionPath + "/")
|
||||
+ newFileName).toString();
|
||||
newFilePath = new Path(config.getBasePath(), relativePath);
|
||||
@@ -177,18 +184,25 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
}
|
||||
|
||||
/**
|
||||
* Load the new incoming records in a map and return partitionPath.
|
||||
* Initialize a spillable map for incoming records.
|
||||
*/
|
||||
private void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
protected void initializeIncomingRecordsMap() {
|
||||
try {
|
||||
// Load the new records in a map
|
||||
long memoryForMerge = IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config.getProps());
|
||||
LOG.info("MaxMemoryPerPartitionMerge => " + memoryForMerge);
|
||||
this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(),
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(writerSchema));
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(writerSchema));
|
||||
} catch (IOException io) {
|
||||
throw new HoodieIOException("Cannot instantiate an ExternalSpillableMap", io);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Load the new incoming records in a map and return partitionPath.
|
||||
*/
|
||||
protected void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
initializeIncomingRecordsMap();
|
||||
while (newRecordsItr.hasNext()) {
|
||||
HoodieRecord<T> record = newRecordsItr.next();
|
||||
// update the new location of the record, so we know where to find it next
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordLocation;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
@@ -39,6 +40,10 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.index.FlinkHoodieIndex;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.io.FlinkCreateHandle;
|
||||
import org.apache.hudi.io.FlinkMergeHandle;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.io.MiniBatchHandle;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.table.HoodieFlinkTable;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
@@ -50,6 +55,7 @@ import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Comparator;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
@@ -58,12 +64,19 @@ import java.util.stream.Collectors;
|
||||
public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
AbstractHoodieWriteClient<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
|
||||
/**
|
||||
* FileID to write handle mapping in order to record the write handles for each file group,
|
||||
* so that we can append the mini-batch data buffer incrementally.
|
||||
*/
|
||||
private Map<String, HoodieWriteHandle<?, ?, ?, ?>> bucketToHandles;
|
||||
|
||||
public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
|
||||
super(context, clientConfig);
|
||||
this(context, clientConfig, false);
|
||||
}
|
||||
|
||||
public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending) {
|
||||
super(context, writeConfig, rollbackPending);
|
||||
this.bucketToHandles = new HashMap<>();
|
||||
}
|
||||
|
||||
public HoodieFlinkWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig, boolean rollbackPending,
|
||||
@@ -111,7 +124,23 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
|
||||
table.validateUpsertSchema();
|
||||
preWrite(instantTime, WriteOperationType.UPSERT);
|
||||
HoodieWriteMetadata<List<WriteStatus>> result = table.upsert(context, instantTime, records);
|
||||
final HoodieRecord<T> record = records.get(0);
|
||||
final HoodieRecordLocation loc = record.getCurrentLocation();
|
||||
final String fileID = loc.getFileId();
|
||||
final boolean isInsert = loc.getInstantTime().equals("I");
|
||||
final HoodieWriteHandle<?, ?, ?, ?> writeHandle;
|
||||
if (bucketToHandles.containsKey(fileID)) {
|
||||
writeHandle = bucketToHandles.get(fileID);
|
||||
} else {
|
||||
// create the write handle if not exists
|
||||
writeHandle = isInsert
|
||||
? new FlinkCreateHandle<>(getConfig(), instantTime, table, record.getPartitionPath(),
|
||||
fileID, table.getTaskContextSupplier())
|
||||
: new FlinkMergeHandle<>(getConfig(), instantTime, table, records.listIterator(), record.getPartitionPath(),
|
||||
fileID, table.getTaskContextSupplier());
|
||||
bucketToHandles.put(fileID, writeHandle);
|
||||
}
|
||||
HoodieWriteMetadata<List<WriteStatus>> result = ((HoodieFlinkTable<T>) table).upsert(context, writeHandle, instantTime, records);
|
||||
if (result.getIndexLookupDuration().isPresent()) {
|
||||
metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
|
||||
}
|
||||
@@ -202,6 +231,17 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
return getTableAndInitCtx(metaClient, operationType);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clean the write handles within a checkpoint interval, this operation
|
||||
* would close the underneath file handles.
|
||||
*/
|
||||
public void cleanHandles() {
|
||||
this.bucketToHandles.values().forEach(handle -> {
|
||||
((MiniBatchHandle) handle).finishWrite();
|
||||
});
|
||||
this.bucketToHandles.clear();
|
||||
}
|
||||
|
||||
private HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) {
|
||||
if (operationType == WriteOperationType.DELETE) {
|
||||
setWriteSchemaForDeletes(metaClient);
|
||||
|
||||
@@ -24,18 +24,21 @@ import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
/**
|
||||
* Create handle factory for Flink writer, use the specified fileID directly
|
||||
* because it is unique anyway.
|
||||
* Create handle factory for Flink writer, use the specified write handle directly.
|
||||
*/
|
||||
public class FlinkCreateHandleFactory<T extends HoodieRecordPayload, I, K, O>
|
||||
public class ExplicitCreateHandleFactory<T extends HoodieRecordPayload, I, K, O>
|
||||
extends CreateHandleFactory<T, I, K, O> {
|
||||
private HoodieWriteHandle<T, I, K, O> writeHandle;
|
||||
|
||||
public ExplicitCreateHandleFactory(HoodieWriteHandle<T, I, K, O> writeHandle) {
|
||||
this.writeHandle = writeHandle;
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteHandle<T, I, K, O> create(
|
||||
HoodieWriteConfig hoodieConfig, String commitTime,
|
||||
HoodieTable<T, I, K, O> hoodieTable, String partitionPath,
|
||||
String fileIdPrefix, TaskContextSupplier taskContextSupplier) {
|
||||
return new HoodieCreateHandle(hoodieConfig, commitTime, hoodieTable, partitionPath,
|
||||
fileIdPrefix, taskContextSupplier);
|
||||
return writeHandle;
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,148 @@
|
||||
/*
|
||||
* 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.client.WriteStatus;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.HoodieWriteStat;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieInsertException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A {@link HoodieCreateHandle} that supports create write incrementally(mini-batches).
|
||||
*
|
||||
* <p>For the first mini-batch, it initialize and set up the next file path to write,
|
||||
* but does not close the file writer until all the mini-batches write finish. Each mini-batch
|
||||
* data are appended to the same file.
|
||||
*
|
||||
* @param <T> Payload type
|
||||
* @param <I> Input type
|
||||
* @param <K> Key type
|
||||
* @param <O> Output type
|
||||
*/
|
||||
public class FlinkCreateHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
extends HoodieCreateHandle<T, I, K, O> implements MiniBatchHandle {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(FlinkCreateHandle.class);
|
||||
private long lastFileSize = 0L;
|
||||
|
||||
public FlinkCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileId, TaskContextSupplier taskContextSupplier) {
|
||||
this(config, instantTime, hoodieTable, partitionPath, fileId, getWriterSchemaIncludingAndExcludingMetadataPair(config),
|
||||
taskContextSupplier);
|
||||
}
|
||||
|
||||
public FlinkCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileId, Pair<Schema, Schema> writerSchemaIncludingAndExcludingMetadataPair,
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable, partitionPath, fileId, writerSchemaIncludingAndExcludingMetadataPair,
|
||||
taskContextSupplier);
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by the compactor code path.
|
||||
*/
|
||||
public FlinkCreateHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
String partitionPath, String fileId, Map<String, HoodieRecord<T>> recordMap,
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable, partitionPath, fileId, recordMap, taskContextSupplier);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the incremental write status. In mini-batch write mode,
|
||||
* this handle would be reused for a checkpoint bucket(the bucket is appended as mini-batches),
|
||||
* thus, after a mini-batch append finish, we do not close the underneath writer but return
|
||||
* the incremental WriteStatus instead.
|
||||
*
|
||||
* @return the incremental write status
|
||||
*/
|
||||
private WriteStatus getIncrementalWriteStatus() {
|
||||
try {
|
||||
long fileSizeInBytes = FSUtils.getFileSize(fs, path);
|
||||
setUpWriteStatus(fileSizeInBytes);
|
||||
// reset the write status
|
||||
recordsWritten = 0;
|
||||
recordsDeleted = 0;
|
||||
insertRecordsWritten = 0;
|
||||
this.lastFileSize = fileSizeInBytes;
|
||||
writeStatus.setTotalErrorRecords(0);
|
||||
return writeStatus;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Set up the write status.
|
||||
*
|
||||
* @param fileSizeInBytes File size in bytes
|
||||
* @throws IOException if error occurs
|
||||
*/
|
||||
private void setUpWriteStatus(long fileSizeInBytes) throws IOException {
|
||||
HoodieWriteStat stat = new HoodieWriteStat();
|
||||
stat.setPartitionPath(writeStatus.getPartitionPath());
|
||||
stat.setNumWrites(recordsWritten);
|
||||
stat.setNumDeletes(recordsDeleted);
|
||||
stat.setNumInserts(insertRecordsWritten);
|
||||
stat.setPrevCommit(HoodieWriteStat.NULL_COMMIT);
|
||||
stat.setFileId(writeStatus.getFileId());
|
||||
stat.setPath(new Path(config.getBasePath()), path);
|
||||
stat.setTotalWriteBytes(fileSizeInBytes - lastFileSize);
|
||||
stat.setFileSizeInBytes(fileSizeInBytes);
|
||||
stat.setTotalWriteErrors(writeStatus.getTotalErrorRecords());
|
||||
HoodieWriteStat.RuntimeStats runtimeStats = new HoodieWriteStat.RuntimeStats();
|
||||
runtimeStats.setTotalCreateTime(timer.endTimer());
|
||||
stat.setRuntimeStats(runtimeStats);
|
||||
timer = new HoodieTimer().startTimer();
|
||||
writeStatus.setStat(stat);
|
||||
}
|
||||
|
||||
public void finishWrite() {
|
||||
LOG.info("Closing the file " + writeStatus.getFileId() + " as we are done with all the records " + recordsWritten);
|
||||
try {
|
||||
fileWriter.close();
|
||||
} catch (IOException e) {
|
||||
throw new HoodieInsertException("Failed to close the Insert Handle for path " + path, e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Performs actions to durably, persist the current changes and returns a WriteStatus object.
|
||||
*/
|
||||
@Override
|
||||
public List<WriteStatus> close() {
|
||||
return Collections.singletonList(getIncrementalWriteStatus());
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,202 @@
|
||||
/*
|
||||
* 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.client.WriteStatus;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.util.HoodieTimer;
|
||||
import org.apache.hudi.common.util.collection.ExternalSpillableMap;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* A {@link HoodieMergeHandle} that supports merge write incrementally(small data buffers).
|
||||
*
|
||||
* <p>For a new data buffer, it initialize and set up the next file path to write,
|
||||
* and closes the file path when the data buffer write finish. When next data buffer
|
||||
* write starts, it rolls over to another new file. If all the data buffers write finish
|
||||
* for a checkpoint round, it renames the last new file path as the desired file name
|
||||
* (name with the expected file ID).
|
||||
*
|
||||
* @param <T> Payload type
|
||||
* @param <I> Input type
|
||||
* @param <K> Key type
|
||||
* @param <O> Output type
|
||||
*/
|
||||
public class FlinkMergeHandle<T extends HoodieRecordPayload, I, K, O>
|
||||
extends HoodieMergeHandle<T, I, K, O>
|
||||
implements MiniBatchHandle {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(FlinkMergeHandle.class);
|
||||
|
||||
/**
|
||||
* Records the current file handles number that rolls over.
|
||||
*/
|
||||
private int rollNumber = 0;
|
||||
/**
|
||||
* Records the rolled over file paths.
|
||||
*/
|
||||
private List<Path> rolloverPaths;
|
||||
/**
|
||||
* Whether it is the first time to generate file handle, E.G. the handle has not rolled over yet.
|
||||
*/
|
||||
private boolean needBootStrap = true;
|
||||
|
||||
public FlinkMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Iterator<HoodieRecord<T>> recordItr, String partitionPath, String fileId,
|
||||
TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable, recordItr, partitionPath, fileId, taskContextSupplier);
|
||||
rolloverPaths = new ArrayList<>();
|
||||
}
|
||||
|
||||
/**
|
||||
* Called by compactor code path.
|
||||
*/
|
||||
public FlinkMergeHandle(HoodieWriteConfig config, String instantTime, HoodieTable<T, I, K, O> hoodieTable,
|
||||
Map<String, HoodieRecord<T>> keyToNewRecords, String partitionPath, String fileId,
|
||||
HoodieBaseFile dataFileToBeMerged, TaskContextSupplier taskContextSupplier) {
|
||||
super(config, instantTime, hoodieTable, keyToNewRecords, partitionPath, fileId,
|
||||
dataFileToBeMerged, taskContextSupplier);
|
||||
}
|
||||
|
||||
/**
|
||||
* Use the fileId + "-" + rollNumber as the new fileId of a mini-batch write.
|
||||
*/
|
||||
protected String generatesDataFileName() {
|
||||
return FSUtils.makeDataFileName(instantTime, writeToken, fileId + "-" + rollNumber, hoodieTable.getBaseFileExtension());
|
||||
}
|
||||
|
||||
public boolean isNeedBootStrap() {
|
||||
return needBootStrap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<WriteStatus> close() {
|
||||
List<WriteStatus> writeStatus = super.close();
|
||||
this.needBootStrap = false;
|
||||
return writeStatus;
|
||||
}
|
||||
|
||||
/**
|
||||
* The difference with the parent method is that there is no need to set up
|
||||
* locations for the records.
|
||||
*
|
||||
* @param fileId The file ID
|
||||
* @param newRecordsItr The incremental records iterator
|
||||
*/
|
||||
@Override
|
||||
protected void init(String fileId, Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
initializeIncomingRecordsMap();
|
||||
while (newRecordsItr.hasNext()) {
|
||||
HoodieRecord<T> record = newRecordsItr.next();
|
||||
// NOTE: Once Records are added to map (spillable-map), DO NOT change it as they won't persist
|
||||
keyToNewRecords.put(record.getRecordKey(), record);
|
||||
}
|
||||
LOG.info(String.format("Number of entries in MemoryBasedMap => %d\n"
|
||||
+ "Total size in bytes of MemoryBasedMap => %d\n"
|
||||
+ "Number of entries in DiskBasedMap => %d\n"
|
||||
+ "Size of file spilled to disk => %d",
|
||||
((ExternalSpillableMap) keyToNewRecords).getInMemoryMapNumEntries(),
|
||||
((ExternalSpillableMap) keyToNewRecords).getCurrentInMemoryMapSize(),
|
||||
((ExternalSpillableMap) keyToNewRecords).getDiskBasedMapNumEntries(),
|
||||
((ExternalSpillableMap) keyToNewRecords).getSizeOfFileOnDiskInBytes()));
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* Rolls over the write handle to prepare for the next batch write.
|
||||
*
|
||||
* <p>It tweaks the handle state as following:
|
||||
*
|
||||
* <ul>
|
||||
* <li>Increment the {@code rollNumber}</li>
|
||||
* <li>Book-keep the last file path, these files (except the last one) are temporary that need to be cleaned</li>
|
||||
* <li>Make the last new file path as old</li>
|
||||
* <li>Initialize the new file path and file writer</li>
|
||||
* </ul>
|
||||
*
|
||||
* @param newRecordsItr The records iterator to update
|
||||
*/
|
||||
public void rollOver(Iterator<HoodieRecord<T>> newRecordsItr) {
|
||||
init(this.fileId, newRecordsItr);
|
||||
this.recordsWritten = 0;
|
||||
this.recordsDeleted = 0;
|
||||
this.updatedRecordsWritten = 0;
|
||||
this.insertRecordsWritten = 0;
|
||||
this.writeStatus.setTotalErrorRecords(0);
|
||||
this.timer = new HoodieTimer().startTimer();
|
||||
|
||||
rollNumber++;
|
||||
|
||||
rolloverPaths.add(newFilePath);
|
||||
oldFilePath = newFilePath;
|
||||
// Use the fileId + "-" + rollNumber as the new fileId of a mini-batch write.
|
||||
String newFileName = generatesDataFileName();
|
||||
String relativePath = new Path((partitionPath.isEmpty() ? "" : partitionPath + "/")
|
||||
+ newFileName).toString();
|
||||
newFilePath = new Path(config.getBasePath(), relativePath);
|
||||
|
||||
try {
|
||||
fileWriter = createNewFileWriter(instantTime, newFilePath, hoodieTable, config, writerSchemaWithMetafields, taskContextSupplier);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Error when creating file writer for path " + newFilePath, e);
|
||||
}
|
||||
|
||||
LOG.info(String.format("Merging new data into oldPath %s, as newPath %s", oldFilePath.toString(),
|
||||
newFilePath.toString()));
|
||||
}
|
||||
|
||||
public void finishWrite() {
|
||||
for (int i = 0; i < rolloverPaths.size() - 1; i++) {
|
||||
Path path = rolloverPaths.get(i);
|
||||
try {
|
||||
fs.delete(path, false);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Error when clean the temporary roll file: " + path, e);
|
||||
}
|
||||
}
|
||||
Path lastPath = rolloverPaths.size() > 0
|
||||
? rolloverPaths.get(rolloverPaths.size() - 1)
|
||||
: newFilePath;
|
||||
String newFileName = FSUtils.makeDataFileName(instantTime, writeToken, fileId, hoodieTable.getBaseFileExtension());
|
||||
String relativePath = new Path((partitionPath.isEmpty() ? "" : partitionPath + "/")
|
||||
+ newFileName).toString();
|
||||
final Path desiredPath = new Path(config.getBasePath(), relativePath);
|
||||
try {
|
||||
fs.rename(lastPath, desiredPath);
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Error when rename the temporary roll file: " + lastPath + " to: " + desiredPath, e);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,30 @@
|
||||
/*
|
||||
* 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;
|
||||
|
||||
/**
|
||||
* Hoodie write handle that supports write as mini-batch.
|
||||
*/
|
||||
public interface MiniBatchHandle {
|
||||
/**
|
||||
* Finish the write of multiple mini-batches. Usually these mini-bathes
|
||||
* come from a checkpoint interval.
|
||||
*/
|
||||
void finishWrite();
|
||||
}
|
||||
@@ -0,0 +1,128 @@
|
||||
/*
|
||||
* 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.client.WriteStatus;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* HoodieTable that need to pass in the
|
||||
* {@link org.apache.hudi.io.HoodieWriteHandle} explicitly.
|
||||
*/
|
||||
public interface ExplicitWriteHandleTable<T extends HoodieRecordPayload> {
|
||||
/**
|
||||
* Upsert a batch of new records into Hoodie table at the supplied instantTime.
|
||||
*
|
||||
* <p>Specifies the write handle explicitly in order to have fine grained control with
|
||||
* the underneath file.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param writeHandle The write handle
|
||||
* @param instantTime Instant Time for the action
|
||||
* @param records hoodieRecords to upsert
|
||||
* @return HoodieWriteMetadata
|
||||
*/
|
||||
HoodieWriteMetadata<List<WriteStatus>> upsert(
|
||||
HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
String instantTime,
|
||||
List<HoodieRecord<T>> records);
|
||||
|
||||
/**
|
||||
* Insert a batch of new records into Hoodie table at the supplied instantTime.
|
||||
*
|
||||
* <p>Specifies the write handle explicitly in order to have fine grained control with
|
||||
* the underneath file.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param writeHandle The write handle
|
||||
* @param instantTime Instant Time for the action
|
||||
* @param records hoodieRecords to upsert
|
||||
* @return HoodieWriteMetadata
|
||||
*/
|
||||
HoodieWriteMetadata<List<WriteStatus>> insert(
|
||||
HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
String instantTime,
|
||||
List<HoodieRecord<T>> records);
|
||||
|
||||
/**
|
||||
* Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be
|
||||
* de-duped and non existent keys will be removed before deleting.
|
||||
*
|
||||
* <p>Specifies the write handle explicitly in order to have fine grained control with
|
||||
* the underneath file.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param writeHandle The write handle
|
||||
* @param instantTime Instant Time for the action
|
||||
* @param keys {@link List} of {@link HoodieKey}s to be deleted
|
||||
* @return HoodieWriteMetadata
|
||||
*/
|
||||
HoodieWriteMetadata<List<WriteStatus>> delete(
|
||||
HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
String instantTime,
|
||||
List<HoodieKey> keys);
|
||||
|
||||
/**
|
||||
* Upserts the given prepared records into the Hoodie table, at the supplied instantTime.
|
||||
*
|
||||
* <p>This implementation requires that the input records are already tagged, and de-duped if needed.
|
||||
*
|
||||
* <p>Specifies the write handle explicitly in order to have fine grained control with
|
||||
* the underneath file.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param instantTime Instant Time for the action
|
||||
* @param preppedRecords hoodieRecords to upsert
|
||||
* @return HoodieWriteMetadata
|
||||
*/
|
||||
HoodieWriteMetadata<List<WriteStatus>> upsertPrepped(
|
||||
HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
String instantTime,
|
||||
List<HoodieRecord<T>> preppedRecords);
|
||||
|
||||
/**
|
||||
* Inserts the given prepared records into the Hoodie table, at the supplied instantTime.
|
||||
*
|
||||
* <p>This implementation requires that the input records are already tagged, and de-duped if needed.
|
||||
*
|
||||
* <p>Specifies the write handle explicitly in order to have fine grained control with
|
||||
* the underneath file.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param instantTime Instant Time for the action
|
||||
* @param preppedRecords hoodieRecords to upsert
|
||||
* @return HoodieWriteMetadata
|
||||
*/
|
||||
HoodieWriteMetadata<List<WriteStatus>> insertPrepped(
|
||||
HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
String instantTime,
|
||||
List<HoodieRecord<T>> preppedRecords);
|
||||
}
|
||||
@@ -34,6 +34,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
|
||||
import org.apache.hudi.table.action.clean.FlinkCleanActionExecutor;
|
||||
@@ -61,14 +62,117 @@ public class HoodieFlinkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
super(config, context, metaClient);
|
||||
}
|
||||
|
||||
/**
|
||||
* Upsert a batch of new records into Hoodie table at the supplied instantTime.
|
||||
*
|
||||
* <p>Specifies the write handle explicitly in order to have fine grained control with
|
||||
* the underneath file.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param writeHandle The write handle
|
||||
* @param instantTime Instant Time for the action
|
||||
* @param records hoodieRecords to upsert
|
||||
* @return HoodieWriteMetadata
|
||||
*/
|
||||
public HoodieWriteMetadata<List<WriteStatus>> upsert(
|
||||
HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
String instantTime,
|
||||
List<HoodieRecord<T>> records) {
|
||||
return new FlinkUpsertCommitActionExecutor<>(context, writeHandle, config, this, instantTime, records).execute();
|
||||
}
|
||||
|
||||
/**
|
||||
* Insert a batch of new records into Hoodie table at the supplied instantTime.
|
||||
*
|
||||
* <p>Specifies the write handle explicitly in order to have fine grained control with
|
||||
* the underneath file.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param writeHandle The write handle
|
||||
* @param instantTime Instant Time for the action
|
||||
* @param records hoodieRecords to upsert
|
||||
* @return HoodieWriteMetadata
|
||||
*/
|
||||
public HoodieWriteMetadata<List<WriteStatus>> insert(
|
||||
HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
String instantTime,
|
||||
List<HoodieRecord<T>> records) {
|
||||
return new FlinkInsertCommitActionExecutor<>(context, writeHandle, config, this, instantTime, records).execute();
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be
|
||||
* de-duped and non existent keys will be removed before deleting.
|
||||
*
|
||||
* <p>Specifies the write handle explicitly in order to have fine grained control with
|
||||
* the underneath file.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param writeHandle The write handle
|
||||
* @param instantTime Instant Time for the action
|
||||
* @param keys {@link List} of {@link HoodieKey}s to be deleted
|
||||
* @return HoodieWriteMetadata
|
||||
*/
|
||||
public HoodieWriteMetadata<List<WriteStatus>> delete(
|
||||
HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
String instantTime,
|
||||
List<HoodieKey> keys) {
|
||||
return new FlinkDeleteCommitActionExecutor<>(context, writeHandle, config, this, instantTime, keys).execute();
|
||||
}
|
||||
|
||||
/**
|
||||
* Upserts the given prepared records into the Hoodie table, at the supplied instantTime.
|
||||
*
|
||||
* <p>This implementation requires that the input records are already tagged, and de-duped if needed.
|
||||
*
|
||||
* <p>Specifies the write handle explicitly in order to have fine grained control with
|
||||
* the underneath file.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param instantTime Instant Time for the action
|
||||
* @param preppedRecords hoodieRecords to upsert
|
||||
* @return HoodieWriteMetadata
|
||||
*/
|
||||
public HoodieWriteMetadata<List<WriteStatus>> upsertPrepped(
|
||||
HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
String instantTime,
|
||||
List<HoodieRecord<T>> preppedRecords) {
|
||||
return new FlinkUpsertPreppedCommitActionExecutor<>(context, writeHandle, config, this, instantTime, preppedRecords).execute();
|
||||
}
|
||||
|
||||
/**
|
||||
* Inserts the given prepared records into the Hoodie table, at the supplied instantTime.
|
||||
*
|
||||
* <p>This implementation requires that the input records are already tagged, and de-duped if needed.
|
||||
*
|
||||
* <p>Specifies the write handle explicitly in order to have fine grained control with
|
||||
* the underneath file.
|
||||
*
|
||||
* @param context HoodieEngineContext
|
||||
* @param instantTime Instant Time for the action
|
||||
* @param preppedRecords hoodieRecords to upsert
|
||||
* @return HoodieWriteMetadata
|
||||
*/
|
||||
public HoodieWriteMetadata<List<WriteStatus>> insertPrepped(
|
||||
HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
String instantTime,
|
||||
List<HoodieRecord<T>> preppedRecords) {
|
||||
return new FlinkInsertPreppedCommitActionExecutor<>(context, writeHandle, config, this, instantTime, preppedRecords).execute();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> upsert(HoodieEngineContext context, String instantTime, List<HoodieRecord<T>> records) {
|
||||
return new FlinkUpsertCommitActionExecutor<>(context, config, this, instantTime, records).execute();
|
||||
throw new HoodieNotSupportedException("This method should not be invoked");
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> insert(HoodieEngineContext context, String instantTime, List<HoodieRecord<T>> records) {
|
||||
return new FlinkInsertCommitActionExecutor<>(context, config, this, instantTime, records).execute();
|
||||
throw new HoodieNotSupportedException("This method should not be invoked");
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -81,7 +185,7 @@ public class HoodieFlinkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> delete(HoodieEngineContext context, String instantTime, List<HoodieKey> keys) {
|
||||
return new FlinkDeleteCommitActionExecutor<>(context, config, this, instantTime, keys).execute();
|
||||
throw new HoodieNotSupportedException("This method should not be invoked");
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -91,12 +195,12 @@ public class HoodieFlinkCopyOnWriteTable<T extends HoodieRecordPayload> extends
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> upsertPrepped(HoodieEngineContext context, String instantTime, List<HoodieRecord<T>> preppedRecords) {
|
||||
return new FlinkUpsertPreppedCommitActionExecutor<>(context, config, this, instantTime, preppedRecords).execute();
|
||||
throw new HoodieNotSupportedException("This method should not be invoked");
|
||||
}
|
||||
|
||||
@Override
|
||||
public HoodieWriteMetadata<List<WriteStatus>> insertPrepped(HoodieEngineContext context, String instantTime, List<HoodieRecord<T>> preppedRecords) {
|
||||
return new FlinkInsertPreppedCommitActionExecutor<>(context, config, this, instantTime, preppedRecords).execute();
|
||||
throw new HoodieNotSupportedException("This method should not be invoked");
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -36,7 +36,8 @@ import org.apache.hudi.index.HoodieIndex;
|
||||
import java.util.List;
|
||||
|
||||
public abstract class HoodieFlinkTable<T extends HoodieRecordPayload>
|
||||
extends HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> {
|
||||
extends HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>>
|
||||
implements ExplicitWriteHandleTable<T> {
|
||||
protected HoodieFlinkTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
|
||||
super(config, context, metaClient);
|
||||
}
|
||||
|
||||
@@ -32,11 +32,14 @@ import org.apache.hudi.common.util.CommitUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieCommitException;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.exception.HoodieUpsertException;
|
||||
import org.apache.hudi.execution.FlinkLazyInsertIterable;
|
||||
import org.apache.hudi.io.FlinkCreateHandleFactory;
|
||||
import org.apache.hudi.io.ExplicitCreateHandleFactory;
|
||||
import org.apache.hudi.io.FlinkMergeHandle;
|
||||
import org.apache.hudi.io.HoodieCreateHandle;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.io.HoodieSortedMergeHandle;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
@@ -71,21 +74,26 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(BaseFlinkCommitActionExecutor.class);
|
||||
|
||||
private HoodieWriteHandle<?, ?, ?, ?> writeHandle;
|
||||
|
||||
public BaseFlinkCommitActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable table,
|
||||
String instantTime,
|
||||
WriteOperationType operationType) {
|
||||
super(context, config, table, instantTime, operationType, Option.empty());
|
||||
this(context, writeHandle, config, table, instantTime, operationType, Option.empty());
|
||||
}
|
||||
|
||||
public BaseFlinkCommitActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable table,
|
||||
String instantTime,
|
||||
WriteOperationType operationType,
|
||||
Option extraMetadata) {
|
||||
super(context, config, table, instantTime, operationType, extraMetadata);
|
||||
this.writeHandle = writeHandle;
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -182,6 +190,16 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
case INSERT:
|
||||
return handleInsert(fileIdHint, recordItr);
|
||||
case UPDATE:
|
||||
if (this.writeHandle instanceof HoodieCreateHandle) {
|
||||
// During one checkpoint interval, an insert record could also be updated,
|
||||
// for example, for an operation sequence of a record:
|
||||
// I, U, | U, U
|
||||
// - batch1 - | - batch2 -
|
||||
// the first batch(batch1) operation triggers an INSERT bucket,
|
||||
// the second batch batch2 tries to reuse the same bucket
|
||||
// and append instead of UPDATE.
|
||||
return handleInsert(fileIdHint, recordItr);
|
||||
}
|
||||
return handleUpdate(partitionPath, fileIdHint, recordItr);
|
||||
default:
|
||||
throw new HoodieUpsertException("Unknown bucketType " + bucketType + " for partition :" + partitionPath);
|
||||
@@ -203,7 +221,7 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
|
||||
}
|
||||
// these are updates
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(partitionPath, fileId, recordItr);
|
||||
HoodieMergeHandle upsertHandle = getUpdateHandle(recordItr);
|
||||
return handleUpdateInternal(upsertHandle, fileId);
|
||||
}
|
||||
|
||||
@@ -225,11 +243,16 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
return Collections.singletonList(upsertHandle.writeStatuses()).iterator();
|
||||
}
|
||||
|
||||
protected HoodieMergeHandle getUpdateHandle(String partitionPath, String fileId, Iterator<HoodieRecord<T>> recordItr) {
|
||||
protected FlinkMergeHandle getUpdateHandle(Iterator<HoodieRecord<T>> recordItr) {
|
||||
if (table.requireSortedRecords()) {
|
||||
return new HoodieSortedMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier);
|
||||
throw new HoodieNotSupportedException("Sort records are not supported in Flink streaming write");
|
||||
} else {
|
||||
return new HoodieMergeHandle<>(config, instantTime, table, recordItr, partitionPath, fileId, taskContextSupplier);
|
||||
FlinkMergeHandle writeHandle = (FlinkMergeHandle) this.writeHandle;
|
||||
// add the incremental records.
|
||||
if (!writeHandle.isNeedBootStrap()) {
|
||||
writeHandle.rollOver(recordItr);
|
||||
}
|
||||
return writeHandle;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -242,6 +265,6 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
return Collections.singletonList((List<WriteStatus>) Collections.EMPTY_LIST).iterator();
|
||||
}
|
||||
return new FlinkLazyInsertIterable<>(recordItr, true, config, instantTime, table, idPfx,
|
||||
taskContextSupplier, new FlinkCreateHandleFactory<>());
|
||||
taskContextSupplier, new ExplicitCreateHandleFactory<>(writeHandle));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieKey;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
@@ -33,9 +34,12 @@ public class FlinkDeleteCommitActionExecutor<T extends HoodieRecordPayload<T>> e
|
||||
private final List<HoodieKey> keys;
|
||||
|
||||
public FlinkDeleteCommitActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, List<HoodieKey> keys) {
|
||||
super(context, config, table, instantTime, WriteOperationType.DELETE);
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable table,
|
||||
String instantTime,
|
||||
List<HoodieKey> keys) {
|
||||
super(context, writeHandle, config, table, instantTime, WriteOperationType.DELETE);
|
||||
this.keys = keys;
|
||||
}
|
||||
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
@@ -34,11 +35,12 @@ public class FlinkInsertCommitActionExecutor<T extends HoodieRecordPayload<T>> e
|
||||
private List<HoodieRecord<T>> inputRecords;
|
||||
|
||||
public FlinkInsertCommitActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable table,
|
||||
String instantTime,
|
||||
List<HoodieRecord<T>> inputRecords) {
|
||||
super(context, config, table, instantTime, WriteOperationType.INSERT);
|
||||
super(context, writeHandle, config, table, instantTime, WriteOperationType.INSERT);
|
||||
this.inputRecords = inputRecords;
|
||||
}
|
||||
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
@@ -34,9 +35,10 @@ public class FlinkInsertPreppedCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
private final List<HoodieRecord<T>> preppedRecords;
|
||||
|
||||
public FlinkInsertPreppedCommitActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, List<HoodieRecord<T>> preppedRecords) {
|
||||
super(context, config, table, instantTime, WriteOperationType.INSERT_PREPPED);
|
||||
super(context, writeHandle, config, table, instantTime, WriteOperationType.INSERT_PREPPED);
|
||||
this.preppedRecords = preppedRecords;
|
||||
}
|
||||
|
||||
|
||||
@@ -27,6 +27,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.queue.BoundedInMemoryExecutor;
|
||||
import org.apache.hudi.common.util.queue.IteratorBasedQueueProducer;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.io.FlinkMergeHandle;
|
||||
import org.apache.hudi.io.HoodieMergeHandle;
|
||||
import org.apache.hudi.io.storage.HoodieFileReader;
|
||||
import org.apache.hudi.io.storage.HoodieFileReaderFactory;
|
||||
@@ -64,13 +65,15 @@ public class FlinkMergeHelper<T extends HoodieRecordPayload> extends AbstractMer
|
||||
HoodieMergeHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> upsertHandle) throws IOException {
|
||||
final boolean externalSchemaTransformation = table.getConfig().shouldUseExternalSchemaTransformation();
|
||||
Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
|
||||
HoodieMergeHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> mergeHandle = upsertHandle;
|
||||
FlinkMergeHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> mergeHandle =
|
||||
(FlinkMergeHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>>) upsertHandle;
|
||||
HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
|
||||
|
||||
final GenericDatumWriter<GenericRecord> gWriter;
|
||||
final GenericDatumReader<GenericRecord> gReader;
|
||||
Schema readSchema;
|
||||
if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) {
|
||||
if (mergeHandle.isNeedBootStrap()
|
||||
&& (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent())) {
|
||||
readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
|
||||
gWriter = new GenericDatumWriter<>(readSchema);
|
||||
gReader = new GenericDatumReader<>(readSchema, mergeHandle.getWriterSchemaWithMetafields());
|
||||
@@ -84,7 +87,7 @@ public class FlinkMergeHelper<T extends HoodieRecordPayload> extends AbstractMer
|
||||
HoodieFileReader<GenericRecord> reader = HoodieFileReaderFactory.<GenericRecord>getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());
|
||||
try {
|
||||
final Iterator<GenericRecord> readerIterator;
|
||||
if (baseFile.getBootstrapBaseFile().isPresent()) {
|
||||
if (mergeHandle.isNeedBootStrap() && baseFile.getBootstrapBaseFile().isPresent()) {
|
||||
readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
|
||||
} else {
|
||||
readerIterator = reader.getRecordIterator(readSchema);
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
@@ -34,11 +35,12 @@ public class FlinkUpsertCommitActionExecutor<T extends HoodieRecordPayload<T>> e
|
||||
private List<HoodieRecord<T>> inputRecords;
|
||||
|
||||
public FlinkUpsertCommitActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
HoodieWriteConfig config,
|
||||
HoodieTable table,
|
||||
String instantTime,
|
||||
List<HoodieRecord<T>> inputRecords) {
|
||||
super(context, config, table, instantTime, WriteOperationType.UPSERT);
|
||||
super(context, writeHandle, config, table, instantTime, WriteOperationType.UPSERT);
|
||||
this.inputRecords = inputRecords;
|
||||
}
|
||||
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieRecord;
|
||||
import org.apache.hudi.common.model.HoodieRecordPayload;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.io.HoodieWriteHandle;
|
||||
import org.apache.hudi.table.HoodieTable;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
@@ -34,9 +35,10 @@ public class FlinkUpsertPreppedCommitActionExecutor<T extends HoodieRecordPayloa
|
||||
private final List<HoodieRecord<T>> preppedRecords;
|
||||
|
||||
public FlinkUpsertPreppedCommitActionExecutor(HoodieEngineContext context,
|
||||
HoodieWriteHandle<?, ?, ?, ?> writeHandle,
|
||||
HoodieWriteConfig config, HoodieTable table,
|
||||
String instantTime, List<HoodieRecord<T>> preppedRecords) {
|
||||
super(context, config, table, instantTime, WriteOperationType.UPSERT_PREPPED);
|
||||
super(context, writeHandle, config, table, instantTime, WriteOperationType.UPSERT_PREPPED);
|
||||
this.preppedRecords = preppedRecords;
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user