diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/connector/LakeSoulPartitionReader.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/connector/LakeSoulPartitionReader.java index 97c4b3ad8..e886e34db 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/connector/LakeSoulPartitionReader.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/connector/LakeSoulPartitionReader.java @@ -47,10 +47,12 @@ public LakeSoulPartitionReader(Configuration conf, RowType schema, List this.primaryKeys = primaryKeys; this.schema = schema; this.capacity = conf.getInteger(LakeSoulOptions.LAKESOUL_NATIVE_IO_BATCH_SIZE); - this.conf = new Configuration(conf);; + this.conf = new Configuration(conf); + ; this.awaitTimeout = 10000; this.curPartitionId = -1; } + /** * Opens the reader with given partitions. * @@ -80,7 +82,7 @@ public RowData read(RowData reuse) throws IOException { } private Optional nextRecord() throws IOException { - if (curArrowReader == null || curRecordId >= currentVSR.getRowCount()) { + if (curArrowReader == null || curRecordId >= currentVSR.getRowCount()) { curArrowReader = nextBatch(); } if (curArrowReader == null) return Optional.empty(); @@ -109,7 +111,7 @@ private void recreateInnerReaderForSinglePartition(int partitionIndex) throws IO } nativeIOReader = new NativeIOReader(); LakeSoulPartition partition = partitions.get(partitionIndex); - for (Path path: partition.getPaths()) { + for (Path path : partition.getPaths()) { nativeIOReader.addFile(FlinkUtil.makeQualifiedPath(path).toString()); } @@ -121,7 +123,7 @@ private void recreateInnerReaderForSinglePartition(int partitionIndex) throws IO nativeIOReader.setPrimaryKeys(primaryKeys); } Schema arrowSchema = ArrowUtils.toArrowSchema(schema); - FlinkUtil.setFSConfigs(conf, nativeIOReader); + FlinkUtil.setIOConfigs(conf, nativeIOReader); nativeIOReader.setSchema(arrowSchema); nativeIOReader.setBatchSize(capacity); nativeIOReader.initializeReader(); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTableSinkStreamBuilder.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTableSinkStreamBuilder.java index 2be009399..878480de4 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTableSinkStreamBuilder.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTableSinkStreamBuilder.java @@ -77,6 +77,13 @@ public DataStreamSink buildLakeSoulDMLSink(DataStream buildArrowSink(Context context, DataStream stream) { + return buildArrowSink(context, stream, 1); + } + + public static DataStreamSink buildArrowSink(Context context, + DataStream stream, + int parallelism + ) { if (!context.conf.contains(AUTO_SCHEMA_CHANGE)) { context.conf.set(AUTO_SCHEMA_CHANGE, true); } @@ -92,7 +99,7 @@ public static DataStreamSink buildArrowSink(Context contex .withRollingPolicy(rollingPolicy) .withOutputFileConfig(fileNameConfig) .build(); - return stream.sinkTo(sink).name("LakeSoul MultiTable Arrow Sink"); + return stream.sinkTo(sink).name("LakeSoul MultiTable Arrow Sink").setParallelism(parallelism); } public DataStreamSink printStream(DataStream stream, String name) { diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTablesSink.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTablesSink.java index d1d847ef5..f561bc49b 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTablesSink.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/LakeSoulMultiTablesSink.java @@ -25,6 +25,7 @@ import org.apache.flink.util.FlinkRuntimeException; import java.io.IOException; +import java.text.SimpleDateFormat; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -87,6 +88,7 @@ public Optional> createCommitter() @Override public List commit(List committables) throws IOException, InterruptedException { + System.out.println(new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(System.currentTimeMillis()) + " org.apache.flink.api.connector.sink.Committer.commit: " + committables); return Collections.emptyList(); } diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkCommitter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkCommitter.java index c5dd92ecb..2c60e96a2 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkCommitter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkCommitter.java @@ -129,7 +129,7 @@ public List commit(List String.format("%s,%s,%d,%s", op.getPath(), op.getFileOp(), op.getSize(), - op.getFileExistCols())).collect(Collectors.joining("\n\t")); + "op.getFileExistCols()")).collect(Collectors.joining("\n\t")); LOG.info("Commit to LakeSoul: Table={}, TableId={}, Partition={}, Files:\n\t{}, " + "CommitOp={}, Timestamp={}, UUID={}", identity.tableId.identifier(), tableNameId.getTableId(), partition, fileOpStr, dataCommitInfo.getCommitOp(), diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkGlobalCommitter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkGlobalCommitter.java index 7c0e64839..a8956b0f1 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkGlobalCommitter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/committer/LakeSoulSinkGlobalCommitter.java @@ -141,7 +141,6 @@ public List commit( StructType sparkSchema = ArrowUtils.fromArrowSchema(msgSchema); TableInfo tableInfo = dbManager.getTableInfoByNameAndNamespace(tableName, tableNamespace); - LOG.info("Committing: {}, {}, {}, {} {}", tableNamespace, tableName, isCdc, msgSchema, tableInfo); if (tableInfo == null) { if (!conf.getBoolean(AUTO_SCHEMA_CHANGE)) { throw new SuppressRestartsException( diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/state/LakeSoulWriterBucketState.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/state/LakeSoulWriterBucketState.java index 4419533ca..0c9d53e50 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/state/LakeSoulWriterBucketState.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/state/LakeSoulWriterBucketState.java @@ -33,6 +33,8 @@ public class LakeSoulWriterBucketState { private final Map> pendingFileRecoverableMap; + private final int restartTimes; + public LakeSoulWriterBucketState( TableSchemaIdentity identity, String bucketId, @@ -44,12 +46,22 @@ public LakeSoulWriterBucketState( this.bucketPath = bucketPath; this.pendingFileRecoverableMap = new HashMap<>(); this.pendingFileRecoverableMap.put(bucketId, pendingFileRecoverableList); + restartTimes = 0; } public LakeSoulWriterBucketState( TableSchemaIdentity identity, Path bucketPath, HashMap> pendingFileRecoverableMap + ) { + this(identity, bucketPath, pendingFileRecoverableMap, 0); + } + + public LakeSoulWriterBucketState( + TableSchemaIdentity identity, + Path bucketPath, + HashMap> pendingFileRecoverableMap, + int restartTimes ) { this.identity = identity; Optional>> first = pendingFileRecoverableMap.entrySet().stream().findFirst(); @@ -61,6 +73,7 @@ public LakeSoulWriterBucketState( this.bucketPath = bucketPath; this.pendingFileRecoverableMap = pendingFileRecoverableMap; + this.restartTimes = restartTimes; } public String getBucketId() { @@ -71,6 +84,10 @@ public Path getBucketPath() { return bucketPath; } + public int getRestartTimes() { + return restartTimes; + } + @Override public String toString() { diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/state/LakeSoulWriterBucketStateSerializer.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/state/LakeSoulWriterBucketStateSerializer.java index 8fcbc440a..3fd7304da 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/state/LakeSoulWriterBucketStateSerializer.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/state/LakeSoulWriterBucketStateSerializer.java @@ -67,6 +67,7 @@ public LakeSoulWriterBucketState deserialize(int version, byte[] serialized) thr private void serialize(LakeSoulWriterBucketState state, DataOutputView dataOutputView) throws IOException { // dataOutputView.writeUTF(state.getBucketId()); + dataOutputView.writeInt(state.getRestartTimes()); dataOutputView.writeUTF(state.getBucketPath().toString()); SimpleVersionedSerialization.writeVersionAndSerialize( @@ -102,6 +103,7 @@ private LakeSoulWriterBucketState internalDeserialize( throws IOException { // String bucketId = dataInputView.readUTF(); + int restartTimes = dataInputView.readInt(); String bucketPathStr = dataInputView.readUTF(); TableSchemaIdentity identity = SimpleVersionedSerialization.readVersionAndDeSerialize( @@ -123,7 +125,9 @@ private LakeSoulWriterBucketState internalDeserialize( return new LakeSoulWriterBucketState( identity, new Path(bucketPathStr), - pendingFileRecoverableMap); + pendingFileRecoverableMap, + restartTimes + ); } private void validateMagicNumber(DataInputView in) throws IOException { diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/AbstractLakeSoulMultiTableSinkWriter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/AbstractLakeSoulMultiTableSinkWriter.java index cebc93720..c98fb2aa4 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/AbstractLakeSoulMultiTableSinkWriter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/AbstractLakeSoulMultiTableSinkWriter.java @@ -12,8 +12,6 @@ import org.apache.flink.lakesoul.sink.LakeSoulMultiTablesSink; import org.apache.flink.lakesoul.sink.state.LakeSoulMultiTableSinkCommittable; import org.apache.flink.lakesoul.sink.state.LakeSoulWriterBucketState; -import org.apache.flink.lakesoul.sink.writer.arrow.LakeSoulArrowWriterBucket; -import org.apache.flink.lakesoul.tool.FlinkUtil; import org.apache.flink.lakesoul.tool.LakeSoulSinkOptions; import org.apache.flink.lakesoul.types.TableSchemaIdentity; import org.apache.flink.metrics.Counter; @@ -116,7 +114,7 @@ public void initializeState(List bucketStates) throws bucketFactory.restoreBucket( subTaskId, state.getIdentity(), - creator.createBucketWriter(), + creator.createBucketWriter(getSubTaskId()), rollingPolicy, state, outputFileConfig); @@ -222,12 +220,16 @@ protected LakeSoulWriterBucket getOrCreateBucketForBucketId( LakeSoulWriterBucket bucket = activeBuckets.get(Tuple2.of(identity, bucketId)); if (bucket == null) { final Path bucketPath = creator.tableLocation; - BucketWriter bucketWriter = creator.createBucketWriter(); + BucketWriter bucketWriter = creator.createBucketWriter(getSubTaskId()); bucket = bucketFactory.getNewBucket( subTaskId, creator.identity, - bucketId, bucketPath, bucketWriter, rollingPolicy, outputFileConfig); + bucketId, + bucketPath, + bucketWriter, + rollingPolicy, + outputFileConfig); activeBuckets.put(Tuple2.of(identity, bucketId), bucket); LOG.info("Create new bucket {}, {}, {}", identity, bucketId, bucketPath); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/DynamicPartitionNativeParquetWriter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/DynamicPartitionNativeParquetWriter.java deleted file mode 100644 index 134a1ade2..000000000 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/DynamicPartitionNativeParquetWriter.java +++ /dev/null @@ -1,220 +0,0 @@ -// SPDX-FileCopyrightText: 2023 LakeSoul Contributors -// -// SPDX-License-Identifier: Apache-2.0 - -package org.apache.flink.lakesoul.sink.writer; - -import com.dmetasoul.lakesoul.lakesoul.io.NativeIOWriter; -import org.apache.arrow.vector.VectorSchemaRoot; -import org.apache.arrow.vector.types.pojo.Schema; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.Path; -import org.apache.flink.lakesoul.tool.FlinkUtil; -import org.apache.flink.lakesoul.tool.LakeSoulSinkOptions; -import org.apache.flink.streaming.api.functions.sink.filesystem.InProgressFileWriter; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.arrow.ArrowUtils; -import org.apache.flink.table.runtime.arrow.ArrowWriter; -import org.apache.flink.table.types.logical.RowType; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; - -import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.DYNAMIC_BUCKET; -import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.MAX_ROW_GROUP_SIZE; -import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.SORT_FIELD; - -public class DynamicPartitionNativeParquetWriter implements InProgressFileWriter { - - private static final Logger LOG = LoggerFactory.getLogger(DynamicPartitionNativeParquetWriter.class); - - private final RowType rowType; - - private ArrowWriter arrowWriter; - private final List primaryKeys; - private final List rangeColumns; - private final Configuration conf; - - private NativeIOWriter nativeWriter; - - private final int maxRowGroupRows; - - private final long creationTime; - - private VectorSchemaRoot batch; - - private int rowsInBatch; - - long lastUpdateTime; - - String prefix; - - private long totalRows = 0; - - public DynamicPartitionNativeParquetWriter(RowType rowType, - List primaryKeys, - List rangeColumns, - Path path, - long creationTime, - Configuration conf) throws IOException { - this.maxRowGroupRows = conf.getInteger(MAX_ROW_GROUP_SIZE); - this.creationTime = creationTime; - this.rowsInBatch = 0; - this.rowType = rowType; - this.primaryKeys = primaryKeys; - this.rangeColumns = rangeColumns; - this.prefix = path.makeQualified(path.getFileSystem()).toString(); - this.conf = conf; - initNativeWriter(); - } - - private void initNativeWriter() throws IOException { - ArrowUtils.setLocalTimeZone(FlinkUtil.getLocalTimeZone(conf)); - Schema arrowSchema = ArrowUtils.toArrowSchema(rowType); - nativeWriter = new NativeIOWriter(arrowSchema); - nativeWriter.setPrimaryKeys(primaryKeys); - nativeWriter.setRangePartitions(rangeColumns); - if (conf.getBoolean(LakeSoulSinkOptions.isMultiTableSource)) { - nativeWriter.setAuxSortColumns(Collections.singletonList(SORT_FIELD)); - } - nativeWriter.setHashBucketNum(conf.getInteger(LakeSoulSinkOptions.HASH_BUCKET_NUM)); - - nativeWriter.setRowGroupRowNumber(this.maxRowGroupRows); - batch = VectorSchemaRoot.create(arrowSchema, nativeWriter.getAllocator()); - arrowWriter = ArrowUtils.createRowDataArrowWriter(batch, rowType); - - - nativeWriter.withPrefix(this.prefix); - nativeWriter.useDynamicPartition(true); - - FlinkUtil.setFSConfigs(conf, nativeWriter); - nativeWriter.initializeWriter(); - LOG.info("Initialized DynamicPartitionNativeParquetWriter: {}", this); - } - - @Override - public void write(RowData element, long currentTime) throws IOException { - this.lastUpdateTime = currentTime; - this.arrowWriter.write(element); - this.rowsInBatch++; - this.totalRows++; - if (this.rowsInBatch >= this.maxRowGroupRows) { - this.arrowWriter.finish(); - this.nativeWriter.write(this.batch); - // in native writer, batch may be kept in memory for sorting, - // so we have to release ownership in java - this.batch.clear(); - this.arrowWriter.reset(); - this.rowsInBatch = 0; - } - } - - @Override - public InProgressFileRecoverable persist() throws IOException { - // we currently do not support persist - return null; - } - - - @Override - public PendingFileRecoverable closeForCommit() throws IOException { - this.arrowWriter.finish(); - this.nativeWriter.write(this.batch); - HashMap> partitionDescAndFilesMap = this.nativeWriter.flush(); - this.arrowWriter.reset(); - this.rowsInBatch = 0; - this.batch.clear(); - this.batch.close(); - try { - this.nativeWriter.close(); - initNativeWriter(); - } catch (Exception e) { - throw new RuntimeException(e); - } - return new NativeParquetWriter.NativeWriterPendingFileRecoverable(this.prefix, this.creationTime); - } - - public Map> closeForCommitWithRecoverableMap() throws IOException { - this.arrowWriter.finish(); - Map> recoverableMap = new HashMap<>(); - if (this.batch.getRowCount() > 0) { - this.nativeWriter.write(this.batch); - HashMap> partitionDescAndFilesMap = this.nativeWriter.flush(); - for (Map.Entry> entry : partitionDescAndFilesMap.entrySet()) { - recoverableMap.put( - entry.getKey(), - entry.getValue() - .stream() - .map(path -> new NativeParquetWriter.NativeWriterPendingFileRecoverable(path, - creationTime)) - .collect(Collectors.toList()) - ); - } - this.arrowWriter.reset(); - this.rowsInBatch = 0; - this.batch.clear(); - this.batch.close(); - try { - this.nativeWriter.close(); - initNativeWriter(); - } catch (Exception e) { - throw new RuntimeException(e); - } - } - LOG.info("CloseForCommitWithRecoverableMap done, recoverableMap={}", recoverableMap); - return recoverableMap; - } - - @Override - public void dispose() { - try { - this.arrowWriter.finish(); - this.batch.close(); - this.nativeWriter.close(); - this.nativeWriter = null; - - } catch (Exception e) { - throw new RuntimeException(e); - } - } - - @Override - public String getBucketId() { - return DYNAMIC_BUCKET; - } - - @Override - public long getCreationTime() { - return this.creationTime; - } - - @Override - public long getSize() throws IOException { - return totalRows; - } - - @Override - public long getLastUpdateTime() { - return this.lastUpdateTime; - } - - @Override public String toString() { - return "DynamicPartitionNativeParquetWriter{" + - "rowType=" + rowType + - ", primaryKeys=" + primaryKeys + - ", rangeColumns=" + rangeColumns + - ", maxRowGroupRows=" + maxRowGroupRows + - ", creationTime=" + creationTime + - ", rowsInBatch=" + rowsInBatch + - ", lastUpdateTime=" + lastUpdateTime + - ", prefix='" + prefix + '\'' + - ", totalRows=" + totalRows + - '}'; - } -} diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/LakeSoulWriterBucket.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/LakeSoulWriterBucket.java index c548ec51c..778a2335a 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/LakeSoulWriterBucket.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/LakeSoulWriterBucket.java @@ -63,6 +63,8 @@ public class LakeSoulWriterBucket { private long tsMs; + private int restartTimes; + private final Map> pendingFilesMap = new HashMap<>(); @@ -96,6 +98,7 @@ private LakeSoulWriterBucket( this.uniqueId = UUID.randomUUID().toString(); this.partCounter = 0; + this.restartTimes = 0; } /** @@ -127,6 +130,7 @@ private void restoreState(LakeSoulWriterBucketState state) throws IOException { .entrySet()) { pendingFilesMap.computeIfAbsent(entry.getKey(), key -> new ArrayList<>()).addAll(entry.getValue()); } + restartTimes = state.getRestartTimes(); } public String getBucketId() { @@ -257,44 +261,19 @@ private Path assembleBucketPath(Path basePath, String bucketId) { * Constructor a new PartPath and increment the partCounter. */ private Path assembleNewPartPath() { - if (DYNAMIC_BUCKET.equals(bucketId)) { - return bucketPath; - } - long currentPartCounter = partCounter++; - String count = String.format("%03d", currentPartCounter); - String subTask = String.format("%05d", this.subTaskId); - return new Path( - assembleBucketPath(bucketPath, bucketId), - outputFileConfig.getPartPrefix() - + '-' - + subTask - + '-' - + uniqueId - + '_' - + subTask - + ".c" - + count - + outputFileConfig.getPartSuffix()); + return bucketPath; } private void closePartFile() throws IOException { if (inProgressPartWriter != null) { long start = System.currentTimeMillis(); - if (inProgressPartWriter instanceof DynamicPartitionNativeParquetWriter) { - Map> pendingFileRecoverableMap = - ((DynamicPartitionNativeParquetWriter) inProgressPartWriter).closeForCommitWithRecoverableMap(); - for (Map.Entry> entry : pendingFileRecoverableMap.entrySet()) { - pendingFilesMap.computeIfAbsent(entry.getKey(), bucketId -> new ArrayList()) - .addAll(entry.getValue()); - } - } else { - InProgressFileWriter.PendingFileRecoverable pendingFileRecoverable = - inProgressPartWriter.closeForCommit(); - pendingFilesMap.computeIfAbsent(bucketId, bucketId -> new ArrayList()).add(pendingFileRecoverable); - inProgressPartWriter = null; - LOG.info("Closed part file {} for {}ms", pendingFileRecoverable.getPath(), - (System.currentTimeMillis() - start)); + Map> pendingFileRecoverableMap = + ((NativeParquetWriter) inProgressPartWriter).closeForCommitWithRecoverableMap(); + for (Map.Entry> entry : pendingFileRecoverableMap.entrySet()) { + pendingFilesMap.computeIfAbsent(entry.getKey(), bucketId -> new ArrayList()) + .addAll(entry.getValue()); } + inProgressPartWriter = null; } } diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/NativeBucketWriter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/NativeBucketWriter.java index d3d8648b5..ae1acd461 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/NativeBucketWriter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/NativeBucketWriter.java @@ -7,8 +7,6 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.streaming.api.functions.sink.filesystem.BucketWriter; import org.apache.flink.streaming.api.functions.sink.filesystem.InProgressFileWriter; import org.apache.flink.streaming.api.functions.sink.filesystem.WriterProperties; @@ -18,8 +16,6 @@ import java.io.IOException; import java.util.List; -import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.DYNAMIC_BUCKET; - public class NativeBucketWriter implements BucketWriter { private final RowType rowType; @@ -28,20 +24,19 @@ public class NativeBucketWriter implements BucketWriter { private final Configuration conf; private final List partitionKeys; + private final int subTaskId; - public NativeBucketWriter(RowType rowType, List primaryKeys, List partitionKeys, Configuration conf) { + public NativeBucketWriter(RowType rowType, List primaryKeys, List partitionKeys, Configuration conf, int subTaskId) { this.rowType = rowType; this.primaryKeys = primaryKeys; this.partitionKeys = partitionKeys; this.conf = conf; + this.subTaskId = subTaskId; } @Override public InProgressFileWriter openNewInProgressFile(String bucketId, Path path, long creationTime) throws IOException { - if (DYNAMIC_BUCKET.equals(bucketId)) { - return new DynamicPartitionNativeParquetWriter(rowType, primaryKeys, partitionKeys, path, creationTime, conf); - } - return new NativeParquetWriter(rowType, primaryKeys, bucketId, path, creationTime, conf); + return new NativeParquetWriter(rowType, primaryKeys, partitionKeys, bucketId, path, creationTime, conf, subTaskId); } @Override diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/NativeParquetWriter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/NativeParquetWriter.java index 54156f605..9f5728577 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/NativeParquetWriter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/NativeParquetWriter.java @@ -14,6 +14,7 @@ import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.lakesoul.tool.FlinkUtil; import org.apache.flink.lakesoul.tool.LakeSoulSinkOptions; +import org.apache.flink.lakesoul.tool.NativeOptions; import org.apache.flink.streaming.api.functions.sink.filesystem.InProgressFileWriter; import org.apache.flink.table.data.RowData; import org.apache.flink.table.runtime.arrow.ArrowUtils; @@ -25,26 +26,27 @@ import javax.annotation.Nullable; import java.io.IOException; import java.io.Serializable; -import java.util.Collections; -import java.util.List; -import java.util.Objects; +import java.util.*; +import java.util.stream.Collectors; -import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.MAX_ROW_GROUP_SIZE; -import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.SORT_FIELD; +import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.*; public class NativeParquetWriter implements InProgressFileWriter { private static final Logger LOG = LoggerFactory.getLogger(NativeParquetWriter.class); - private final ArrowWriter arrowWriter; - + private final RowType rowType; + private final int subTaskId; + private ArrowWriter arrowWriter; + private final List primaryKeys; + private final List rangeColumns; private NativeIOWriter nativeWriter; - + private final Configuration conf; private final int maxRowGroupRows; private final long creationTime; - private final VectorSchemaRoot batch; + private VectorSchemaRoot batch; private final String bucketID; @@ -52,36 +54,64 @@ public class NativeParquetWriter implements InProgressFileWriter primaryKeys, + List rangeColumns, String bucketID, Path path, long creationTime, - Configuration conf) throws IOException { + Configuration conf, + int subTaskId) throws IOException { this.maxRowGroupRows = conf.getInteger(MAX_ROW_GROUP_SIZE); this.creationTime = creationTime; this.bucketID = bucketID; + this.isDynamicBucket = DYNAMIC_BUCKET.equals(bucketID); this.rowsInBatch = 0; + this.rowType = rowType; + this.primaryKeys = primaryKeys; + this.rangeColumns = rangeColumns; + this.conf = conf; + this.subTaskId = subTaskId; + + this.prefix = path.makeQualified(path.getFileSystem()); + if (!bucketID.isEmpty() && !isDynamicBucket) { + this.prefix = new Path(this.prefix, bucketID); + } + initNativeWriter(); + + } + + private void initNativeWriter() throws IOException { ArrowUtils.setLocalTimeZone(FlinkUtil.getLocalTimeZone(conf)); Schema arrowSchema = ArrowUtils.toArrowSchema(rowType); nativeWriter = new NativeIOWriter(arrowSchema); nativeWriter.setPrimaryKeys(primaryKeys); + if (conf.getBoolean(LakeSoulSinkOptions.isMultiTableSource)) { nativeWriter.setAuxSortColumns(Collections.singletonList(SORT_FIELD)); } + nativeWriter.setHashBucketNum(conf.getInteger(LakeSoulSinkOptions.HASH_BUCKET_NUM)); + nativeWriter.setRowGroupRowNumber(this.maxRowGroupRows); batch = VectorSchemaRoot.create(arrowSchema, nativeWriter.getAllocator()); arrowWriter = ArrowUtils.createRowDataArrowWriter(batch, rowType); - this.path = path.makeQualified(path.getFileSystem()); - nativeWriter.addFile(this.path.toUri().toString()); - FlinkUtil.setFSConfigs(conf, this.nativeWriter); - this.nativeWriter.initializeWriter(); + nativeWriter.withPrefix(this.prefix.toString()); + nativeWriter.setOption(NativeOptions.HASH_BUCKET_ID.key(), String.valueOf(subTaskId)); + if (isDynamicBucket) { + nativeWriter.setRangePartitions(rangeColumns); + nativeWriter.useDynamicPartition(true); + } + + FlinkUtil.setIOConfigs(conf, nativeWriter); + nativeWriter.initializeWriter(); + LOG.info("Initialized NativeParquetWriter: {}", this); } @Override @@ -190,7 +220,40 @@ public PendingFileRecoverable closeForCommit() throws IOException { } catch (Exception e) { throw new RuntimeException(e); } - return new NativeWriterPendingFileRecoverable(this.path.toString(), this.creationTime); + return new NativeWriterPendingFileRecoverable(this.prefix.toString(), this.creationTime); + } + + public Map> closeForCommitWithRecoverableMap() throws IOException { + long timer = System.currentTimeMillis(); + this.arrowWriter.finish(); + Map> recoverableMap = new HashMap<>(); + if (this.batch.getRowCount() > 0) { + this.nativeWriter.write(this.batch); + } + HashMap> partitionDescAndFilesMap = this.nativeWriter.flush(); + for (Map.Entry> entry : partitionDescAndFilesMap.entrySet()) { + String key = isDynamicBucket ? entry.getKey() : bucketID; + recoverableMap.put( + key, + entry.getValue() + .stream() + .map(path -> new NativeParquetWriter.NativeWriterPendingFileRecoverable(path, + creationTime)) + .collect(Collectors.toList()) + ); + } + this.arrowWriter.reset(); + this.rowsInBatch = 0; + this.batch.clear(); + this.batch.close(); + try { + this.nativeWriter.close(); + initNativeWriter(); + } catch (Exception e) { + throw new RuntimeException(e); + } + LOG.info("CloseForCommitWithRecoverableMap done, costTime={}ms, recoverableMap={}", System.currentTimeMillis() - timer, recoverableMap); + return recoverableMap; } @Override @@ -225,14 +288,15 @@ public long getLastUpdateTime() { return this.lastUpdateTime; } - @Override public String toString() { + @Override + public String toString() { return "NativeParquetWriter{" + "maxRowGroupRows=" + maxRowGroupRows + ", creationTime=" + creationTime + ", bucketID='" + bucketID + '\'' + ", rowsInBatch=" + rowsInBatch + ", lastUpdateTime=" + lastUpdateTime + - ", path=" + path + + ", path=" + prefix + ", totalRows=" + totalRows + '}'; } diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/TableSchemaWriterCreator.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/TableSchemaWriterCreator.java index 095d38efe..6374dbdae 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/TableSchemaWriterCreator.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/TableSchemaWriterCreator.java @@ -5,14 +5,12 @@ package org.apache.flink.lakesoul.sink.writer; import com.dmetasoul.lakesoul.lakesoul.io.NativeIOBase; -import org.apache.arrow.vector.VectorSchemaRoot; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.Path; import org.apache.flink.lakesoul.sink.bucket.CdcPartitionComputer; import org.apache.flink.lakesoul.sink.bucket.FlinkBucketAssigner; import org.apache.flink.lakesoul.sink.writer.arrow.NativeArrowBucketWriter; import org.apache.flink.lakesoul.tool.FlinkUtil; -import org.apache.flink.lakesoul.tool.LakeSoulSinkOptions; import org.apache.flink.lakesoul.types.TableId; import org.apache.flink.lakesoul.types.TableSchemaIdentity; import org.apache.flink.lakesoul.types.arrow.LakeSoulArrowWrapper; @@ -27,7 +25,6 @@ import java.io.IOException; import java.io.Serializable; import java.util.List; -import java.util.Vector; import static com.dmetasoul.lakesoul.meta.DBConfig.LAKESOUL_NULL_STRING; import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.*; @@ -85,10 +82,10 @@ public static TableSchemaWriterCreator create( return creator; } - public BucketWriter createBucketWriter() throws IOException { + public BucketWriter createBucketWriter(int subTaskId) throws IOException { if (NativeIOBase.isNativeIOLibExist()) { LOG.info("Create natvie bucket writer"); - return new NativeBucketWriter(this.identity.rowType, this.primaryKeys, this.partitionKeyList, this.conf); + return new NativeBucketWriter(this.identity.rowType, this.primaryKeys, this.partitionKeyList, this.conf, subTaskId); } else { String msg = "Cannot load lakesoul native writer"; LOG.error(msg); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/LakeSoulArrowMultiTableSinkWriter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/LakeSoulArrowMultiTableSinkWriter.java index da14f1eff..5f2e0ba93 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/LakeSoulArrowMultiTableSinkWriter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/LakeSoulArrowMultiTableSinkWriter.java @@ -28,8 +28,7 @@ import java.io.IOException; import java.util.*; -import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.DYNAMIC_BUCKET; -import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.DYNAMIC_BUCKETING; +import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.*; import static org.apache.flink.util.Preconditions.checkNotNull; public class LakeSoulArrowMultiTableSinkWriter extends AbstractLakeSoulMultiTableSinkWriter { @@ -97,8 +96,7 @@ protected LakeSoulArrowWriterBucket getOrCreateBucketForTableId(TableSchemaIdent getRollingPolicy(), getOutputFileConfig()); activeArrowBuckets.put(identity, bucket); - LOG.info("Create new bucket {}, {}", - identity, bucketPath); + LOG.info("Create new bucket {}, {}", identity, bucketPath); } return bucket; } @@ -111,6 +109,7 @@ public void initializeState(List bucketStates) throws LOG.info("initializeState restoring state: {}", state); TableSchemaIdentity identity = state.getIdentity(); + conf.set(SINK_RESTART_TIMES, state.getRestartTimes() + 1); BucketWriter bucketWriter = new NativeArrowBucketWriter(identity.rowType, identity.primaryKeys, identity.partitionKeyList, conf); LakeSoulArrowWriterBucket restoredBucket = arrowBucketFactory.restoreBucket( @@ -145,6 +144,7 @@ public List snapshotState(long checkpointId) throws I LakeSoulWriterBucketState state = bucket.snapshotState(); states.add(state); } + LOG.info("snapshotState: checkpointId={}, states={}", checkpointId, states); return states; } @@ -159,6 +159,7 @@ public void close() { @Override public List prepareCommit(boolean flush) throws IOException { + long timer = System.currentTimeMillis(); List committables = new ArrayList<>(); String dmlType = this.conf.getString(LakeSoulSinkOptions.DML_TYPE); String sourcePartitionInfo = this.conf.getString(LakeSoulSinkOptions.SOURCE_PARTITION_INFO); @@ -175,6 +176,7 @@ public List prepareCommit(boolean flush) thro committables.addAll(entry.getValue().prepareCommit(flush, dmlType, sourcePartitionInfo)); } } + LOG.info("LakeSoulArrowMultiTableSinkWriter.prepareCommit done, costTime={}ms, subTaskId={}, flush={}, {}", String.format("%06d", System.currentTimeMillis() - timer), getSubTaskId(), flush, committables); return committables; } diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/LakeSoulArrowWriterBucket.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/LakeSoulArrowWriterBucket.java index 7051e8d18..5f90f195f 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/LakeSoulArrowWriterBucket.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/LakeSoulArrowWriterBucket.java @@ -55,6 +55,8 @@ public class LakeSoulArrowWriterBucket { private long partCounter; + private int restartTimes; + @Nullable private InProgressFileWriter inProgressPartWriter; @@ -80,6 +82,7 @@ public LakeSoulArrowWriterBucket( this.uniqueId = UUID.randomUUID().toString(); this.partCounter = 0; + this.restartTimes = 0; } /** @@ -109,6 +112,7 @@ private void restoreState(LakeSoulWriterBucketState state) throws IOException { .entrySet()) { pendingFilesMap.computeIfAbsent(entry.getKey(), key -> new ArrayList<>()).addAll(entry.getValue()); } + restartTimes = state.getRestartTimes(); } public String getBucketId() { @@ -156,28 +160,34 @@ List prepareCommit(boolean flush, String dmlT // we always close part file and do not keep in-progress file // since the native parquet writer doesn't support resume if (inProgressPartWriter != null) { - LOG.info( - "Closing in-progress part file for bucket id={} on checkpoint.", getBucketId()); closePartFile(); + LOG.info( + "Closing in-progress part file for flush={} bucket id={} subTaskId={} tableId={} pendingFilesMap={} on checkpoint.", + flush, + getBucketId(), + subTaskId, + tableId, + pendingFilesMap); + } List committables = new ArrayList<>(); - long time = pendingFilesMap.isEmpty() ? Long.MIN_VALUE : - ((NativeParquetWriter.NativeWriterPendingFileRecoverable) pendingFilesMap.values().stream().findFirst() - .get().get(0)).creationTime; - - committables.add(new LakeSoulMultiTableSinkCommittable( - tableId, - new HashMap<>(pendingFilesMap), - time, - UUID.randomUUID().toString(), - tsMs, - dmlType, - sourcePartitionInfo - )); - LOG.info("org.apache.flink.lakesoul.sink.writer.arrow.LakeSoulArrowWriterBucket.prepareCommit {}", - committables); - pendingFilesMap.clear(); + if (!pendingFilesMap.isEmpty()) { + long time = ((NativeParquetWriter.NativeWriterPendingFileRecoverable) pendingFilesMap.values().stream().findFirst() + .get().get(0)).creationTime; + + committables.add(new LakeSoulMultiTableSinkCommittable( + tableId, + new HashMap<>(pendingFilesMap), + time, + UUID.randomUUID().toString(), + tsMs, + dmlType, + sourcePartitionInfo + )); + LOG.info("prepareCommit {}", committables); + pendingFilesMap.clear(); + } return committables; } @@ -253,7 +263,7 @@ private Path assembleNewPartPath() { } private void closePartFile() throws IOException { - LOG.info("ClosePartFile {}", inProgressPartWriter); +// LOG.info("ClosePartFile {}", inProgressPartWriter); if (inProgressPartWriter != null) { if (inProgressPartWriter instanceof NativeLakeSoulArrowWrapperWriter) { Map> pendingFileRecoverableMap = @@ -262,6 +272,7 @@ private void closePartFile() throws IOException { pendingFilesMap.computeIfAbsent(entry.getKey(), bucketId -> new ArrayList()) .addAll(entry.getValue()); } + inProgressPartWriter = null; } else { throw new RuntimeException( "inProgressPartWriter only support instanceof NativeLakeSoulArrowWrapperWriter"); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/NativeLakeSoulArrowWrapperWriter.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/NativeLakeSoulArrowWrapperWriter.java index 103a175b1..34472045d 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/NativeLakeSoulArrowWrapperWriter.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/sink/writer/arrow/NativeLakeSoulArrowWrapperWriter.java @@ -19,15 +19,14 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.io.UnsupportedEncodingException; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.DYNAMIC_BUCKET; -import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.MAX_ROW_GROUP_SIZE; -import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.SORT_FIELD; +import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.*; public class NativeLakeSoulArrowWrapperWriter implements InProgressFileWriter { @@ -81,13 +80,17 @@ private void initNativeWriter() throws IOException { nativeWriter.setHashBucketNum(conf.getInteger(LakeSoulSinkOptions.HASH_BUCKET_NUM)); nativeWriter.setRowGroupRowNumber(this.maxRowGroupRows); + nativeWriter.setBatchSize(conf.get(BATCH_SIZE)); + int maxRowGroupValueNumber = conf.getInteger(MAX_ROW_GROUP_VALUE_NUMBER); + LOG.info("maxRowGroupValueNumber={}", maxRowGroupValueNumber); + if (maxRowGroupValueNumber != -1) nativeWriter.setRowGroupValueNumber(maxRowGroupValueNumber); nativeWriter.withPrefix(this.prefix); nativeWriter.useDynamicPartition(true); - FlinkUtil.setFSConfigs(conf, nativeWriter); + FlinkUtil.setIOConfigs(conf, nativeWriter); nativeWriter.initializeWriter(); - LOG.info("Initialized NativeLakeSoulArrowWrapperWriter: {}", this); +// LOG.info("Initialized NativeLakeSoulArrowWrapperWriter: {}", this); } @Override @@ -104,17 +107,11 @@ public InProgressFileRecoverable persist() throws IOException { @Override public PendingFileRecoverable closeForCommit() throws IOException { - HashMap> partitionDescAndFilesMap = this.nativeWriter.flush(); - try { - this.nativeWriter.close(); - initNativeWriter(); - } catch (Exception e) { - throw new RuntimeException(e); - } - return new NativeParquetWriter.NativeWriterPendingFileRecoverable(this.prefix, this.creationTime); + throw new UnsupportedEncodingException(); } public Map> closeForCommitWithRecoverableMap() throws IOException { + long timer = System.currentTimeMillis(); Map> recoverableMap = new HashMap<>(); HashMap> partitionDescAndFilesMap = this.nativeWriter.flush(); @@ -130,12 +127,11 @@ public Map> closeForCommitWithRecoverableMa try { this.nativeWriter.close(); - initNativeWriter(); } catch (Exception e) { throw new RuntimeException(e); } - LOG.info("CloseForCommitWithRecoverableMap done, recoverableMap={}", recoverableMap); + LOG.info("CloseForCommitWithRecoverableMap done, costTime={}ms, recoverableMap={}", String.format("%06d", System.currentTimeMillis() - timer), recoverableMap); return recoverableMap; } diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java index 50f7bda7d..9dfba2f71 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/LakeSoulOneSplitRecordsReader.java @@ -122,7 +122,7 @@ private void initializeReader() throws IOException { Schema arrowSchema = ArrowUtils.toArrowSchema(projectedRowTypeWithPk); reader.setSchema(arrowSchema); reader.setPrimaryKeys(pkColumns); - FlinkUtil.setFSConfigs(conf, reader); + FlinkUtil.setIOConfigs(conf, reader); } reader.setPartitionSchema(partitionSchema); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/arrow/LakeSoulArrowSplitRecordsReader.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/arrow/LakeSoulArrowSplitRecordsReader.java index 710af7618..9a9537256 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/arrow/LakeSoulArrowSplitRecordsReader.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/source/arrow/LakeSoulArrowSplitRecordsReader.java @@ -126,7 +126,7 @@ private void initializeReader() throws IOException { Schema arrowSchema = ArrowUtils.toArrowSchema(projectedRowTypeWithPk); reader.setSchema(arrowSchema); reader.setPrimaryKeys(pkColumns); - FlinkUtil.setFSConfigs(conf, reader); + FlinkUtil.setIOConfigs(conf, reader); } reader.setPartitionSchema(partitionSchema); diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/FlinkUtil.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/FlinkUtil.java index d9b033849..dddd8d727 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/FlinkUtil.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/FlinkUtil.java @@ -16,7 +16,7 @@ import com.dmetasoul.lakesoul.meta.dao.TableInfoDao; import com.dmetasoul.lakesoul.meta.entity.TableInfo; import org.apache.arrow.vector.types.pojo.Field; -import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.GlobalConfiguration; import org.apache.flink.core.fs.FileSystem; @@ -351,7 +351,7 @@ public static String getDatabaseName(String fullDatabaseName) { return splited[splited.length - 1]; } - public static void setFSConfigs(Configuration conf, NativeIOBase io) { + public static void setIOConfigs(Configuration conf, NativeIOBase io) { conf.addAll(GlobalConfiguration.loadConfiguration()); try { FlinkUtil.class.getClassLoader().loadClass("org.apache.hadoop.hdfs.HdfsConfiguration"); @@ -380,6 +380,16 @@ public static void setFSConfigs(Configuration conf, NativeIOBase io) { setFSConf(conf, "s3.endpoint.region", "fs.s3a.endpoint.region", io); setFSConf(conf, S3_PATH_STYLE_ACCESS.key(), "fs.s3a.path.style.access", io); setFSConf(conf, S3_BUCKET.key(), "fs.s3a.bucket", io); + + // try other native options + for (ConfigOption option : NativeOptions.OPTION_LIST) { + String value = conf.get(option); + if (value != null) { + int lastDot = option.key().lastIndexOf('.'); + String key = lastDot == -1 ? option.key() : option.key().substring(lastDot + 1); + io.setOption(key, value); + } + } } public static void setFSConf(Configuration conf, String confKey, String fsConfKey, NativeIOBase io) { diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/LakeSoulSinkOptions.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/LakeSoulSinkOptions.java index 84f4091a8..e9147e129 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/LakeSoulSinkOptions.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/LakeSoulSinkOptions.java @@ -50,7 +50,6 @@ public class LakeSoulSinkOptions { public static final String VIEW_ORIGINAL_QUERY = "original_query"; public static final String VIEW_EXPANDED_QUERY = "expand_query"; - public static final ConfigOption CATALOG_PATH = ConfigOptions .key("path") .stringType() @@ -178,6 +177,12 @@ public class LakeSoulSinkOptions { .defaultValue(250000) .withDescription("Max row group size for LakeSoul writer"); + public static final ConfigOption MAX_ROW_GROUP_VALUE_NUMBER = ConfigOptions + .key("lakesoul.file.max_row_group_value_number") + .intType() + .defaultValue(-1) + .withDescription("Max row group value number for LakeSoul writer"); + public static final ConfigOption SOURCE_DB_SCHEMA_TABLES = ConfigOptions .key("source_db.schema_tables") .stringType() @@ -208,6 +213,12 @@ public class LakeSoulSinkOptions { .defaultValue(true) .withDescription("If true, lakesoul sink use dynamic bucketing writer"); + public static final ConfigOption SINK_RESTART_TIMES = ConfigOptions + .key("lakesoul.sink.restart_times") + .intType() + .defaultValue(0) + .withDescription("record restart time for adaptable handling sink configuration"); + public static final ConfigOption INFERRING_SCHEMA = ConfigOptions .key("lakesoul.sink.inferring_schema") .booleanType() diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/NativeOptions.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/NativeOptions.java new file mode 100644 index 000000000..b16d4958f --- /dev/null +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/tool/NativeOptions.java @@ -0,0 +1,30 @@ +package org.apache.flink.lakesoul.tool; + +import org.apache.flink.configuration.ConfigOption; + +import java.util.Arrays; +import java.util.List; + +import static org.apache.flink.configuration.ConfigOptions.key; + +public class NativeOptions { + public static final ConfigOption MEM_LIMIT = + key("lakesoul.native_writer.mem_limit") + .stringType() + .defaultValue(String.valueOf(50 * 1024 * 1024)) + .withDescription("Option to set memory limit of native writer"); + + public static final ConfigOption HASH_BUCKET_ID = + key("lakesoul.native_writer.hash_bucket_id") + .stringType() + .defaultValue("0") + .withDescription("Option to set hash bucket id of native writer"); + + public static final ConfigOption KEEP_ORDERS = + key("lakesoul.native_writer.keep_orders") + .stringType() + .noDefaultValue() + .withDescription("Option to set if keep order of records for native writer"); + + public static final List> OPTION_LIST = Arrays.asList(MEM_LIMIT, KEEP_ORDERS); +} diff --git a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/types/TableSchemaIdentity.java b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/types/TableSchemaIdentity.java index 2cdf22134..51f4a0dbc 100644 --- a/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/types/TableSchemaIdentity.java +++ b/lakesoul-flink/src/main/java/org/apache/flink/lakesoul/types/TableSchemaIdentity.java @@ -82,7 +82,7 @@ public int hashCode() { public String toString() { return "TableSchemaIdentity{" + "tableId=" + tableId + - ", rowType=" + rowType + +// ", rowType=" + rowType + ", tableLocation='" + tableLocation + '\'' + ", primaryKeys=" + primaryKeys + ", partitionKeyList=" + partitionKeyList + diff --git a/lakesoul-flink/src/test/java/com/dmetasoul/lakesoul/SinkMemoryLeakTest.java b/lakesoul-flink/src/test/java/com/dmetasoul/lakesoul/SinkMemoryLeakTest.java new file mode 100644 index 000000000..44914e2ac --- /dev/null +++ b/lakesoul-flink/src/test/java/com/dmetasoul/lakesoul/SinkMemoryLeakTest.java @@ -0,0 +1,68 @@ +package com.dmetasoul.lakesoul; + +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamStatementSet; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; + +public class SinkMemoryLeakTest { + public static void main(String[] args) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.getCheckpointConfig().setCheckpointInterval(60_000); + TableEnvironment tableEnvironment = StreamTableEnvironment.create(env); + tableEnvironment.executeSql("create catalog lakesoul with ('type'='lakesoul')").await(); + tableEnvironment.executeSql( + "CREATE TABLE source_table (\n" + + " c0 STRING,\n" + + " c1 STRING,\n" + + " c2 STRING,\n" + + " c3 STRING,\n" + + " c4 STRING,\n" + + " c5 STRING,\n" + + " c6 STRING,\n" + + " c7 STRING,\n" + + " c8 STRING,\n" + + " c9 STRING\n" + + ") WITH (\n" + + " 'connector' = 'datagen',\n" + + " 'rows-per-second' = '20000'\n" + + ")").await(); + tableEnvironment.executeSql( + "CREATE TABLE IF NOT EXISTS `lakesoul`.`default`.`sink_table` (\n" + + " c0 STRING PRIMARY KEY NOT ENFORCED,\n" + + " c1 STRING,\n" + + " c2 STRING,\n" + + " c3 STRING,\n" + + " c4 STRING,\n" + + " c5 STRING,\n" + + " c6 STRING,\n" + + " c7 STRING,\n" + + " c8 STRING,\n" + + " c9 STRING\n" + + ") WITH (\n" + + " 'connector' = 'lakesoul',\n" + + " 'path' = 'file:///tmp/test_sink_table',\n" + + " 'hashBucketNum' = '1'\n" + + ")").await(); + tableEnvironment.executeSql( + "CREATE TABLE sink_table2 (\n" + + " c0 STRING,\n" + + " c1 STRING,\n" + + " c2 STRING,\n" + + " c3 STRING,\n" + + " c4 STRING,\n" + + " c5 STRING,\n" + + " c6 STRING,\n" + + " c7 STRING,\n" + + " c8 STRING,\n" + + " c9 STRING\n" + + ") WITH (\n" + + " 'connector' = 'blackhole'\n" + + ")").await(); + StreamStatementSet statementSet = (StreamStatementSet) tableEnvironment.createStatementSet(); + statementSet.addInsertSql("INSERT INTO `lakesoul`.`default`.`sink_table` SELECT * FROM source_table"); +// statementSet.addInsertSql("INSERT INTO sink_table2 SELECT * FROM source_table"); + statementSet.attachAsDataStream(); + env.execute(); + } +} \ No newline at end of file diff --git a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/sink/ArrowSinkDemo.java b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/sink/ArrowSinkDemo.java new file mode 100644 index 000000000..a5583ab2a --- /dev/null +++ b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/sink/ArrowSinkDemo.java @@ -0,0 +1,315 @@ + +package org.apache.flink.lakesoul.sink; + +import com.dmetasoul.lakesoul.meta.DBUtil; +import com.dmetasoul.lakesoul.meta.entity.TableInfo; +import org.apache.arrow.memory.BufferAllocator; +import org.apache.arrow.memory.RootAllocator; +import org.apache.arrow.vector.IntVector; +import org.apache.arrow.vector.VarCharVector; +import org.apache.arrow.vector.VectorSchemaRoot; +import org.apache.arrow.vector.types.pojo.ArrowType; +import org.apache.arrow.vector.types.pojo.Field; +import org.apache.arrow.vector.types.pojo.FieldType; +import org.apache.arrow.vector.types.pojo.Schema; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.MemorySize; +import org.apache.flink.configuration.TaskManagerOptions; +import org.apache.flink.lakesoul.metadata.LakeSoulCatalog; +import org.apache.flink.lakesoul.sink.LakeSoulMultiTableSinkStreamBuilder; +import org.apache.flink.lakesoul.tool.NativeOptions; +import org.apache.flink.lakesoul.types.arrow.LakeSoulArrowWrapper; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.datastream.DataStreamSink; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.sink.PrintSinkFunction; +import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.runtime.arrow.ArrowUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.CollectionUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.BATCH_SIZE; +import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.MAX_ROW_GROUP_VALUE_NUMBER; + +public class ArrowSinkDemo { + static long checkpointInterval = 5 * 1000; + static int tableNum = 8; + + + public static void main(String[] args) throws Exception { + +// read data + +// TableEnvironment tEnv = TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); +// tEnv.registerCatalog("lakesoul", new LakeSoulCatalog()); +// tEnv.useCatalog("lakesoul"); +// long total = 0; +// for (int i = 0; i < tableNum; i++) { +// List collect = CollectionUtil.iteratorToList(tEnv.executeSql("select count(*) as `rows` from `default`.`qar_table_" + i + "`").collect()); +// total += (long) collect.get(0).getField(0); +// } +// System.out.println(total); +// System.exit(0); + + new LakeSoulCatalog().cleanForTest(); + + Configuration conf = new Configuration(); + conf.set(ExecutionCheckpointingOptions.ENABLE_CHECKPOINTS_AFTER_TASKS_FINISH, true); + conf.set(TaskManagerOptions.TOTAL_PROCESS_MEMORY, MemorySize.parse("512m")); + conf.set(TaskManagerOptions.TASK_OFF_HEAP_MEMORY, MemorySize.parse("512m")); + conf.set(NativeOptions.MEM_LIMIT, String.valueOf(1024 * 1024 * 10)); +// conf.set(TaskManagerOptions.JVM_OVERHEAD_MAX, MemorySize.parse("20m")); +// conf.set(TaskManagerOptions.JVM_METASPACE, MemorySize.parse("512m")); +// conf.set(ExecutionCheckpointingOptions.TOLERABLE_FAILURE_NUMBER, 2); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironmentWithWebUI(conf); +// StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(conf); + + int cols = 2000; + int batchSize = 2000; + int batchPerSecond = 20 * tableNum; + int sourceParallelism = 8; + int sinkParallelism = 4; + + int batchPerTask = 100 * tableNum; + + // TableInfo object can be reused + List sinkTableInfo = new ArrayList<>(); + for (int i = 0; i < tableNum; i++) { + List fields = new ArrayList<>(); + for (int j = 0; j < cols; j++) { + fields.add(new Field("f_i32_" + i + "_" + j, FieldType.nullable(new ArrowType.Int(32, true)), null)); + } + fields.add(new Field("date", FieldType.nullable(ArrowType.Utf8.INSTANCE), null)); + fields.add(new Field("fltNum", FieldType.nullable(ArrowType.Utf8.INSTANCE), null)); + fields.add(new Field("tailNum", FieldType.nullable(ArrowType.Utf8.INSTANCE), null)); + Schema arrowSchema = new Schema(fields); + TableInfo tableInfo = TableInfo + .newBuilder() + .setTableId("NOT_USED") + .setTableNamespace("default") + .setTableName("qar_table_" + i) + .setTableSchema(arrowSchema.toJson()) + .setTablePath("file:///tmp/test_arrow_sink_" + i) + .setPartitions(DBUtil.formatTableInfoPartitionsField( + // no primary field + Collections.emptyList(), + // partition fields + Collections.emptyList() +// , Arrays.asList("date", "fltNum", "tailNum") + )) + .setProperties("{}") + .build(); + sinkTableInfo.add(tableInfo); + } + + DataStreamSource + source = + env.addSource(new ArrowDataGenSource(sinkTableInfo, cols, batchSize, batchPerSecond, batchPerTask)) + .setParallelism(sourceParallelism); + env.getCheckpointConfig().setCheckpointInterval(checkpointInterval); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(2, 1000L)); + LakeSoulMultiTableSinkStreamBuilder.Context context = new LakeSoulMultiTableSinkStreamBuilder.Context(); + context.env = env; + context.conf = (Configuration) env.getConfiguration(); + int rowGroupValues = 1 * (cols + 3) * batchSize; + System.out.println("MAX_ROW_GROUP_VALUE_NUMBER=" + rowGroupValues); +// context.conf.set(MAX_ROW_GROUP_VALUE_NUMBER, rowGroupValues); + context.conf.set(BATCH_SIZE, batchSize); + + LakeSoulMultiTableSinkStreamBuilder.buildArrowSink(context, source, sinkParallelism); +// String name = "Print Sink"; +// PrintSinkFunction printFunction = new PrintSinkFunction<>(name, false); +// +// DataStreamSink sink = source.addSink(printFunction).name(name).setParallelism(2); + + env.execute("Test Arrow Sink"); + + + } + + public static class ArrowDataGenSource extends RichParallelSourceFunction + implements CheckpointedFunction { + private static final Logger LOG = LoggerFactory.getLogger(ArrowDataGenSource.class); + + private transient ListState checkpointedCount; + + public int count; + + int cols; + int batchSize; + int batchPerSecond; + List arrowSchema; + List tableInfoEncoded; + private volatile transient boolean isRunning; + private transient int outputSoFar = 0; + + public ArrowDataGenSource(List sinkTableInfo, int cols, int batchSize, int batchPerSecond, int total) { + this.cols = cols; + this.batchSize = batchSize; + this.batchPerSecond = batchPerSecond; + arrowSchema = sinkTableInfo.stream().map(TableInfo::getTableSchema).collect(Collectors.toList()); + tableInfoEncoded = sinkTableInfo.stream().map(TableInfo::toByteArray).collect(Collectors.toList()); + count = total; + } + + @Override + public void run(SourceContext ctx) throws Exception { + int batchRate = batchPerSecond / getRuntimeContext().getNumberOfParallelSubtasks(); + batchRate = Math.max(1, batchRate); + LOG.info("Batch rate: {}", batchRate); + long nextReadTime = System.currentTimeMillis(); + List schema = new ArrayList<>(); + for (String s : arrowSchema) { + Schema fromJSON = Schema.fromJSON(s); + schema.add(fromJSON); + } + while (isRunning) { + for (int i = 0; i < batchRate; i++) { + if (isRunning) { + synchronized (ctx.getCheckpointLock()) { + LakeSoulArrowWrapper generateArrow = generateArrow(schema.get(outputSoFar % tableNum), outputSoFar); + outputSoFar++; + if (count > 0) { + count--; + } else { + isRunning = false; + break; + } + ctx.collectWithTimestamp(generateArrow, System.currentTimeMillis()); + } + } else { + break; + } + } + nextReadTime += 1000; + long toWaitMs = nextReadTime - System.currentTimeMillis(); + while (toWaitMs > 0) { + Thread.sleep(toWaitMs); + toWaitMs = nextReadTime - System.currentTimeMillis(); + } + + } + Thread.sleep(checkpointInterval / 10 * 9); + } + + @Override + public void cancel() { + isRunning = false; + } + + @Override + public void close() throws Exception { + super.close(); + LOG.info("Closing, generated {} batches", outputSoFar); + } + + private LakeSoulArrowWrapper generateArrow(Schema schema, int outputSoFar) { + int tableIdx = outputSoFar % tableNum; + try ( + BufferAllocator allocator = ArrowUtils.getRootAllocator(); + VectorSchemaRoot arrowBatch = VectorSchemaRoot.create(schema, allocator) + ) { + for (int i = 0; i < cols; i++) { + IntVector intVector = (IntVector) arrowBatch.getVector("f_i32_" + tableIdx + "_" + i); + intVector.allocateNew(batchSize); + for (int j = 0; j < batchSize; j++) { + intVector.set(j, i + j); + } + } + byte[] date; + byte[] fltNum; + byte[] tailNum; + switch (outputSoFar % 3) { +// case 0: +// date = "2024-07-01".getBytes(); +// fltNum = "1234".getBytes(); +// tailNum = "B4567".getBytes(); +// break; +// case 1: +// date = "2024-07-02".getBytes(); +// fltNum = "12 5".getBytes(); +// tailNum = "B4568".getBytes(); +// break; +// case 2: +// date = "2024-07-01".getBytes(); +// fltNum = "1236".getBytes(); +// tailNum = "B4569".getBytes(); +// break; + default: + date = "2024-07-01".getBytes(); + fltNum = "1236".getBytes(); + tailNum = "B4569".getBytes(); + } + VarCharVector dateVector = (VarCharVector) arrowBatch.getVector("date"); + dateVector.allocateNew(batchSize); + VarCharVector fltNumVector = (VarCharVector) arrowBatch.getVector("fltNum"); + fltNumVector.allocateNew(batchSize); + VarCharVector tailNumVector = (VarCharVector) arrowBatch.getVector("tailNum"); + tailNumVector.allocateNew(batchSize); + for (int j = 0; j < batchSize; j++) { + dateVector.set(j, date); + fltNumVector.set(j, fltNum); + tailNumVector.set(j, tailNum); + } + arrowBatch.setRowCount(batchSize); + return new LakeSoulArrowWrapper(tableInfoEncoded.get(tableIdx), arrowBatch); + } + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + System.out.println("============= Source snapshotState getCheckpointId=" + context.getCheckpointId() + " ================"); + System.out.println(new SimpleDateFormat("yyyy-MM-dd HH:mm:ss").format(System.currentTimeMillis()) + " snapshotState context.getCheckpointId=" + context.getCheckpointId() + ", count=" + count); + this.checkpointedCount.clear(); + try { + this.checkpointedCount.add(count); + } catch (Exception e) { + throw new RuntimeException(e); + } + LOG.info("Snapshot state, generated {} batches", outputSoFar); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + isRunning = true; + try { + this.checkpointedCount = context + .getOperatorStateStore() + .getListState(new ListStateDescriptor<>("count", Integer.class)); + + if (context.isRestored()) { + for (Integer count : this.checkpointedCount.get()) { + this.count = count; + } + } + System.out.println("initializeState count=" + count); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + + } +} + + \ No newline at end of file diff --git a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/connector/LakeSoulArrowConnectorCase.java b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/connector/LakeSoulArrowConnectorCase.java index 7961c4d17..04ebac55d 100644 --- a/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/connector/LakeSoulArrowConnectorCase.java +++ b/lakesoul-flink/src/test/java/org/apache/flink/lakesoul/test/connector/LakeSoulArrowConnectorCase.java @@ -46,7 +46,7 @@ import static org.apache.flink.lakesoul.tool.LakeSoulSinkOptions.MAX_ROW_GROUP_SIZE; public class LakeSoulArrowConnectorCase extends AbstractTestBase { - @Test + // @Test public void test() throws Exception { int parallelism = 2; StreamExecutionEnvironment execEnv = diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOBase.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOBase.java index bfa340d19..4d5d6e6cd 100644 --- a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOBase.java +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOBase.java @@ -57,14 +57,18 @@ public NativeIOBase(String allocatorName) { ioConfigBuilder = libLakeSoulIO.new_lakesoul_io_config_builder(); tokioRuntimeBuilder = libLakeSoulIO.new_tokio_runtime_builder(); - fixedBuffer = Runtime.getRuntime(libLakeSoulIO).getMemoryManager().allocateDirect(5000L); - mutableBuffer = Runtime.getRuntime(libLakeSoulIO).getMemoryManager().allocateDirect(1 << 12); + fixedBuffer = getRuntime().getMemoryManager().allocateDirect(5000L); + mutableBuffer = getRuntime().getMemoryManager().allocateDirect(1 << 12); setBatchSize(10240); setThreadNum(2); libLakeSoulIO.rust_logger_init(); } + protected Runtime getRuntime() { + return Runtime.getRuntime(libLakeSoulIO); + } + public ObjectReferenceManager getIntReferenceManager() { return intReferenceManager; } @@ -168,6 +172,13 @@ public void setObjectStoreOption(String key, String value) { } } + public void setOption(String key, String value) { + assert ioConfigBuilder != null; + if (key != null && value != null) { + ioConfigBuilder = libLakeSoulIO.lakesoul_config_builder_set_option(ioConfigBuilder, key, value); + } + } + @Override public void close() throws Exception { if (tokioRuntime != null) { diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java index c6624bff0..9f3f636b9 100644 --- a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/NativeIOWriter.java @@ -4,6 +4,7 @@ package com.dmetasoul.lakesoul.lakesoul.io; +import jnr.ffi.Memory; import jnr.ffi.Pointer; import jnr.ffi.Runtime; import org.apache.arrow.c.ArrowArray; @@ -16,6 +17,7 @@ import java.io.ByteArrayInputStream; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -47,6 +49,10 @@ public void setRowGroupRowNumber(int rowNum) { ioConfigBuilder = libLakeSoulIO.lakesoul_config_builder_set_max_row_group_size(ioConfigBuilder, rowNum); } + public void setRowGroupValueNumber(int valueNum) { + ioConfigBuilder = libLakeSoulIO.lakesoul_config_builder_set_max_row_group_num_values(ioConfigBuilder, valueNum); + } + public void initializeWriter() throws IOException { assert tokioRuntimeBuilder != null; assert ioConfigBuilder != null; @@ -64,6 +70,16 @@ public void initializeWriter() throws IOException { } public int writeIpc(byte[] encodedBatch) throws IOException { +// Pointer ipc = getRuntime().getMemoryManager().allocateDirect(encodedBatch.length + 1, true); +// ipc.put(0, encodedBatch, 0, encodedBatch.length); +// ipc.putByte(encodedBatch.length, (byte) 0); +// String msg = libLakeSoulIO.write_record_batch_ipc_blocked(writer, ipc.address(), ipc.size()); +// if (!msg.startsWith("Ok: ")) { +// throw new IOException("Native writer write batch failed with error: " + msg); +// } +// +// return Integer.parseInt(msg.substring(4)); + int batchSize = 0; try (ArrowStreamReader reader = new ArrowStreamReader(new ByteArrayInputStream(encodedBatch), allocator)) { if (reader.loadNextBatch()) { @@ -84,6 +100,7 @@ public int writeIpc(byte[] encodedBatch) throws IOException { } public void write(VectorSchemaRoot batch) throws IOException { + System.out.println("writing batch: " + batch.getRowCount()); ArrowArray array = ArrowArray.allocateNew(allocator); ArrowSchema schema = ArrowSchema.allocateNew(allocator); Data.exportVectorSchemaRoot(allocator, batch, provider, array, schema); diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/jnr/LibLakeSoulIO.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/jnr/LibLakeSoulIO.java index 170bce4e3..35950fa4d 100644 --- a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/jnr/LibLakeSoulIO.java +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/io/jnr/LibLakeSoulIO.java @@ -44,6 +44,8 @@ public interface LibLakeSoulIO { Pointer lakesoul_config_builder_set_object_store_option(Pointer builder, String key, String value); + Pointer lakesoul_config_builder_set_option(Pointer builder, String key, String value); + Pointer lakesoul_config_builder_set_thread_num(Pointer builder, int thread_num); Pointer lakesoul_config_builder_set_dynamic_partition(Pointer builder, boolean enable); @@ -56,6 +58,8 @@ public interface LibLakeSoulIO { Pointer lakesoul_config_builder_set_max_row_group_size(Pointer builder, int row_group_size); + Pointer lakesoul_config_builder_set_max_row_group_num_values(Pointer builder, int row_group_num_values); + Pointer create_lakesoul_io_config_from_builder(Pointer builder); Pointer create_lakesoul_reader_from_config(Pointer config, Pointer runtime); @@ -90,7 +94,7 @@ interface IntegerCallback { // type representing callback String write_record_batch_blocked(Pointer writer, @LongLong long schemaAddr, @LongLong long arrayAddr); - String write_record_batch_ipc_blocked(Pointer writer, @LongLong long schemaAddr, @LongLong long arrayAddr); + String write_record_batch_ipc_blocked(Pointer writer, @LongLong long ipcAddr, @LongLong long len); void free_lakesoul_reader(Pointer reader); diff --git a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/memory/ArrowMemoryUtils.java b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/memory/ArrowMemoryUtils.java index 1ee907950..99d4ed0af 100644 --- a/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/memory/ArrowMemoryUtils.java +++ b/native-io/lakesoul-io-java/src/main/java/com/dmetasoul/lakesoul/lakesoul/memory/ArrowMemoryUtils.java @@ -9,4 +9,8 @@ public class ArrowMemoryUtils { public final static BufferAllocator rootAllocator = new RootAllocator(); + + static { + Runtime.getRuntime().addShutdownHook(new Thread(rootAllocator::close)); + } } diff --git a/rust/Cargo.lock b/rust/Cargo.lock index 86eb13656..9a9def58b 100644 --- a/rust/Cargo.lock +++ b/rust/Cargo.lock @@ -1098,6 +1098,22 @@ dependencies = [ "syn 1.0.109", ] +[[package]] +name = "dhat" +version = "0.3.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "98cd11d84628e233de0ce467de10b8633f4ddaecafadefc86e13b84b8739b827" +dependencies = [ + "backtrace", + "lazy_static", + "mintex", + "parking_lot", + "rustc-hash", + "serde", + "serde_json", + "thousands", +] + [[package]] name = "digest" version = "0.10.7" @@ -1895,6 +1911,7 @@ dependencies = [ "arrow-buffer", "arrow-cast", "arrow-schema", + "async-recursion", "async-trait", "atomic_refcell", "bytes", @@ -1905,6 +1922,7 @@ dependencies = [ "datafusion-common", "datafusion-substrait", "derivative", + "dhat", "env_logger", "futures", "half", @@ -1927,6 +1945,7 @@ dependencies = [ "tokio-stream", "tokio-util", "tracing", + "tracing-subscriber", "url", "whoami", ] @@ -1936,6 +1955,8 @@ name = "lakesoul-io-c" version = "2.5.0" dependencies = [ "arrow", + "arrow-buffer", + "arrow-ipc", "bytes", "cbindgen", "datafusion-substrait", @@ -2181,6 +2202,12 @@ dependencies = [ "adler", ] +[[package]] +name = "mintex" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "9bec4598fddb13cc7b528819e697852653252b760f1228b7642679bf2ff2cd07" + [[package]] name = "mio" version = "0.8.11" @@ -2911,6 +2938,12 @@ version = "0.1.24" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "719b953e2095829ee67db738b3bfa9fa368c94900df327b3f07fe6e794d2fe1f" +[[package]] +name = "rustc-hash" +version = "1.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" + [[package]] name = "rustc_version" version = "0.4.0" @@ -3505,6 +3538,12 @@ dependencies = [ "syn 2.0.66", ] +[[package]] +name = "thousands" +version = "0.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "3bf63baf9f5039dadc247375c29eb13706706cfde997d0330d05aa63a77d8820" + [[package]] name = "thread_local" version = "1.1.8" diff --git a/rust/Cargo.toml b/rust/Cargo.toml index 96d6e6378..b6ccdd2e0 100644 --- a/rust/Cargo.toml +++ b/rust/Cargo.toml @@ -28,6 +28,7 @@ arrow-array = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "a arrow-buffer = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } arrow-cast = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } arrow-arith = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } +arrow-ipc = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } parquet = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "arrow-rs-48-parquet-bufferred" } object_store = { git = "https://github.com/lakesoul-io/arrow-rs.git", branch = "object_store_0.7_opt", features = ["aws", "http"] } diff --git a/rust/lakesoul-datafusion/src/datasource/file_format/metadata_format.rs b/rust/lakesoul-datafusion/src/datasource/file_format/metadata_format.rs index 2f11fb3ca..cd564761d 100644 --- a/rust/lakesoul-datafusion/src/datasource/file_format/metadata_format.rs +++ b/rust/lakesoul-datafusion/src/datasource/file_format/metadata_format.rs @@ -33,6 +33,7 @@ use datafusion::{ physical_plan::{ExecutionPlan, PhysicalExpr}, }; use futures::StreamExt; +use lakesoul_io::async_writer::{AsyncBatchWriter, MultiPartAsyncWriter}; use lakesoul_io::datasource::file_format::{compute_project_column_indices, flatten_file_scan_config}; use lakesoul_io::datasource::physical_plan::MergeParquetExec; use lakesoul_io::helpers::{ @@ -40,7 +41,6 @@ use lakesoul_io::helpers::{ partition_desc_from_file_scan_config, }; use lakesoul_io::lakesoul_io_config::LakeSoulIOConfig; -use lakesoul_io::lakesoul_writer::{AsyncBatchWriter, MultiPartAsyncWriter}; use lakesoul_metadata::MetaDataClientRef; use object_store::{ObjectMeta, ObjectStore}; use proto::proto::entity::TableInfo; diff --git a/rust/lakesoul-datafusion/src/test/upsert_tests.rs b/rust/lakesoul-datafusion/src/test/upsert_tests.rs index 5f91239bb..9f4ef72ab 100644 --- a/rust/lakesoul-datafusion/src/test/upsert_tests.rs +++ b/rust/lakesoul-datafusion/src/test/upsert_tests.rs @@ -86,7 +86,7 @@ mod upsert_with_io_config_tests { let builder = builder.with_file(file.clone()).with_schema(batch.schema()); let config = builder.clone().build(); - let writer = + let mut writer = SyncSendableMutableLakeSoulWriter::try_new(config, Builder::new_current_thread().build().unwrap()).unwrap(); let _ = writer.write_batch(batch); let _ = writer.flush_and_close(); diff --git a/rust/lakesoul-io-c/Cargo.toml b/rust/lakesoul-io-c/Cargo.toml index b013a903c..fd6b7fcd3 100644 --- a/rust/lakesoul-io-c/Cargo.toml +++ b/rust/lakesoul-io-c/Cargo.toml @@ -13,6 +13,8 @@ crate-type = ["cdylib"] [dependencies] lakesoul-io = { path = "../lakesoul-io" } arrow = { workspace = true, features = ["ffi"] } +arrow-buffer = { workspace = true } +arrow-ipc = { workspace = true } tokio = { version = "1", features = ["full"] } serde_json = "1.0" serde = { version = "1.0", default-features = false, features = ["derive", "std"], optional = true } diff --git a/rust/lakesoul-io-c/lakesoul_c_bindings.h b/rust/lakesoul-io-c/lakesoul_c_bindings.h index b97b6b43b..1490ed35e 100644 --- a/rust/lakesoul-io-c/lakesoul_c_bindings.h +++ b/rust/lakesoul-io-c/lakesoul_c_bindings.h @@ -103,6 +103,9 @@ IOConfigBuilder *lakesoul_config_builder_set_batch_size(IOConfigBuilder *builder IOConfigBuilder *lakesoul_config_builder_set_max_row_group_size(IOConfigBuilder *builder, c_size_t max_row_group_size); +IOConfigBuilder *lakesoul_config_builder_set_max_row_group_num_values(IOConfigBuilder *builder, + c_size_t max_row_group_num_values); + IOConfigBuilder *lakesoul_config_builder_set_buffer_size(IOConfigBuilder *builder, c_size_t buffer_size); @@ -113,6 +116,10 @@ IOConfigBuilder *lakesoul_config_builder_set_object_store_option(IOConfigBuilder const char *key, const char *value); +IOConfigBuilder *lakesoul_config_builder_set_option(IOConfigBuilder *builder, + const char *key, + const char *value); + IOConfigBuilder *lakesoul_config_builder_add_files(IOConfigBuilder *builder, const char *const *files, c_size_t file_num); @@ -175,6 +182,10 @@ const char *write_record_batch_blocked(CResult *writer, c_ptrdiff_t schema_addr, c_ptrdiff_t array_addr); +const char *write_record_batch_ipc_blocked(CResult *writer, + c_ptrdiff_t ipc_addr, + int64_t len); + void export_bytes_result(void (*callback)(bool, const char*), CResult *bytes, int32_t len, diff --git a/rust/lakesoul-io-c/src/lib.rs b/rust/lakesoul-io-c/src/lib.rs index f463212a1..91f8f906e 100644 --- a/rust/lakesoul-io-c/src/lib.rs +++ b/rust/lakesoul-io-c/src/lib.rs @@ -262,6 +262,17 @@ pub extern "C" fn lakesoul_config_builder_set_max_row_group_size( ) } + +#[no_mangle] +pub extern "C" fn lakesoul_config_builder_set_max_row_group_num_values( + builder: NonNull, + max_row_group_num_values: c_size_t, +) -> NonNull { + convert_to_opaque( + from_opaque::(builder).with_max_row_group_num_values(max_row_group_num_values), + ) +} + #[no_mangle] pub extern "C" fn lakesoul_config_builder_set_buffer_size( builder: NonNull, @@ -295,6 +306,21 @@ pub extern "C" fn lakesoul_config_builder_set_object_store_option( } } +#[no_mangle] +pub extern "C" fn lakesoul_config_builder_set_option( + builder: NonNull, + key: *const c_char, + value: *const c_char, +) -> NonNull { + unsafe { + let key = CStr::from_ptr(key).to_str().unwrap().to_string(); + let value = CStr::from_ptr(value).to_str().unwrap().to_string(); + convert_to_opaque( + from_opaque::(builder).with_option(key, value), + ) + } +} + #[no_mangle] pub extern "C" fn lakesoul_config_builder_add_files( builder: NonNull, @@ -698,7 +724,7 @@ pub extern "C" fn write_record_batch( callback: ResultCallback, ) { unsafe { - let writer = NonNull::new_unchecked(writer.as_ref().ptr as *mut SyncSendableMutableLakeSoulWriter); + let writer = NonNull::new_unchecked(writer.as_ref().ptr as *mut SyncSendableMutableLakeSoulWriter).as_mut(); let mut ffi_array = FFI_ArrowArray::empty(); (array_addr as *mut FFI_ArrowArray).copy_to(&mut ffi_array as *mut FFI_ArrowArray, 1); let mut ffi_schema = FFI_ArrowSchema::empty(); @@ -707,7 +733,7 @@ pub extern "C" fn write_record_batch( let array_data = from_ffi(ffi_array, &ffi_schema)?; let struct_array = StructArray::from(array_data); let rb = RecordBatch::from(struct_array); - writer.as_ref().write_batch(rb)?; + writer.write_batch(rb)?; Ok(()) }; let result: lakesoul_io::Result<()> = result_fn(); @@ -729,7 +755,7 @@ pub extern "C" fn write_record_batch_blocked( array_addr: c_ptrdiff_t, ) -> *const c_char { unsafe { - let writer = NonNull::new_unchecked(writer.as_ref().ptr as *mut SyncSendableMutableLakeSoulWriter); + let writer = NonNull::new_unchecked(writer.as_ref().ptr as *mut SyncSendableMutableLakeSoulWriter).as_mut(); let mut ffi_array = FFI_ArrowArray::empty(); (array_addr as *mut FFI_ArrowArray).copy_to(&mut ffi_array as *mut FFI_ArrowArray, 1); let mut ffi_schema = FFI_ArrowSchema::empty(); @@ -738,7 +764,7 @@ pub extern "C" fn write_record_batch_blocked( let array_data = from_ffi(ffi_array, &ffi_schema)?; let struct_array = StructArray::from(array_data); let rb = RecordBatch::from(struct_array); - writer.as_ref().write_batch(rb)?; + writer.write_batch(rb)?; Ok(()) }; let result: lakesoul_io::Result<()> = result_fn(); @@ -749,6 +775,46 @@ pub extern "C" fn write_record_batch_blocked( } } +#[no_mangle] +pub extern "C" fn write_record_batch_ipc_blocked( + writer: NonNull>, + ipc_addr: c_ptrdiff_t, + len: i64, +) -> *const c_char { + let writer = unsafe { + NonNull::new_unchecked(writer.as_ref().ptr as *mut SyncSendableMutableLakeSoulWriter).as_mut() + }; + let raw_parts = unsafe { + std::slice::from_raw_parts(ipc_addr as *const u8, len as usize) + }; + + let reader = std::io::Cursor::new(raw_parts); + let mut reader = arrow_ipc::reader::StreamReader::try_new(reader, None).unwrap(); + let mut row_count = 0; + loop { + if reader.is_finished() { + break; + } + match reader.next().transpose() { + Ok(Some(batch)) => { + let num_rows = batch.num_rows(); + match writer.write_batch(batch) { + Ok(_) => { + row_count += num_rows + } + Err(e) => return CString::new(format!("Error: {}", e).as_str()).unwrap().into_raw(), + } + } + Ok(None) => { + break; + } + Err(e) => return CString::new(format!("Error: {}", e).as_str()).unwrap().into_raw(), + } + } + CString::new(format!("Ok: {}", row_count).as_str()).unwrap().into_raw() +} + + #[no_mangle] pub extern "C" fn export_bytes_result( callback: extern "C" fn(bool, *const c_char), diff --git a/rust/lakesoul-io/Cargo.toml b/rust/lakesoul-io/Cargo.toml index 99ace6c18..d1b12e5cd 100644 --- a/rust/lakesoul-io/Cargo.toml +++ b/rust/lakesoul-io/Cargo.toml @@ -45,10 +45,14 @@ anyhow = { workspace = true, features = [] } prost = { workspace = true } env_logger = "0.11" hex = "0.4" +dhat = { version="0.3.3", optional = true } +async-recursion = "1.1.1" [features] hdfs = ["dep:hdrs", "dep:hdfs-sys"] default = [] +dhat-heap = ["dep:dhat"] # if you are doing heap profiling +dhat-ad-hoc = [] # if you are doing ad hoc profiling [target.'cfg(target_os = "windows")'.dependencies] datafusion-substrait = { workspace = true } @@ -61,4 +65,5 @@ datafusion-substrait = { workspace = true, features = ["protoc"] } tempfile = "3.3.0" comfy-table = "6.0" whoami = "1.5" +tracing-subscriber = "0.3.18" diff --git a/rust/lakesoul-io/examples/writer_profiling.rs b/rust/lakesoul-io/examples/writer_profiling.rs new file mode 100644 index 000000000..811019826 --- /dev/null +++ b/rust/lakesoul-io/examples/writer_profiling.rs @@ -0,0 +1,102 @@ +use std::{fs::File, sync::Arc}; + +use arrow_array::{ArrayRef, RecordBatch, StringArray}; +use datafusion_common::Result; +use lakesoul_io::{lakesoul_io_config::LakeSoulIOConfigBuilder, lakesoul_writer::SyncSendableMutableLakeSoulWriter}; +use parquet::arrow::arrow_reader::ParquetRecordBatchReader; +use rand::distributions::DistString; +use tokio::{runtime::Builder, time::Instant}; + +#[cfg(feature = "dhat-heap")] +#[global_allocator] +static ALLOC: dhat::Alloc = dhat::Alloc; + +fn create_batch(num_columns: usize, num_rows: usize, str_len: usize) -> RecordBatch { + let mut rng = rand::thread_rng(); + let iter = (0..num_columns) + .into_iter() + .map(|i| { + ( + format!("col_{}", i), + Arc::new(StringArray::from( + (0..num_rows) + .into_iter() + .map(|_| rand::distributions::Alphanumeric.sample_string(&mut rng, str_len)) + .collect::>(), + )) as ArrayRef, + true, + ) + }) + .collect::>(); + RecordBatch::try_from_iter_with_nullable(iter).unwrap() +} + +fn main() -> Result<()> { + #[cfg(feature = "dhat-heap")] + let _profiler = dhat::Profiler::new_heap(); + + let runtime = Builder::new_multi_thread().enable_all().build().unwrap(); + let num_batch = 1000; + let num_rows = 1000; + let num_columns = 20; + let str_len = 4; + + let to_write = create_batch(num_columns, num_rows, str_len); + let temp_dir = tempfile::tempdir()?; + let path = temp_dir + .into_path() + .join("test.parquet") + .into_os_string() + .into_string() + .unwrap(); + dbg!(&path); + let writer_conf = LakeSoulIOConfigBuilder::new() + .with_files(vec![path.clone()]) + .with_thread_num(2) + .with_batch_size(num_rows) + // .with_max_row_group_size(2000) + // .with_max_row_group_num_values(4_00_000) + .with_schema(to_write.schema()) + // .with_primary_keys(vec!["col_2".to_string()]) + .with_primary_keys( + (0..num_columns - 1) + .into_iter() + .map(|i| format!("col_{}", i)) + .collect::>(), + ) + // .with_aux_sort_column("col2".to_string()) + .build(); + + let writer = SyncSendableMutableLakeSoulWriter::try_new(writer_conf, runtime)?; + + let start = Instant::now(); + for _ in 0..num_batch { + let once_start = Instant::now(); + writer.write_batch(create_batch(num_columns, num_rows, str_len))?; + // println!("write batch once cost: {}", once_start.elapsed().as_millis()); + } + let flush_start = Instant::now(); + writer.flush_and_close()?; + println!("flush cost: {}", flush_start.elapsed().as_millis()); + println!( + "num_batch={}, num_columns={}, num_rows={}, str_len={}, cost_mills={}", + num_batch, + num_columns, + num_rows, + str_len, + start.elapsed().as_millis() + ); + + // let file = File::open(path.clone())?; + // let mut record_batch_reader = ParquetRecordBatchReader::try_new(file, 100_000).unwrap(); + + // let actual_batch = record_batch_reader + // .next() + // .expect("No batch found") + // .expect("Unable to get batch"); + + // assert_eq!(to_write.schema(), actual_batch.schema()); + // assert_eq!(num_columns, actual_batch.num_columns()); + // assert_eq!(num_rows * num_batch, actual_batch.num_rows()); + Ok(()) +} diff --git a/rust/lakesoul-io/src/async_writer/mod.rs b/rust/lakesoul-io/src/async_writer/mod.rs new file mode 100644 index 000000000..bbbe8b000 --- /dev/null +++ b/rust/lakesoul-io/src/async_writer/mod.rs @@ -0,0 +1,144 @@ +// SPDX-FileCopyrightText: 2023 LakeSoul Contributors +// +// SPDX-License-Identifier: Apache-2.0 + +mod multipart_writer; +pub use multipart_writer::MultiPartAsyncWriter; + +mod sort_writer; +pub use sort_writer::SortAsyncWriter; + +mod partitioning_writer; +pub use partitioning_writer::PartitioningAsyncWriter; + +use std::{ + any::Any, + collections::VecDeque, + fmt::{Debug, Formatter}, + io::{ErrorKind, Write}, + sync::Arc, +}; + +use arrow_array::RecordBatch; +use arrow_schema::SchemaRef; +use atomic_refcell::AtomicRefCell; +use datafusion::{ + execution::{SendableRecordBatchStream, TaskContext}, + physical_expr::PhysicalSortExpr, + physical_plan::{ + stream::RecordBatchReceiverStreamBuilder, DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, + }, +}; +use datafusion_common::{DataFusionError, Result}; +use parquet::format::FileMetaData; + + +// The result of a flush operation with format (partition_desc, file_path, file_meta) +pub type WriterFlushResult = Result>; + +#[async_trait::async_trait] +pub trait AsyncBatchWriter { + async fn write_record_batch(&mut self, batch: RecordBatch) -> Result<()>; + + async fn flush_and_close(self: Box) -> WriterFlushResult; + + async fn abort_and_close(self: Box) -> Result<()>; + + fn schema(&self) -> SchemaRef; + + fn buffered_size(&self) -> u64 { + 0 + } +} + +/// A VecDeque which is both std::io::Write and bytes::Buf +#[derive(Clone)] +struct InMemBuf(Arc>>); + +impl Write for InMemBuf { + #[inline] + fn write(&mut self, buf: &[u8]) -> std::io::Result { + let mut v = self + .0 + .try_borrow_mut() + .map_err(|_| std::io::Error::from(ErrorKind::AddrInUse))?; + v.extend(buf); + Ok(buf.len()) + } + + #[inline] + fn flush(&mut self) -> std::io::Result<()> { + Ok(()) + } + + #[inline] + fn write_all(&mut self, buf: &[u8]) -> std::io::Result<()> { + let mut v = self + .0 + .try_borrow_mut() + .map_err(|_| std::io::Error::from(ErrorKind::AddrInUse))?; + v.extend(buf); + Ok(()) + } +} + +pub struct ReceiverStreamExec { + receiver_stream_builder: AtomicRefCell>, + schema: SchemaRef, +} + +impl ReceiverStreamExec { + pub fn new(receiver_stream_builder: RecordBatchReceiverStreamBuilder, schema: SchemaRef) -> Self { + Self { + receiver_stream_builder: AtomicRefCell::new(Some(receiver_stream_builder)), + schema, + } + } +} + +impl Debug for ReceiverStreamExec { + fn fmt(&self, _f: &mut Formatter<'_>) -> std::fmt::Result { + todo!() + } +} + +impl DisplayAs for ReceiverStreamExec { + fn fmt_as(&self, _t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { + write!(f, "ReceiverStreamExec") + } +} + +impl ExecutionPlan for ReceiverStreamExec { + fn as_any(&self) -> &dyn Any { + self + } + + fn schema(&self) -> SchemaRef { + Arc::clone(&self.schema) + } + + fn output_partitioning(&self) -> Partitioning { + Partitioning::UnknownPartitioning(1) + } + + fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { + None + } + + fn children(&self) -> Vec> { + unimplemented!() + } + + fn with_new_children(self: Arc, _children: Vec>) -> Result> { + unimplemented!() + } + + fn execute(&self, _partition: usize, _context: Arc) -> Result { + let builder = self + .receiver_stream_builder + .borrow_mut() + .take() + .ok_or(DataFusionError::Internal("empty receiver stream".to_string()))?; + Ok(builder.build()) + } +} diff --git a/rust/lakesoul-io/src/async_writer/multipart_writer.rs b/rust/lakesoul-io/src/async_writer/multipart_writer.rs new file mode 100644 index 000000000..f249809b3 --- /dev/null +++ b/rust/lakesoul-io/src/async_writer/multipart_writer.rs @@ -0,0 +1,217 @@ +// SPDX-FileCopyrightText: 2023 LakeSoul Contributors +// +// SPDX-License-Identifier: Apache-2.0 + +use std::{collections::VecDeque, sync::Arc}; + +use arrow_array::RecordBatch; +use arrow_schema::SchemaRef; +use atomic_refcell::AtomicRefCell; +use datafusion::execution::{object_store::ObjectStoreUrl, TaskContext}; +use datafusion_common::{project_schema, DataFusionError, Result}; +use object_store::{path::Path, MultipartId, ObjectStore}; +use parquet::{arrow::ArrowWriter, basic::Compression, file::properties::WriterProperties}; +use tokio::io::{AsyncWrite, AsyncWriteExt}; +use url::Url; + +use crate::{ + constant::TBD_PARTITION_DESC, helpers::get_batch_memory_size, lakesoul_io_config::{create_session_context, LakeSoulIOConfig}, transform::{uniform_record_batch, uniform_schema} +}; + +use super::{AsyncBatchWriter, WriterFlushResult, InMemBuf}; + +/// An async writer using object_store's multi-part upload feature for cloud storage. +/// This writer uses a `VecDeque` as `std::io::Write` for arrow-rs's ArrowWriter. +/// Everytime when a new RowGroup is flushed, the length of the VecDeque would grow. +/// At this time, we pass the VecDeque as `bytes::Buf` to `AsyncWriteExt::write_buf` provided +/// by object_store, which would drain and copy the content of the VecDeque so that we could reuse it. +/// The `CloudMultiPartUpload` itself would try to concurrently upload parts, and +/// all parts will be committed to cloud storage by shutdown the `AsyncWrite` object. +pub struct MultiPartAsyncWriter { + in_mem_buf: InMemBuf, + task_context: Arc, + schema: SchemaRef, + writer: Box, + multi_part_id: MultipartId, + arrow_writer: ArrowWriter, + _config: LakeSoulIOConfig, + object_store: Arc, + path: Path, + absolute_path: String, + num_rows: u64, + buffered_size: u64, +} + +impl MultiPartAsyncWriter { + pub async fn try_new_with_context(config: &mut LakeSoulIOConfig, task_context: Arc) -> Result { + if config.files.is_empty() { + return Err(DataFusionError::Internal( + "wrong number of file names provided for writer".to_string(), + )); + } + let file_name = &config + .files + .last() + .ok_or(DataFusionError::Internal("wrong file name".to_string()))?; + + // local style path should have already been handled in create_session_context, + // so we don't have to deal with ParseError::RelativeUrlWithoutBase here + let (object_store, path) = match Url::parse(file_name.as_str()) { + Ok(url) => Ok(( + task_context + .runtime_env() + .object_store(ObjectStoreUrl::parse(&url[..url::Position::BeforePath])?)?, + Path::from_url_path(url.path())?, + )), + Err(e) => Err(DataFusionError::External(Box::new(e))), + }?; + + // get underlying multipart uploader + let (multipart_id, async_writer) = object_store.put_multipart(&path).await?; + let in_mem_buf = InMemBuf(Arc::new(AtomicRefCell::new(VecDeque::::with_capacity( + 16 * 1024, // 16kb + )))); + let schema = uniform_schema(config.target_schema.0.clone()); + + // O(nm), n = number of fields, m = number of range partitions + let schema_projection_excluding_range = schema + .fields() + .iter() + .enumerate() + .filter_map(|(idx, field)| match config.range_partitions.contains(field.name()) { + true => None, + false => Some(idx), + }) + .collect::>(); + let writer_schema = project_schema(&schema, Some(&schema_projection_excluding_range))?; + + let max_row_group_size = if config.max_row_group_size * schema.fields().len() > config.max_row_group_num_values + { + config + .batch_size + .max(config.max_row_group_num_values / schema.fields().len()) + } else { + config.max_row_group_size + }; + let arrow_writer = ArrowWriter::try_new( + in_mem_buf.clone(), + writer_schema, + Some( + WriterProperties::builder() + .set_max_row_group_size(max_row_group_size) + .set_write_batch_size(config.batch_size) + .set_compression(Compression::SNAPPY) + // .set_statistics_enabled(parquet::file::properties::EnabledStatistics::None) + .build(), + ), + )?; + + Ok(MultiPartAsyncWriter { + in_mem_buf, + task_context, + schema, + writer: async_writer, + multi_part_id: multipart_id, + arrow_writer, + _config: config.clone(), + object_store, + path, + absolute_path: file_name.to_string(), + num_rows: 0, + buffered_size: 0, + }) + } + + pub async fn try_new(mut config: LakeSoulIOConfig) -> Result { + let task_context = create_session_context(&mut config)?.task_ctx(); + Self::try_new_with_context(&mut config, task_context).await + } + + async fn write_batch( + batch: RecordBatch, + arrow_writer: &mut ArrowWriter, + in_mem_buf: &mut InMemBuf, + // underlying writer + writer: &mut Box, + ) -> Result<()> { + arrow_writer.write(&batch)?; + let mut v = in_mem_buf + .0 + .try_borrow_mut() + .map_err(|e| DataFusionError::Internal(format!("{:?}", e)))?; + if v.len() > 0 { + MultiPartAsyncWriter::write_part(writer, &mut v).await + } else { + Ok(()) + } + } + + pub async fn write_part( + writer: &mut Box, + in_mem_buf: &mut VecDeque, + ) -> Result<()> { + writer.write_all_buf(in_mem_buf).await?; + Ok(()) + } + + pub fn nun_rows(&self) -> u64 { + self.num_rows + } + + pub fn absolute_path(&self) -> String { + self.absolute_path.clone() + } + + pub fn task_ctx(&self) -> Arc { + self.task_context.clone() + } +} + +#[async_trait::async_trait] +impl AsyncBatchWriter for MultiPartAsyncWriter { + + async fn write_record_batch(&mut self, batch: RecordBatch) -> Result<()> { + let batch = uniform_record_batch(batch)?; + self.num_rows += batch.num_rows() as u64; + self.buffered_size += get_batch_memory_size(&batch)? as u64; + MultiPartAsyncWriter::write_batch(batch, &mut self.arrow_writer, &mut self.in_mem_buf, &mut self.writer).await + } + + async fn flush_and_close(self: Box) -> WriterFlushResult { + // close arrow writer to flush remaining rows + let mut this = *self; + let arrow_writer = this.arrow_writer; + let file_path = this.absolute_path.clone(); + let metadata = arrow_writer.close()?; + let mut v = this + .in_mem_buf + .0 + .try_borrow_mut() + .map_err(|e| DataFusionError::Internal(format!("{:?}", e)))?; + if v.len() > 0 { + MultiPartAsyncWriter::write_part(&mut this.writer, &mut v).await?; + } + // shutdown multi-part async writer to complete the upload + this.writer.flush().await?; + this.writer.shutdown().await?; + Ok(vec![(TBD_PARTITION_DESC.to_string(), file_path, metadata)]) + } + + async fn abort_and_close(self: Box) -> Result<()> { + let this = *self; + this.object_store + .abort_multipart(&this.path, &this.multi_part_id) + .await + .map_err(DataFusionError::ObjectStore)?; + Ok(()) + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn buffered_size(&self) -> u64 { + self.buffered_size + } + +} diff --git a/rust/lakesoul-io/src/async_writer/partitioning_writer.rs b/rust/lakesoul-io/src/async_writer/partitioning_writer.rs new file mode 100644 index 000000000..288d213ab --- /dev/null +++ b/rust/lakesoul-io/src/async_writer/partitioning_writer.rs @@ -0,0 +1,379 @@ +// SPDX-FileCopyrightText: 2023 LakeSoul Contributors +// +// SPDX-License-Identifier: Apache-2.0 + +use std::{collections::HashMap, sync::Arc}; + +use arrow_array::RecordBatch; +use arrow_schema::{SchemaRef, SortOptions}; +use datafusion::{ + execution::TaskContext, + physical_expr::{ + expressions::{col, Column}, + PhysicalSortExpr, + }, + physical_plan::{ + projection::ProjectionExec, + sorts::sort::SortExec, + stream::RecordBatchReceiverStream, + ExecutionPlan, Partitioning, PhysicalExpr, + }, +}; +use datafusion_common::{DataFusionError, Result}; + +use rand::distributions::DistString; +use tokio::{ + sync::mpsc::Sender, + task::JoinHandle, +}; +use tokio_stream::StreamExt; +use tracing::debug; + +use crate::{ + helpers::{columnar_values_to_partition_desc, columnar_values_to_sub_path, get_batch_memory_size, get_columnar_values}, + lakesoul_io_config::{create_session_context, LakeSoulIOConfig, LakeSoulIOConfigBuilder}, + repartition::RepartitionByRangeAndHashExec, +}; + +use super::{AsyncBatchWriter, WriterFlushResult, MultiPartAsyncWriter, ReceiverStreamExec}; + +// type PartitionedWriterInfo = Arc>>>; + +/// Wrap the above async writer with a RepartitionExec to +/// dynamic repartitioning the batches before write to async writer +pub struct PartitioningAsyncWriter { + schema: SchemaRef, + sorter_sender: Sender>, + _partitioning_exec: Arc, + join_handle: Option>, + err: Option, + buffered_size: u64, +} + +impl PartitioningAsyncWriter { + pub fn try_new(config: LakeSoulIOConfig) -> Result { + let mut config = config.clone(); + let task_context = create_session_context(&mut config)?.task_ctx(); + + let schema = config.target_schema.0.clone(); + let receiver_stream_builder = RecordBatchReceiverStream::builder(schema.clone(), 8); + let tx = receiver_stream_builder.tx(); + let recv_exec = ReceiverStreamExec::new(receiver_stream_builder, schema.clone()); + + let partitioning_exec = PartitioningAsyncWriter::get_partitioning_exec(recv_exec, config.clone())?; + + // launch one async task per *input* partition + let mut join_handles = vec![]; + + let write_id = rand::distributions::Alphanumeric.sample_string(&mut rand::thread_rng(), 16); + + // let partitioned_file_path_and_row_count = Arc::new(Mutex::new(HashMap::, u64)>::new())); + for i in 0..partitioning_exec.output_partitioning().partition_count() { + let sink_task = tokio::spawn(Self::pull_and_sink( + partitioning_exec.clone(), + i, + task_context.clone(), + config.clone().into(), + Arc::new(config.range_partitions.clone()), + write_id.clone() + )); + // // In a separate task, wait for each input to be done + // // (and pass along any errors, including panic!s) + join_handles.push(sink_task); + } + + let join_handle = tokio::spawn(Self::await_and_summary( + join_handles, + // partitioned_file_path_and_row_count, + )); + + Ok(Self { + schema, + sorter_sender: tx, + _partitioning_exec: partitioning_exec, + join_handle: Some(join_handle), + err: None, + buffered_size: 0, + }) + } + + fn get_partitioning_exec(input: ReceiverStreamExec, config: LakeSoulIOConfig) -> Result> { + let sort_exprs: Vec = config + .range_partitions + .iter() + .chain(config.primary_keys.iter()) + // add aux sort cols to sort expr + .chain(config.aux_sort_cols.iter()) + .map(|pk| { + let col = Column::new_with_schema(pk.as_str(), &config.target_schema.0)?; + Ok(PhysicalSortExpr { + expr: Arc::new(col), + options: SortOptions::default(), + }) + }) + .collect::>>()?; + if sort_exprs.is_empty() { + return Ok(Arc::new(input)); + } + + let sort_exec = Arc::new(SortExec::new(sort_exprs, Arc::new(input))); + + // see if we need to prune aux sort cols + let sort_exec: Arc = if config.aux_sort_cols.is_empty() { + sort_exec + } else { + // O(nm), n = number of target schema fields, m = number of aux sort cols + let proj_expr: Vec<(Arc, String)> = config + .target_schema + .0 + .fields + .iter() + .filter_map(|f| { + if config.aux_sort_cols.contains(f.name()) { + // exclude aux sort cols + None + } else { + Some(col(f.name().as_str(), &config.target_schema.0).map(|e| (e, f.name().clone()))) + } + }) + .collect::, String)>>>()?; + Arc::new(ProjectionExec::try_new(proj_expr, sort_exec)?) + }; + + let exec_plan = if config.primary_keys.is_empty() && config.range_partitions.is_empty() { + sort_exec + } else { + let sorted_schema = sort_exec.schema(); + + let range_partitioning_expr: Vec> = config + .range_partitions + .iter() + .map(|col| { + let idx = sorted_schema.index_of(col.as_str())?; + Ok(Arc::new(Column::new(col.as_str(), idx)) as Arc) + }) + .collect::>>()?; + + let hash_partitioning_expr: Vec> = config + .primary_keys + .iter() + .map(|col| { + let idx = sorted_schema.index_of(col.as_str())?; + Ok(Arc::new(Column::new(col.as_str(), idx)) as Arc) + }) + .collect::>>()?; + let hash_partitioning = Partitioning::Hash(hash_partitioning_expr, config.hash_bucket_num); + + Arc::new(RepartitionByRangeAndHashExec::try_new( + sort_exec, + range_partitioning_expr, + hash_partitioning, + )?) + }; + + Ok(exec_plan) + } + + async fn pull_and_sink( + input: Arc, + partition: usize, + context: Arc, + config_builder: LakeSoulIOConfigBuilder, + range_partitions: Arc>, + write_id: String, + // partitioned_flush_result: PartitionedWriterInfo, + ) -> Result>> { + let mut data = input.execute(partition, context.clone())?; + // O(nm), n = number of data fields, m = number of range partitions + let schema_projection_excluding_range = data + .schema() + .fields() + .iter() + .enumerate() + .filter_map(|(idx, field)| match range_partitions.contains(field.name()) { + true => None, + false => Some(idx), + }) + .collect::>(); + + let mut err = None; + + + let mut partitioned_writer = HashMap::>::new(); + let mut flush_join_handle_list = Vec::new(); + // let mut partitioned_flush_result_locked = partitioned_flush_result.lock().await; + while let Some(batch_result) = data.next().await { + match batch_result { + Ok(batch) => { + debug!("write record_batch with {} rows", batch.num_rows()); + let columnar_values = get_columnar_values(&batch, range_partitions.clone())?; + let partition_desc = columnar_values_to_partition_desc(&columnar_values); + let partition_sub_path = columnar_values_to_sub_path(&columnar_values); + let batch_excluding_range = batch.project(&schema_projection_excluding_range)?; + + let file_absolute_path = format!( + "{}{}part-{}_{:0>4}.parquet", + config_builder.prefix(), + partition_sub_path, + write_id, + partition + ); + + if !partitioned_writer.contains_key(&partition_desc) { + let mut config = config_builder.clone().with_files(vec![file_absolute_path]).build(); + + let writer = MultiPartAsyncWriter::try_new_with_context(&mut config, context.clone()).await?; + partitioned_writer.insert(partition_desc.clone(), Box::new(writer)); + } + + if let Some(async_writer) = partitioned_writer.get_mut(&partition_desc) { + // row_count += batch_excluding_range.num_rows(); + async_writer.write_record_batch(batch_excluding_range).await?; + } + + } + // received abort signal + Err(e) => { + err = Some(e); + break; + } + } + } + if let Some(e) = err { + for (_, writer) in partitioned_writer.into_iter() { + match writer.abort_and_close().await { + Ok(_) => match e { + DataFusionError::Internal(ref err_msg) if err_msg == "external abort" => (), + _ => return Err(e), + }, + Err(abort_err) => { + return Err(DataFusionError::Internal(format!( + "Abort failed {:?}, previous error {:?}", + abort_err, e + ))) + } + } + } + Ok(flush_join_handle_list) + } else { + + for (partition_desc, writer) in partitioned_writer.into_iter() { + + let flush_result = tokio::spawn(async move { + let writer_flush_results =writer.flush_and_close().await?; + Ok( + writer_flush_results.into_iter().map( + |(_, path, file_metadata)| + { + (partition_desc.clone(), path, file_metadata) + } + ).collect::>() + ) + }); + flush_join_handle_list.push(flush_result); + } + Ok(flush_join_handle_list) + } + } + + async fn await_and_summary( + join_handles: Vec>>>>, + // partitioned_file_path_and_row_count: PartitionedWriterInfo, + ) -> WriterFlushResult { + let mut flatten_results = Vec::new(); + let results = futures::future::join_all(join_handles).await; + for result in results { + match result { + Ok(Ok(part_join_handles)) => { + let part_results = futures::future::join_all(part_join_handles).await; + for part_result in part_results { + match part_result { + Ok(Ok(flatten_part_result)) => { + flatten_results.extend(flatten_part_result); + } + Ok(Err(e)) => return Err(DataFusionError::Execution(format!("{}", e))), + Err(e) => return Err(DataFusionError::Execution(format!("{}", e))), + } + } + } + Ok(Err(e)) => return Err(DataFusionError::Execution(format!("{}", e))), + Err(e) => return Err(DataFusionError::Execution(format!("{}", e))), + } + } + Ok(flatten_results) + } +} + +#[async_trait::async_trait] +impl AsyncBatchWriter for PartitioningAsyncWriter { + async fn write_record_batch(&mut self, batch: RecordBatch) -> Result<()> { + // arrow_cast::pretty::print_batches(&[batch.clone()]); + if let Some(err) = &self.err { + return Err(DataFusionError::Internal(format!( + "PartitioningAsyncWriter already failed with error {:?}", + err + ))); + } + + let memory_size = get_batch_memory_size(&batch)? as u64; + let send_result = self.sorter_sender.send(Ok(batch)).await; + self.buffered_size += memory_size; + match send_result { + Ok(_) => Ok(()), + // channel has been closed, indicating error happened during sort write + Err(e) => { + if let Some(join_handle) = self.join_handle.take() { + let result = join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))?; + self.err = result.err(); + Err(DataFusionError::Internal(format!( + "Write to PartitioningAsyncWriter failed: {:?}", + self.err + ))) + } else { + self.err = Some(DataFusionError::External(Box::new(e))); + Err(DataFusionError::Internal(format!( + "Write to PartitioningAsyncWriter failed: {:?}", + self.err + ))) + } + } + } + } + + async fn flush_and_close(self: Box) -> WriterFlushResult { + if let Some(join_handle) = self.join_handle { + let sender = self.sorter_sender; + drop(sender); + join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))? + } else { + Err(DataFusionError::Internal( + "PartitioningAsyncWriter has been aborted, cannot flush".to_string(), + )) + } + } + + async fn abort_and_close(self: Box) -> Result<()> { + if let Some(join_handle) = self.join_handle { + let sender = self.sorter_sender; + // send abort signal to the task + sender + .send(Err(DataFusionError::Internal("external abort".to_string()))) + .await + .map_err(|e| DataFusionError::External(Box::new(e)))?; + drop(sender); + let _ = join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))?; + Ok(()) + } else { + // previous error has already aborted writer + Ok(()) + } + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn buffered_size(&self) -> u64 { + self.buffered_size + } +} diff --git a/rust/lakesoul-io/src/async_writer/sort_writer.rs b/rust/lakesoul-io/src/async_writer/sort_writer.rs new file mode 100644 index 000000000..c10fa4ce1 --- /dev/null +++ b/rust/lakesoul-io/src/async_writer/sort_writer.rs @@ -0,0 +1,206 @@ +// SPDX-FileCopyrightText: 2023 LakeSoul Contributors +// +// SPDX-License-Identifier: Apache-2.0 + +use std::sync::Arc; + +use arrow_array::RecordBatch; +use arrow_schema::{SchemaRef, SortOptions}; +use datafusion::{ + physical_expr::{ + expressions::{col, Column}, + PhysicalSortExpr, + }, + physical_plan::{ + projection::ProjectionExec, + sorts::sort::SortExec, + stream::RecordBatchReceiverStream, + ExecutionPlan, PhysicalExpr, + }, +}; +use datafusion_common::{DataFusionError, Result}; +use tokio::{sync::mpsc::Sender, task::JoinHandle}; +use tokio_stream::StreamExt; + +use crate::{helpers::get_batch_memory_size, lakesoul_io_config::LakeSoulIOConfig}; + +use super::{AsyncBatchWriter, WriterFlushResult, MultiPartAsyncWriter, ReceiverStreamExec}; + +/// Wrap the above async writer with a SortExec to +/// sort the batches before write to async writer +pub struct SortAsyncWriter { + schema: SchemaRef, + sorter_sender: Sender>, + _sort_exec: Arc, + join_handle: Option>, + err: Option, + buffered_size: u64, +} + +impl SortAsyncWriter { + pub fn try_new( + async_writer: MultiPartAsyncWriter, + config: LakeSoulIOConfig, + // runtime: Arc, + ) -> Result { + // let _ = runtime.enter(); + let schema = config.target_schema.0.clone(); + let receiver_stream_builder = RecordBatchReceiverStream::builder(schema.clone(), 8); + let tx = receiver_stream_builder.tx(); + let recv_exec = ReceiverStreamExec::new(receiver_stream_builder, schema.clone()); + + let sort_exprs: Vec = config + .primary_keys + .iter() + // add aux sort cols to sort expr + .chain(config.aux_sort_cols.iter()) + .map(|pk| { + let col = Column::new_with_schema(pk.as_str(), &config.target_schema.0)?; + Ok(PhysicalSortExpr { + expr: Arc::new(col), + options: SortOptions::default(), + }) + }) + .collect::>>()?; + let sort_exec = Arc::new(SortExec::new(sort_exprs, Arc::new(recv_exec))); + + // see if we need to prune aux sort cols + let exec_plan: Arc = if config.aux_sort_cols.is_empty() { + sort_exec + } else { + // O(nm), n = number of target schema fields, m = number of aux sort cols + let proj_expr: Vec<(Arc, String)> = config + .target_schema + .0 + .fields + .iter() + .filter_map(|f| { + if config.aux_sort_cols.contains(f.name()) { + // exclude aux sort cols + None + } else { + Some(col(f.name().as_str(), &config.target_schema.0).map(|e| (e, f.name().clone()))) + } + }) + .collect::, String)>>>()?; + Arc::new(ProjectionExec::try_new(proj_expr, sort_exec)?) + }; + + let mut sorted_stream = exec_plan.execute(0, async_writer.task_ctx())?; + + let mut async_writer = Box::new(async_writer); + let join_handle = tokio::task::spawn(async move { + let mut err = None; + while let Some(batch) = sorted_stream.next().await { + match batch { + Ok(batch) => { + async_writer.write_record_batch(batch).await?; + } + // received abort signal + Err(e) => { + err = Some(e); + break; + } + } + } + if let Some(e) = err { + let result = async_writer.abort_and_close().await; + match result { + Ok(_) => match e { + DataFusionError::Internal(ref err_msg) if err_msg == "external abort" => Ok(vec![]), + _ => Err(e), + }, + Err(abort_err) => Err(DataFusionError::Internal(format!( + "Abort failed {:?}, previous error {:?}", + abort_err, e + ))), + } + } else { + async_writer.flush_and_close().await + } + }); + + Ok(SortAsyncWriter { + schema, + sorter_sender: tx, + _sort_exec: exec_plan, + join_handle: Some(join_handle), + err: None, + buffered_size: 0, + }) + } +} + +#[async_trait::async_trait] +impl AsyncBatchWriter for SortAsyncWriter { + async fn write_record_batch(&mut self, batch: RecordBatch) -> Result<()> { + if let Some(err) = &self.err { + return Err(DataFusionError::Internal(format!( + "SortAsyncWriter already failed with error {:?}", + err + ))); + } + + let memory_size = get_batch_memory_size(&batch)? as u64; + let send_result = self.sorter_sender.send(Ok(batch)).await; + self.buffered_size += memory_size; + + match send_result { + Ok(_) => Ok(()), + // channel has been closed, indicating error happened during sort write + Err(e) => { + if let Some(join_handle) = self.join_handle.take() { + let result = join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))?; + self.err = result.err(); + Err(DataFusionError::Internal(format!( + "Write to SortAsyncWriter failed: {:?}", + self.err + ))) + } else { + self.err = Some(DataFusionError::External(Box::new(e))); + Err(DataFusionError::Internal(format!( + "Write to SortAsyncWriter failed: {:?}", + self.err + ))) + } + } + } + } + + async fn flush_and_close(self: Box) -> WriterFlushResult { + if let Some(join_handle) = self.join_handle { + let sender = self.sorter_sender; + drop(sender); + join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))? + } else { + Err(DataFusionError::Internal( + "SortAsyncWriter has been aborted, cannot flush".to_string(), + )) + } + } + + async fn abort_and_close(self: Box) -> Result<()> { + if let Some(join_handle) = self.join_handle { + let sender = self.sorter_sender; + // send abort signal to the task + sender + .send(Err(DataFusionError::Internal("external abort".to_string()))) + .await + .map_err(|e| DataFusionError::External(Box::new(e)))?; + drop(sender); + let _ = join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))?; + Ok(()) + } else { + // previous error has already aborted writer + Ok(()) + } + } + + fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + fn buffered_size(&self) -> u64 { + self.buffered_size + } +} diff --git a/rust/lakesoul-io/src/constant.rs b/rust/lakesoul-io/src/constant.rs index 67c8c6f96..c4a446658 100644 --- a/rust/lakesoul-io/src/constant.rs +++ b/rust/lakesoul-io/src/constant.rs @@ -17,6 +17,9 @@ pub static LAKESOUL_EMPTY_STRING: &str = "__L@KE$OUL_EMPTY_STRING__"; pub static LAKESOUL_EQ: &str = "__L@KE$OUL_EQ__"; pub static LAKESOUL_COMMA: &str = "__L@KE$OUL_COMMA__"; +pub static DEFAULT_PARTITION_DESC: &str = "-5"; +pub static TBD_PARTITION_DESC: &str = "-4"; + pub static DATE32_FORMAT: &str = "%Y-%m-%d"; pub static FLINK_TIMESTAMP_FORMAT: &str = "%Y-%m-%d %H:%M:%S%.9f"; pub static TIMESTAMP_SECOND_FORMAT: &str = "%Y-%m-%dT%H:%M:%S"; @@ -24,6 +27,8 @@ pub static TIMESTAMP_MILLSECOND_FORMAT: &str = "%Y-%m-%dT%H:%M:%S%.3f"; pub static TIMESTAMP_MICROSECOND_FORMAT: &str = "%Y-%m-%dT%H:%M:%S%.6f"; pub static TIMESTAMP_NANOSECOND_FORMAT: &str = "%Y-%m-%dT%H:%M:%S%.9f"; +pub static NUM_COLUMN_OPTIMIZE_THRESHOLD: usize = 200; + lazy_static! { pub static ref ARROW_CAST_OPTIONS: CastOptions<'static> = CastOptions::default(); } diff --git a/rust/lakesoul-io/src/filter/parser.rs b/rust/lakesoul-io/src/filter/parser.rs index 970673bab..0d3369e0c 100644 --- a/rust/lakesoul-io/src/filter/parser.rs +++ b/rust/lakesoul-io/src/filter/parser.rs @@ -229,6 +229,7 @@ impl Parser { } pub(crate) fn parse_proto(plan: &Plan, df_schema: &DFSchema) -> Result { + let function_extension = plan .extensions .iter() @@ -732,7 +733,7 @@ fn _from_nullability(nullability: Nullability) -> bool { mod tests { use std::result::Result; - use datafusion::prelude::{ParquetReadOptions, SessionContext}; + use datafusion::{logical_expr::{LogicalPlan, TableScan}, prelude::{ParquetReadOptions, SessionContext}}; use prost::Message; use super::*; @@ -749,6 +750,7 @@ mod tests { #[tokio::test] async fn tt() { + let ctx = SessionContext::new(); let options = ParquetReadOptions::default(); let table_path = "/var/folders/_b/qyl87wbn1119cvw8kts6fqtw0000gn/T/lakeSource/type/part-00000-97db3149-f99e-404a-aa9a-2af4ab3f7a44_00000.c000.parquet"; diff --git a/rust/lakesoul-io/src/helpers.rs b/rust/lakesoul-io/src/helpers.rs index f4a70baae..e059824d0 100644 --- a/rust/lakesoul-io/src/helpers.rs +++ b/rust/lakesoul-io/src/helpers.rs @@ -7,7 +7,7 @@ use std::{collections::HashMap, sync::Arc}; use arrow::datatypes::UInt32Type; use arrow_array::{RecordBatch, UInt32Array}; use arrow_buffer::i256; -use arrow_schema::{DataType, Field, Schema, SchemaBuilder, SchemaRef, TimeUnit}; +use arrow_schema::{ArrowError, DataType, Field, Schema, SchemaBuilder, SchemaRef, TimeUnit}; use chrono::{DateTime, Duration}; use datafusion::{ datasource::{ @@ -32,7 +32,9 @@ use url::Url; use crate::{ constant::{ - DATE32_FORMAT, FLINK_TIMESTAMP_FORMAT, LAKESOUL_COMMA, LAKESOUL_EMPTY_STRING, LAKESOUL_EQ, LAKESOUL_NULL_STRING, TIMESTAMP_MICROSECOND_FORMAT, TIMESTAMP_MILLSECOND_FORMAT, TIMESTAMP_NANOSECOND_FORMAT, TIMESTAMP_SECOND_FORMAT + DATE32_FORMAT, FLINK_TIMESTAMP_FORMAT, LAKESOUL_EMPTY_STRING, LAKESOUL_NULL_STRING, + TIMESTAMP_MICROSECOND_FORMAT, TIMESTAMP_MILLSECOND_FORMAT, TIMESTAMP_NANOSECOND_FORMAT, + TIMESTAMP_SECOND_FORMAT, LAKESOUL_COMMA, LAKESOUL_EQ }, filter::parser::Parser, lakesoul_io_config::LakeSoulIOConfig, @@ -508,3 +510,22 @@ pub fn timestamp_str_to_unix_time(value: &str, fmt: &str) -> Result { Ok(datetime.signed_duration_since(epoch_time.naive_utc())) } + +pub fn column_with_name_and_name2index<'a>(schema: &'a SchemaRef, name: &str, name_to_index: &Option>) -> Option<(usize, &'a Field)> { + if let Some(name_to_index) = name_to_index { + name_to_index.get(name).map(|index| (*index, schema.field(*index))) + } else { + schema.column_with_name(name) + } +} + +pub fn get_batch_memory_size(batch: &RecordBatch) -> Result { + Ok( + batch.columns() + .iter() + .map(|array| array.to_data().get_slice_memory_size()) + .collect::, ArrowError>>()? + .into_iter() + .sum() + ) +} diff --git a/rust/lakesoul-io/src/lakesoul_io_config.rs b/rust/lakesoul-io/src/lakesoul_io_config.rs index 87d9c3c37..4a50c9741 100644 --- a/rust/lakesoul-io/src/lakesoul_io_config.rs +++ b/rust/lakesoul-io/src/lakesoul_io_config.rs @@ -2,16 +2,15 @@ // // SPDX-License-Identifier: Apache-2.0 -use std::collections::HashMap; -use std::sync::Arc; -use std::time::Duration; +use std::{collections::HashMap, sync::Arc, time::Duration}; use anyhow::anyhow; use arrow::error::ArrowError; use arrow_schema::{Schema, SchemaRef}; -use datafusion::datasource::object_store::ObjectStoreUrl; pub use datafusion::error::{DataFusionError, Result}; use datafusion::execution::context::{QueryPlanner, SessionState}; +use datafusion::execution::memory_pool::FairSpillPool; +use datafusion::execution::object_store::ObjectStoreUrl; use datafusion::execution::runtime_env::{RuntimeConfig, RuntimeEnv}; use datafusion::logical_expr::Expr; use datafusion::optimizer::analyzer::type_coercion::TypeCoercion; @@ -27,6 +26,7 @@ use derivative::Derivative; use log::info; use object_store::aws::AmazonS3Builder; use object_store::{ClientOptions, RetryConfig}; +use tracing::debug; use url::{ParseError, Url}; #[cfg(feature = "hdfs")] @@ -42,6 +42,16 @@ impl Default for IOSchema { } } +pub static OPTION_KEY_KEEP_ORDERS: &str = "keep_orders"; +pub static OPTION_DEFAULT_VALUE_KEEP_ORDERS: &str = "false"; + +pub static OPTION_KEY_MEM_LIMIT: &str = "mem_limit"; +pub static OPTION_KEY_POOL_SIZE: &str = "pool_size"; +pub static OPTION_KEY_HASH_BUCKET_ID : &str = "hash_bucket_id"; +pub static OPTION_KEY_MAX_FILE_SIZE: &str = "max_file_size"; + + + #[derive(Debug, Derivative)] #[derivative(Default, Clone)] pub struct LakeSoulIOConfig { @@ -71,6 +81,9 @@ pub struct LakeSoulIOConfig { // write row group max row num #[derivative(Default(value = "250000"))] pub(crate) max_row_group_size: usize, + // write row group max num of values + #[derivative(Default(value = "2147483647"))] + pub(crate) max_row_group_num_values: usize, #[derivative(Default(value = "1"))] pub(crate) prefetch_size: usize, #[derivative(Default(value = "false"))] @@ -98,6 +111,8 @@ pub struct LakeSoulIOConfig { // to be compatible with hadoop's fs.defaultFS pub(crate) default_fs: String, + pub(super) options: HashMap, + // if dynamic partition #[derivative(Default(value = "false"))] pub(crate) use_dynamic_partition: bool, @@ -105,6 +120,16 @@ pub struct LakeSoulIOConfig { // if inferring schema #[derivative(Default(value = "false"))] pub(crate) inferring_schema: bool, + + #[derivative(Default(value = "None"))] + pub(crate) memory_limit: Option, + + #[derivative(Default(value = "None"))] + pub(crate) memory_pool_size: Option, + + // max file size of bytes + #[derivative(Default(value = "None"))] + pub(crate) max_file_size: Option, } impl LakeSoulIOConfig { @@ -131,6 +156,30 @@ impl LakeSoulIOConfig { pub fn aux_sort_cols_slice(&self) -> &[String] { &self.aux_sort_cols } + + pub fn option(&self, key: &str) -> Option<&String> { + self.options.get(key) + } + + pub fn prefix(&self) -> &String { + &self.prefix + } + + pub fn keep_ordering(&self) -> bool { + self.option(OPTION_KEY_KEEP_ORDERS).map_or(false, |x| x.eq("true")) + } + + pub fn mem_limit(&self) -> Option { + self.option(OPTION_KEY_MEM_LIMIT).map(|x| x.parse().unwrap()) + } + + pub fn pool_size(&self) -> Option { + self.option(OPTION_KEY_POOL_SIZE).map(|x| x.parse().unwrap()) + } + + pub fn hash_bucket_id(&self) -> usize { + self.option(OPTION_KEY_HASH_BUCKET_ID).map_or(0, |x| x.parse().unwrap()) + } } #[derive(Derivative, Debug)] @@ -206,6 +255,11 @@ impl LakeSoulIOConfigBuilder { self } + pub fn with_max_row_group_num_values(mut self, max_row_group_num_values: usize) -> Self { + self.config.max_row_group_num_values = max_row_group_num_values; + self + } + pub fn with_prefetch_size(mut self, prefetch_size: usize) -> Self { self.config.prefetch_size = prefetch_size; self @@ -261,6 +315,11 @@ impl LakeSoulIOConfigBuilder { self } + pub fn with_option(mut self, key: impl Into, value: impl Into) -> Self { + self.config.options.insert(key.into(), value.into()); + self + } + pub fn with_thread_num(mut self, thread_num: usize) -> Self { self.config.thread_num = thread_num; self @@ -276,6 +335,11 @@ impl LakeSoulIOConfigBuilder { self } + pub fn with_max_file_size(mut self, size: u64) -> Self { + self.config.max_file_size = Some(size); + self + } + pub fn build(self) -> LakeSoulIOConfig { self.config } @@ -485,9 +549,17 @@ pub fn create_session_context_with_planner( sess_conf.options_mut().optimizer.prefer_hash_join = false; //if true, panicked at 'range end out of bounds' sess_conf.options_mut().execution.parquet.pushdown_filters = config.parquet_filter_pushdown; sess_conf.options_mut().execution.target_partitions = 1; + // sess_conf.options_mut().execution.sort_in_place_threshold_bytes = 16 * 1024; + // sess_conf.options_mut().execution.sort_spill_reservation_bytes = 2 * 1024 * 1024; // sess_conf.options_mut().catalog.default_catalog = "lakesoul".into(); - let runtime = RuntimeEnv::new(RuntimeConfig::new())?; + let mut runtime_conf = RuntimeConfig::new(); + if let Some(pool_size) = config.pool_size() { + let memory_pool = FairSpillPool::new(pool_size); + dbg!(&memory_pool); + runtime_conf = runtime_conf.with_memory_pool(Arc::new(memory_pool)); + } + let runtime = RuntimeEnv::new(runtime_conf)?; // firstly parse default fs if exist let default_fs = config @@ -507,6 +579,7 @@ pub fn create_session_context_with_planner( let normalized_prefix = register_object_store(&prefix, config, &runtime)?; config.prefix = normalized_prefix; } + debug!("{}", &config.prefix); // register object store(s) for input/output files' path // and replace file names with default fs concatenated if exist diff --git a/rust/lakesoul-io/src/lakesoul_writer.rs b/rust/lakesoul-io/src/lakesoul_writer.rs index 45ced0d56..d77667d16 100644 --- a/rust/lakesoul-io/src/lakesoul_writer.rs +++ b/rust/lakesoul-io/src/lakesoul_writer.rs @@ -2,815 +2,36 @@ // // SPDX-License-Identifier: Apache-2.0 -use crate::helpers::{columnar_values_to_partition_desc, columnar_values_to_sub_path, get_columnar_values}; -use crate::lakesoul_io_config::{create_session_context, IOSchema, LakeSoulIOConfig, LakeSoulIOConfigBuilder}; -use crate::repartition::RepartitionByRangeAndHashExec; -use crate::transform::{uniform_record_batch, uniform_schema}; +use std::borrow::Borrow; +use std::collections::HashMap; +use std::sync::Arc; -use arrow::compute::SortOptions; -use arrow::record_batch::RecordBatch; +use arrow_array::RecordBatch; use arrow_schema::SchemaRef; -use async_trait::async_trait; -use atomic_refcell::AtomicRefCell; -use datafusion::datasource::object_store::ObjectStoreUrl; -use datafusion::error::Result; -use datafusion::execution::context::TaskContext; -use datafusion::physical_expr::expressions::{col, Column}; -use datafusion::physical_expr::{PhysicalExpr, PhysicalSortExpr}; -use datafusion::physical_plan::projection::ProjectionExec; -use datafusion::physical_plan::sorts::sort::SortExec; -use datafusion::physical_plan::stream::{RecordBatchReceiverStream, RecordBatchReceiverStreamBuilder}; -use datafusion::physical_plan::{DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, SendableRecordBatchStream}; -use datafusion_common::DataFusionError::Internal; -use datafusion_common::{project_schema, DataFusionError}; -use object_store::path::Path; -use object_store::{MultipartId, ObjectStore}; -use parquet::arrow::ArrowWriter; -use parquet::basic::Compression; -use parquet::file::properties::WriterProperties; +use datafusion_common::{DataFusionError, Result}; +use parquet::format::FileMetaData; use rand::distributions::DistString; -use std::any::Any; -use std::borrow::Borrow; -use std::collections::{HashMap, VecDeque}; -use std::fmt::{Debug, Formatter}; -use std::io::ErrorKind::AddrInUse; -use std::io::Write; -use std::sync::Arc; -use tokio::io::AsyncWrite; -use tokio::io::AsyncWriteExt; use tokio::runtime::Runtime; -use tokio::sync::mpsc::Sender; use tokio::sync::Mutex; -use tokio::task::JoinHandle; -use tokio_stream::StreamExt; use tracing::debug; -use url::Url; - -#[async_trait] -pub trait AsyncBatchWriter { - async fn write_record_batch(&mut self, batch: RecordBatch) -> Result<()>; - - async fn flush_and_close(self: Box) -> Result>; - - async fn abort_and_close(self: Box) -> Result>; - - fn schema(&self) -> SchemaRef; -} - -/// An async writer using object_store's multi-part upload feature for cloud storage. -/// This writer uses a `VecDeque` as `std::io::Write` for arrow-rs's ArrowWriter. -/// Everytime when a new RowGroup is flushed, the length of the VecDeque would grow. -/// At this time, we pass the VecDeque as `bytes::Buf` to `AsyncWriteExt::write_buf` provided -/// by object_store, which would drain and copy the content of the VecDeque so that we could reuse it. -/// The `CloudMultiPartUpload` itself would try to concurrently upload parts, and -/// all parts will be committed to cloud storage by shutdown the `AsyncWrite` object. -pub struct MultiPartAsyncWriter { - in_mem_buf: InMemBuf, - task_context: Arc, - schema: SchemaRef, - writer: Box, - multi_part_id: MultipartId, - arrow_writer: ArrowWriter, - _config: LakeSoulIOConfig, - object_store: Arc, - path: Path, - absolute_path: String, - num_rows: u64, -} - -/// Wrap the above async writer with a SortExec to -/// sort the batches before write to async writer -pub struct SortAsyncWriter { - schema: SchemaRef, - sorter_sender: Sender>, - _sort_exec: Arc, - join_handle: Option>>>, - err: Option, -} - -/// Wrap the above async writer with a RepartitionExec to -/// dynamic repartitioning the batches before write to async writer -pub struct PartitioningAsyncWriter { - schema: SchemaRef, - sorter_sender: Sender>, - _partitioning_exec: Arc, - join_handle: Option>>>, - err: Option, -} - -/// A VecDeque which is both std::io::Write and bytes::Buf -#[derive(Clone)] -struct InMemBuf(Arc>>); - -impl Write for InMemBuf { - #[inline] - fn write(&mut self, buf: &[u8]) -> std::io::Result { - let mut v = self.0.try_borrow_mut().map_err(|_| std::io::Error::from(AddrInUse))?; - v.extend(buf); - Ok(buf.len()) - } - - #[inline] - fn flush(&mut self) -> std::io::Result<()> { - Ok(()) - } - - #[inline] - fn write_all(&mut self, buf: &[u8]) -> std::io::Result<()> { - let mut v = self.0.try_borrow_mut().map_err(|_| std::io::Error::from(AddrInUse))?; - v.extend(buf); - Ok(()) - } -} - -pub struct ReceiverStreamExec { - receiver_stream_builder: AtomicRefCell>, - schema: SchemaRef, -} - -impl ReceiverStreamExec { - pub fn new(receiver_stream_builder: RecordBatchReceiverStreamBuilder, schema: SchemaRef) -> Self { - Self { - receiver_stream_builder: AtomicRefCell::new(Some(receiver_stream_builder)), - schema, - } - } -} - -impl Debug for ReceiverStreamExec { - fn fmt(&self, _f: &mut Formatter<'_>) -> std::fmt::Result { - todo!() - } -} - -impl DisplayAs for ReceiverStreamExec { - fn fmt_as(&self, _t: DisplayFormatType, f: &mut std::fmt::Formatter) -> std::fmt::Result { - write!(f, "ReceiverStreamExec") - } -} - -impl ExecutionPlan for ReceiverStreamExec { - fn as_any(&self) -> &dyn Any { - self - } - - fn schema(&self) -> SchemaRef { - Arc::clone(&self.schema) - } - - fn output_partitioning(&self) -> Partitioning { - Partitioning::UnknownPartitioning(1) - } - - fn output_ordering(&self) -> Option<&[PhysicalSortExpr]> { - None - } - - fn children(&self) -> Vec> { - unimplemented!() - } - - fn with_new_children(self: Arc, _children: Vec>) -> Result> { - unimplemented!() - } - - fn execute(&self, _partition: usize, _context: Arc) -> Result { - let builder = self - .receiver_stream_builder - .borrow_mut() - .take() - .ok_or(DataFusionError::Internal("empty receiver stream".to_string()))?; - Ok(builder.build()) - } -} - -impl MultiPartAsyncWriter { - pub async fn try_new_with_context(config: &mut LakeSoulIOConfig, task_context: Arc) -> Result { - if config.files.is_empty() { - return Err(Internal("wrong number of file names provided for writer".to_string())); - } - let file_name = &config - .files - .last() - .ok_or(DataFusionError::Internal("wrong file name".to_string()))?; - - // local style path should have already been handled in create_session_context, - // so we don't have to deal with ParseError::RelativeUrlWithoutBase here - let (object_store, path) = match Url::parse(file_name.as_str()) { - Ok(url) => Ok(( - task_context - .runtime_env() - .object_store(ObjectStoreUrl::parse(&url[..url::Position::BeforePath])?)?, - Path::from_url_path(url.path())?, - )), - Err(e) => Err(DataFusionError::External(Box::new(e))), - }?; - - // get underlying multipart uploader - let (multipart_id, async_writer) = object_store.put_multipart(&path).await?; - let in_mem_buf = InMemBuf(Arc::new(AtomicRefCell::new(VecDeque::::with_capacity( - 16 * 1024 * 1024, // 16kb - )))); - let schema = uniform_schema(config.target_schema.0.clone()); - - // O(nm), n = number of fields, m = number of range partitions - let schema_projection_excluding_range = schema - .fields() - .iter() - .enumerate() - .filter_map(|(idx, field)| match config.range_partitions.contains(field.name()) { - true => None, - false => Some(idx), - }) - .collect::>(); - let writer_schema = project_schema(&schema, Some(&schema_projection_excluding_range))?; - - let arrow_writer = ArrowWriter::try_new( - in_mem_buf.clone(), - writer_schema, - Some( - WriterProperties::builder() - .set_max_row_group_size(config.max_row_group_size) - .set_write_batch_size(config.batch_size) - .set_compression(Compression::SNAPPY) - .build(), - ), - )?; - - Ok(MultiPartAsyncWriter { - in_mem_buf, - task_context, - schema, - writer: async_writer, - multi_part_id: multipart_id, - arrow_writer, - _config: config.clone(), - object_store, - path, - absolute_path: file_name.to_string(), - num_rows: 0, - }) - } - - pub async fn try_new(mut config: LakeSoulIOConfig) -> Result { - let task_context = create_session_context(&mut config)?.task_ctx(); - Self::try_new_with_context(&mut config, task_context).await - } - - async fn write_batch( - batch: RecordBatch, - arrow_writer: &mut ArrowWriter, - in_mem_buf: &mut InMemBuf, - // underlying writer - writer: &mut Box, - ) -> Result<()> { - arrow_writer.write(&batch)?; - let mut v = in_mem_buf - .0 - .try_borrow_mut() - .map_err(|e| Internal(format!("{:?}", e)))?; - if v.len() > 0 { - MultiPartAsyncWriter::write_part(writer, &mut v).await - } else { - Ok(()) - } - } - - pub async fn write_part( - writer: &mut Box, - in_mem_buf: &mut VecDeque, - ) -> Result<()> { - writer.write_all_buf(in_mem_buf).await?; - Ok(()) - } - - pub fn nun_rows(&self) -> u64 { - self.num_rows - } - - pub fn path(&self) -> Path { - self.path.clone() - } - - pub fn absolute_path(&self) -> String { - self.absolute_path.clone() - } - - pub fn task_ctx(&self) -> Arc { - self.task_context.clone() - } -} - -#[async_trait] -impl AsyncBatchWriter for MultiPartAsyncWriter { - async fn write_record_batch(&mut self, batch: RecordBatch) -> Result<()> { - let batch = uniform_record_batch(batch)?; - self.num_rows += batch.num_rows() as u64; - MultiPartAsyncWriter::write_batch(batch, &mut self.arrow_writer, &mut self.in_mem_buf, &mut self.writer).await - } - - async fn flush_and_close(self: Box) -> Result> { - // close arrow writer to flush remaining rows - let mut this = *self; - let arrow_writer = this.arrow_writer; - arrow_writer.close()?; - let mut v = this - .in_mem_buf - .0 - .try_borrow_mut() - .map_err(|e| Internal(format!("{:?}", e)))?; - if v.len() > 0 { - MultiPartAsyncWriter::write_part(&mut this.writer, &mut v).await?; - } - // shutdown multi-part async writer to complete the upload - this.writer.flush().await?; - this.writer.shutdown().await?; - Ok(vec![]) - } - - async fn abort_and_close(self: Box) -> Result> { - let this = *self; - this.object_store - .abort_multipart(&this.path, &this.multi_part_id) - .await - .map_err(DataFusionError::ObjectStore)?; - Ok(vec![]) - } - - fn schema(&self) -> SchemaRef { - self.schema.clone() - } -} - -impl SortAsyncWriter { - pub fn try_new( - async_writer: MultiPartAsyncWriter, - config: LakeSoulIOConfig, - runtime: Arc, - ) -> Result { - let _ = runtime.enter(); - let schema = config.target_schema.0.clone(); - let receiver_stream_builder = RecordBatchReceiverStream::builder(schema.clone(), 8); - let tx = receiver_stream_builder.tx(); - let recv_exec = ReceiverStreamExec::new(receiver_stream_builder, schema.clone()); - - let sort_exprs: Vec = config - .primary_keys - .iter() - // add aux sort cols to sort expr - .chain(config.aux_sort_cols.iter()) - .map(|pk| { - let col = Column::new_with_schema(pk.as_str(), &config.target_schema.0)?; - Ok(PhysicalSortExpr { - expr: Arc::new(col), - options: SortOptions::default(), - }) - }) - .collect::>>()?; - let sort_exec = Arc::new(SortExec::new(sort_exprs, Arc::new(recv_exec))); - - // see if we need to prune aux sort cols - let exec_plan: Arc = if config.aux_sort_cols.is_empty() { - sort_exec - } else { - // O(nm), n = number of target schema fields, m = number of aux sort cols - let proj_expr: Vec<(Arc, String)> = config - .target_schema - .0 - .fields - .iter() - .filter_map(|f| { - if config.aux_sort_cols.contains(f.name()) { - // exclude aux sort cols - None - } else { - Some(col(f.name().as_str(), &config.target_schema.0).map(|e| (e, f.name().clone()))) - } - }) - .collect::, String)>>>()?; - Arc::new(ProjectionExec::try_new(proj_expr, sort_exec)?) - }; - - let mut sorted_stream = exec_plan.execute(0, async_writer.task_ctx())?; - - let mut async_writer = Box::new(async_writer); - let join_handle = tokio::task::spawn(async move { - let mut err = None; - while let Some(batch) = sorted_stream.next().await { - match batch { - Ok(batch) => { - async_writer.write_record_batch(batch).await?; - } - // received abort signal - Err(e) => { - err = Some(e); - break; - } - } - } - if let Some(e) = err { - let result = async_writer.abort_and_close().await; - match result { - Ok(_) => match e { - Internal(ref err_msg) if err_msg == "external abort" => Ok(vec![]), - _ => Err(e), - }, - Err(abort_err) => Err(Internal(format!( - "Abort failed {:?}, previous error {:?}", - abort_err, e - ))), - } - } else { - async_writer.flush_and_close().await?; - Ok(vec![]) - } - }); - - Ok(SortAsyncWriter { - schema, - sorter_sender: tx, - _sort_exec: exec_plan, - join_handle: Some(join_handle), - err: None, - }) - } -} - -#[async_trait] -impl AsyncBatchWriter for SortAsyncWriter { - async fn write_record_batch(&mut self, batch: RecordBatch) -> Result<()> { - if let Some(err) = &self.err { - return Err(Internal(format!("SortAsyncWriter already failed with error {:?}", err))); - } - let send_result = self.sorter_sender.send(Ok(batch)).await; - match send_result { - Ok(_) => Ok(()), - // channel has been closed, indicating error happened during sort write - Err(e) => { - if let Some(join_handle) = self.join_handle.take() { - let result = join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))?; - self.err = result.err(); - Err(Internal(format!("Write to SortAsyncWriter failed: {:?}", self.err))) - } else { - self.err = Some(DataFusionError::External(Box::new(e))); - Err(Internal(format!("Write to SortAsyncWriter failed: {:?}", self.err))) - } - } - } - } - - async fn flush_and_close(self: Box) -> Result> { - if let Some(join_handle) = self.join_handle { - let sender = self.sorter_sender; - drop(sender); - join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))? - } else { - Err(Internal("SortAsyncWriter has been aborted, cannot flush".to_string())) - } - } - - async fn abort_and_close(self: Box) -> Result> { - if let Some(join_handle) = self.join_handle { - let sender = self.sorter_sender; - // send abort signal to the task - sender - .send(Err(Internal("external abort".to_string()))) - .await - .map_err(|e| DataFusionError::External(Box::new(e)))?; - drop(sender); - join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))? - } else { - // previous error has already aborted writer - Ok(vec![]) - } - } - - fn schema(&self) -> SchemaRef { - self.schema.clone() - } -} - -type PartitionedWriterInfo = Arc, u64)>>>; - -impl PartitioningAsyncWriter { - pub fn try_new(task_context: Arc, config: LakeSoulIOConfig, runtime: Arc) -> Result { - let _ = runtime.enter(); - let schema = config.target_schema.0.clone(); - let receiver_stream_builder = RecordBatchReceiverStream::builder(schema.clone(), 8); - let tx = receiver_stream_builder.tx(); - let recv_exec = ReceiverStreamExec::new(receiver_stream_builder, schema.clone()); - - let partitioning_exec = PartitioningAsyncWriter::get_partitioning_exec(recv_exec, config.clone())?; - - // launch one async task per *input* partition - let mut join_handles = vec![]; - - let write_id = rand::distributions::Alphanumeric.sample_string(&mut rand::thread_rng(), 16); - - let partitioned_file_path_and_row_count = Arc::new(Mutex::new(HashMap::, u64)>::new())); - for i in 0..partitioning_exec.output_partitioning().partition_count() { - let sink_task = tokio::spawn(Self::pull_and_sink( - partitioning_exec.clone(), - i, - task_context.clone(), - config.clone().into(), - Arc::new(config.range_partitions.clone()), - write_id.clone(), - partitioned_file_path_and_row_count.clone(), - )); - // // In a separate task, wait for each input to be done - // // (and pass along any errors, including panic!s) - join_handles.push(sink_task); - } - - let join_handle = tokio::spawn(Self::await_and_summary( - join_handles, - partitioned_file_path_and_row_count, - )); - - Ok(Self { - schema, - sorter_sender: tx, - _partitioning_exec: partitioning_exec, - join_handle: Some(join_handle), - err: None, - }) - } - - fn get_partitioning_exec(input: ReceiverStreamExec, config: LakeSoulIOConfig) -> Result> { - let sort_exprs: Vec = config - .range_partitions - .iter() - .chain(config.primary_keys.iter()) - // add aux sort cols to sort expr - .chain(config.aux_sort_cols.iter()) - .map(|pk| { - let col = Column::new_with_schema(pk.as_str(), &config.target_schema.0)?; - Ok(PhysicalSortExpr { - expr: Arc::new(col), - options: SortOptions::default(), - }) - }) - .collect::>>()?; - if sort_exprs.is_empty() { - return Ok(Arc::new(input)); - } - - let sort_exec = Arc::new(SortExec::new(sort_exprs, Arc::new(input))); - - // see if we need to prune aux sort cols - let sort_exec: Arc = if config.aux_sort_cols.is_empty() { - sort_exec - } else { - // O(nm), n = number of target schema fields, m = number of aux sort cols - let proj_expr: Vec<(Arc, String)> = config - .target_schema - .0 - .fields - .iter() - .filter_map(|f| { - if config.aux_sort_cols.contains(f.name()) { - // exclude aux sort cols - None - } else { - Some(col(f.name().as_str(), &config.target_schema.0).map(|e| (e, f.name().clone()))) - } - }) - .collect::, String)>>>()?; - Arc::new(ProjectionExec::try_new(proj_expr, sort_exec)?) - }; - - let exec_plan = if config.primary_keys.is_empty() && config.range_partitions.is_empty() { - sort_exec - } else { - let sorted_schema = sort_exec.schema(); - - let range_partitioning_expr: Vec> = config - .range_partitions - .iter() - .map(|col| { - let idx = sorted_schema.index_of(col.as_str())?; - Ok(Arc::new(Column::new(col.as_str(), idx)) as Arc) - }) - .collect::>>()?; - - let hash_partitioning_expr: Vec> = config - .primary_keys - .iter() - .map(|col| { - let idx = sorted_schema.index_of(col.as_str())?; - Ok(Arc::new(Column::new(col.as_str(), idx)) as Arc) - }) - .collect::>>()?; - let hash_partitioning = Partitioning::Hash(hash_partitioning_expr, config.hash_bucket_num); - - Arc::new(RepartitionByRangeAndHashExec::try_new( - sort_exec, - range_partitioning_expr, - hash_partitioning, - )?) - }; - Ok(exec_plan) - } - - async fn pull_and_sink( - input: Arc, - partition: usize, - context: Arc, - config_builder: LakeSoulIOConfigBuilder, - range_partitions: Arc>, - write_id: String, - partitioned_file_path_and_row_count: PartitionedWriterInfo, - ) -> Result { - let mut data = input.execute(partition, context.clone())?; - // O(nm), n = number of data fields, m = number of range partitions - let schema_projection_excluding_range = data - .schema() - .fields() - .iter() - .enumerate() - .filter_map(|(idx, field)| match range_partitions.contains(field.name()) { - true => None, - false => Some(idx), - }) - .collect::>(); - - let mut err = None; - - let mut row_count = 0; - - let mut partitioned_writer = HashMap::>::new(); - let mut partitioned_file_path_and_row_count_locked = partitioned_file_path_and_row_count.lock().await; - while let Some(batch_result) = data.next().await { - match batch_result { - Ok(batch) => { - debug!("write record_batch with {} rows", batch.num_rows()); - let columnar_values = get_columnar_values(&batch, range_partitions.clone())?; - let partition_desc = columnar_values_to_partition_desc(&columnar_values); - let batch_excluding_range = batch.project(&schema_projection_excluding_range)?; - let file_absolute_path = format!( - "{}{}part-{}_{:0>4}.parquet", - config_builder.prefix(), - columnar_values_to_sub_path(&columnar_values), - write_id, - partition - ); - - if !partitioned_writer.contains_key(&partition_desc) { - let mut config = config_builder.clone().with_files(vec![file_absolute_path]).build(); - - let writer = MultiPartAsyncWriter::try_new_with_context(&mut config, context.clone()).await?; - partitioned_writer.insert(partition_desc.clone(), Box::new(writer)); - } +use crate::async_writer::{AsyncBatchWriter, MultiPartAsyncWriter, PartitioningAsyncWriter, SortAsyncWriter}; +use crate::helpers::get_batch_memory_size; +use crate::lakesoul_io_config::{IOSchema, LakeSoulIOConfig}; +use crate::transform::uniform_schema; - if let Some(async_writer) = partitioned_writer.get_mut(&partition_desc) { - row_count += batch_excluding_range.num_rows(); - async_writer.write_record_batch(batch_excluding_range).await?; - } - } - // received abort signal - Err(e) => { - err = Some(e); - break; - } - } - } - if let Some(e) = err { - for (_, writer) in partitioned_writer.into_iter() { - match writer.abort_and_close().await { - Ok(_) => match e { - Internal(ref err_msg) if err_msg == "external abort" => (), - _ => return Err(e), - }, - Err(abort_err) => { - return Err(Internal(format!( - "Abort failed {:?}, previous error {:?}", - abort_err, e - ))) - } - } - } - Ok(row_count as u64) - } else { - for (partition_desc, writer) in partitioned_writer.into_iter() { - let file_absolute_path = writer.absolute_path(); - let num_rows = writer.nun_rows(); - if let Some(file_path_and_row_count) = - partitioned_file_path_and_row_count_locked.get_mut(&partition_desc) - { - file_path_and_row_count.0.push(file_absolute_path); - file_path_and_row_count.1 += num_rows; - } else { - partitioned_file_path_and_row_count_locked - .insert(partition_desc.clone(), (vec![file_absolute_path], num_rows)); - } - writer.flush_and_close().await?; - } - Ok(row_count as u64) - } - } - - async fn await_and_summary( - join_handles: Vec>>, - partitioned_file_path_and_row_count: PartitionedWriterInfo, - ) -> Result> { - let _ = - futures::future::join_all(join_handles) - .await - .iter() - .try_fold(0u64, |counter, result| match &result { - Ok(Ok(count)) => Ok(counter + count), - Ok(Err(e)) => Err(DataFusionError::Execution(format!("{}", e))), - Err(e) => Err(DataFusionError::Execution(format!("{}", e))), - })?; - let partitioned_file_path_and_row_count = partitioned_file_path_and_row_count.lock().await; - - let mut summary = format!("{}", partitioned_file_path_and_row_count.len()); - for (partition_desc, (files, _)) in partitioned_file_path_and_row_count.iter() { - summary += "\x01"; - summary += partition_desc.as_str(); - summary += "\x02"; - summary += files.join("\x02").as_str(); - } - Ok(summary.into_bytes()) - } -} - -#[async_trait] -impl AsyncBatchWriter for PartitioningAsyncWriter { - async fn write_record_batch(&mut self, batch: RecordBatch) -> Result<()> { - // arrow_cast::pretty::print_batches(&[batch.clone()]); - if let Some(err) = &self.err { - return Err(Internal(format!( - "PartitioningAsyncWriter already failed with error {:?}", - err - ))); - } - let send_result = self.sorter_sender.send(Ok(batch)).await; - match send_result { - Ok(_) => Ok(()), - // channel has been closed, indicating error happened during sort write - Err(e) => { - if let Some(join_handle) = self.join_handle.take() { - let result = join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))?; - self.err = result.err(); - Err(Internal(format!( - "Write to PartitioningAsyncWriter failed: {:?}", - self.err - ))) - } else { - self.err = Some(DataFusionError::External(Box::new(e))); - Err(Internal(format!( - "Write to PartitioningAsyncWriter failed: {:?}", - self.err - ))) - } - } - } - } - - async fn flush_and_close(self: Box) -> Result> { - if let Some(join_handle) = self.join_handle { - let sender = self.sorter_sender; - drop(sender); - join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))? - } else { - Err(Internal( - "PartitioningAsyncWriter has been aborted, cannot flush".to_string(), - )) - } - } - - async fn abort_and_close(self: Box) -> Result> { - if let Some(join_handle) = self.join_handle { - let sender = self.sorter_sender; - // send abort signal to the task - sender - .send(Err(Internal("external abort".to_string()))) - .await - .map_err(|e| DataFusionError::External(Box::new(e)))?; - drop(sender); - join_handle.await.map_err(|e| DataFusionError::External(Box::new(e)))? - } else { - // previous error has already aborted writer - Ok(vec![]) - } - } - - fn schema(&self) -> SchemaRef { - self.schema.clone() - } -} pub type SendableWriter = Box; // inner is sort writer // multipart writer pub struct SyncSendableMutableLakeSoulWriter { - inner: Arc>, runtime: Arc, schema: SchemaRef, + config: LakeSoulIOConfig, + /// The in-progress file writer if any + in_progress: Option>>, + flush_results: Vec<(String, String, FileMetaData)>, } impl SyncSendableMutableLakeSoulWriter { @@ -832,68 +53,194 @@ impl SyncSendableMutableLakeSoulWriter { } else { config.target_schema.0.clone() }; - - let mut writer_config = config.clone(); - let writer: Box = if config.use_dynamic_partition { - let task_context = create_session_context(&mut writer_config)?.task_ctx(); - Box::new(PartitioningAsyncWriter::try_new(task_context, config, runtime.clone())?) - } else if !config.primary_keys.is_empty() { - // sort primary key table - - writer_config.target_schema = IOSchema(uniform_schema(writer_schema)); - let writer = MultiPartAsyncWriter::try_new(writer_config).await?; - Box::new(SortAsyncWriter::try_new(writer, config, runtime.clone())?) - } else { - // else multipart - writer_config.target_schema = IOSchema(uniform_schema(writer_schema)); - let writer = MultiPartAsyncWriter::try_new(writer_config).await?; - Box::new(writer) - }; + let writer_config = config.clone(); + let mut config = config.clone(); + let writer = Self::create_writer(writer_schema, writer_config).await?; let schema = writer.schema(); + if let Some(mem_limit) = config.mem_limit() { + if config.use_dynamic_partition { + config.max_file_size = Some((mem_limit as f64 * 0.15) as u64); + } else if !config.primary_keys.is_empty() && !config.keep_ordering() { + config.max_file_size = Some((mem_limit as f64 * 0.2) as u64); + } + } + Ok(SyncSendableMutableLakeSoulWriter { - inner: Arc::new(Mutex::new(writer)), + in_progress: Some(Arc::new(Mutex::new(writer))), runtime, schema, // this should be the final written schema + config, + flush_results: vec![], }) }) } + async fn create_writer(writer_schema: SchemaRef, config: LakeSoulIOConfig) -> Result> { + let mut writer_config = config.clone(); + let writer : Box = if config.use_dynamic_partition { + Box::new(PartitioningAsyncWriter::try_new(writer_config)?) + } else if !writer_config.primary_keys.is_empty() && !writer_config.keep_ordering() { + // sort primary key table + writer_config.target_schema = IOSchema(uniform_schema(writer_schema)); + if writer_config.files.is_empty() && !writer_config.prefix().is_empty() { + writer_config.files = vec![format!( + "{}/part-{}_{:0>4}.parquet", + writer_config.prefix(), + rand::distributions::Alphanumeric.sample_string(&mut rand::thread_rng(), 16), + writer_config.hash_bucket_id() + )]; + } + let writer = MultiPartAsyncWriter::try_new(writer_config).await?; + Box::new(SortAsyncWriter::try_new(writer, config)?) + } else { + // else multipart + writer_config.target_schema = IOSchema(uniform_schema(writer_schema)); + if writer_config.files.is_empty() && !writer_config.prefix().is_empty() { + writer_config.files = vec![format!( + "{}/part-{}_{:0>4}.parquet", + writer_config.prefix(), + rand::distributions::Alphanumeric.sample_string(&mut rand::thread_rng(), 16), + writer_config.hash_bucket_id() + )]; + } + let writer = MultiPartAsyncWriter::try_new(writer_config).await?; + Box::new(writer) + }; + Ok(writer) + } + + pub fn schema(&self) -> SchemaRef { + self.schema.clone() + } + + pub fn config(&self) -> &LakeSoulIOConfig { + &self.config + } + // blocking method for writer record batch. // since the underlying multipart upload would accumulate buffers // and upload concurrently in background, we only need blocking method here // for ffi callers - pub fn write_batch(&self, record_batch: RecordBatch) -> Result<()> { - let inner_writer = self.inner.clone(); + pub fn write_batch(&mut self, record_batch: RecordBatch) -> Result<()> { + let runtime = self.runtime.clone(); runtime.block_on(async move { - let mut writer = inner_writer.lock().await; - writer.write_record_batch(record_batch).await + self.write_batch_async(record_batch, false).await }) } + #[async_recursion::async_recursion(?Send)] + async fn write_batch_async(&mut self, record_batch: RecordBatch, do_spill: bool) -> Result<()> { + debug!(record_batch_row=?record_batch.num_rows(), do_spill=?do_spill, "write_batch_async"); + let schema = self.schema(); + let config = self.config().clone(); + if let Some(max_file_size) = self.config().max_file_size { + // if max_file_size is set, we need to split batch into multiple files + let in_progress_writer = match &mut self.in_progress { + Some(writer) => writer, + x => + x.insert( + Arc::new(Mutex::new( + Self::create_writer(schema, config).await? + )) + ) + }; + let mut guard = in_progress_writer.lock().await; + + let batch_memory_size = get_batch_memory_size(&record_batch)? as u64; + let batch_rows = record_batch.num_rows() as u64; + // If would exceed max_file_size, split batch + if !do_spill && guard.buffered_size() + batch_memory_size > max_file_size { + let to_write = (batch_rows * (max_file_size - guard.buffered_size())) / batch_memory_size; + if to_write + 1 < batch_rows { + let to_write = to_write as usize + 1; + let a = record_batch.slice(0, to_write); + let b = record_batch.slice(to_write, record_batch.num_rows() - to_write); + drop(guard); + self.write_batch_async(a, true).await?; + return self.write_batch_async(b, false).await; + } + } + guard.write_record_batch(record_batch).await?; + + if do_spill { + dbg!(format!("spilling writer with size: {}", guard.buffered_size())); + drop(guard); + if let Some(writer) = self.in_progress.take() { + let inner_writer = match Arc::try_unwrap(writer) { + Ok(inner) => inner, + Err(_) => { + return Err(DataFusionError::Internal("Cannot get ownership of inner writer".to_string())) + }, + }; + let writer = inner_writer.into_inner(); + let results = writer.flush_and_close().await?; + self.flush_results.extend(results); + } + } + Ok(()) + } else if let Some(inner_writer) = &self.in_progress { + let inner_writer = inner_writer.clone(); + let mut writer = inner_writer.lock().await; + writer.write_record_batch(record_batch).await + } else { + Err(DataFusionError::Internal("Invalid state of inner writer".to_string())) + } + + } + pub fn flush_and_close(self) -> Result> { - let inner_writer = match Arc::try_unwrap(self.inner) { - Ok(inner) => inner, - Err(_) => return Err(Internal("Cannot get ownership of inner writer".to_string())), - }; - let runtime = self.runtime; - runtime.block_on(async move { - let writer = inner_writer.into_inner(); - writer.flush_and_close().await - }) + if let Some(inner_writer) = self.in_progress { + let inner_writer = match Arc::try_unwrap(inner_writer) { + Ok(inner) => inner, + Err(_) => return Err(DataFusionError::Internal("Cannot get ownership of inner writer".to_string())), + }; + let runtime = self.runtime; + runtime.block_on(async move { + let writer = inner_writer.into_inner(); + + let mut grouped_results: HashMap> = HashMap::new(); + let results = writer.flush_and_close().await?; + for (partition_desc, file, _) in self.flush_results.into_iter().chain(results) { + match grouped_results.get_mut(&partition_desc) { + Some(files) => { + files.push(file); + } + None => { + grouped_results.insert(partition_desc, vec![file]); + } + } + } + let mut summary = format!("{}", grouped_results.len()); + for (partition_desc, files) in grouped_results.iter() { + summary += "\x01"; + summary += partition_desc.as_str(); + summary += "\x02"; + summary += files.join("\x02").as_str(); + } + Ok(summary.into_bytes()) + + }) + } else { + Ok(vec![]) + } } - pub fn abort_and_close(self) -> Result> { - let inner_writer = match Arc::try_unwrap(self.inner) { - Ok(inner) => inner, - Err(_) => return Err(Internal("Cannot get ownership of inner writer".to_string())), - }; - let runtime = self.runtime; - runtime.block_on(async move { - let writer = inner_writer.into_inner(); - writer.abort_and_close().await - }) + pub fn abort_and_close(self) -> Result<()> { + if let Some(inner_writer) = self.in_progress { + let inner_writer = match Arc::try_unwrap(inner_writer) { + Ok(inner) => inner, + Err(_) => return Err(DataFusionError::Internal("Cannot get ownership of inner writer".to_string())), + }; + let runtime = self.runtime; + runtime.block_on(async move { + let writer = inner_writer.into_inner(); + writer.abort_and_close().await + }) + } else { + Ok(()) + } } pub fn get_schema(&self) -> SchemaRef { @@ -903,18 +250,23 @@ impl SyncSendableMutableLakeSoulWriter { #[cfg(test)] mod tests { - use crate::lakesoul_io_config::LakeSoulIOConfigBuilder; - use crate::lakesoul_reader::LakeSoulReader; - use crate::lakesoul_writer::{AsyncBatchWriter, MultiPartAsyncWriter, SyncSendableMutableLakeSoulWriter}; - use arrow::array::{ArrayRef, Int64Array}; - use arrow::record_batch::RecordBatch; - use arrow_array::Array; + use crate::{ + lakesoul_io_config::{LakeSoulIOConfigBuilder, OPTION_KEY_MEM_LIMIT}, + lakesoul_reader::LakeSoulReader, + lakesoul_writer::{AsyncBatchWriter, MultiPartAsyncWriter, SyncSendableMutableLakeSoulWriter}, + }; + use arrow::{ + array::{ArrayRef, Int64Array}, + record_batch::RecordBatch, + }; + use arrow_array::{Array, StringArray}; use arrow_schema::{DataType, Field, Schema}; use datafusion::error::Result; use parquet::arrow::arrow_reader::ParquetRecordBatchReader; - use std::fs::File; - use std::sync::Arc; - use tokio::runtime::Builder; + use rand::{distributions::DistString, Rng}; + use tracing_subscriber::layer::SubscriberExt; + use std::{fs::File, sync::Arc}; + use tokio::{runtime::Builder, time::Instant}; use super::SortAsyncWriter; @@ -970,7 +322,7 @@ mod tests { .build(); let async_writer = MultiPartAsyncWriter::try_new(writer_conf.clone()).await?; - let mut async_writer = SortAsyncWriter::try_new(async_writer, writer_conf, runtime.clone())?; + let mut async_writer = SortAsyncWriter::try_new(async_writer, writer_conf)?; async_writer.write_record_batch(to_write.clone()).await?; Box::new(async_writer).flush_and_close().await?; @@ -1021,7 +373,7 @@ mod tests { .with_aux_sort_column("col2".to_string()) .build(); - let writer = SyncSendableMutableLakeSoulWriter::try_new(writer_conf, runtime)?; + let mut writer = SyncSendableMutableLakeSoulWriter::try_new(writer_conf, runtime)?; writer.write_batch(to_write.clone())?; writer.flush_and_close()?; @@ -1123,7 +475,7 @@ mod tests { .with_schema(schema) .build(); let async_writer = MultiPartAsyncWriter::try_new(write_conf.clone()).await?; - let mut async_writer = SortAsyncWriter::try_new(async_writer, write_conf, runtime.clone())?; + let mut async_writer = SortAsyncWriter::try_new(async_writer, write_conf)?; while let Some(rb) = reader.next_rb().await { let rb = rb?; @@ -1169,7 +521,7 @@ mod tests { .with_primary_keys(vec!["str0".to_string(), "str1".to_string(), "int1".to_string()]) .build(); let async_writer = MultiPartAsyncWriter::try_new(write_conf.clone()).await?; - let mut async_writer = SortAsyncWriter::try_new(async_writer, write_conf, runtime.clone())?; + let mut async_writer = SortAsyncWriter::try_new(async_writer, write_conf)?; while let Some(rb) = reader.next_rb().await { let rb = rb?; @@ -1182,4 +534,177 @@ mod tests { Ok(()) }) } + + fn create_batch(num_columns: usize, num_rows: usize, str_len: usize) -> RecordBatch { + let mut rng = rand::thread_rng(); + let mut len_rng = rand::thread_rng(); + let iter = (0..num_columns) + .into_iter() + .map(|i| { + ( + format!("col_{}", i), + Arc::new(StringArray::from( + (0..num_rows) + .into_iter() + .map(|_| rand::distributions::Alphanumeric.sample_string(&mut rng, len_rng.gen_range(str_len..str_len * 3))) + .collect::>(), + )) as ArrayRef, + true, + ) + }) + .collect::>(); + RecordBatch::try_from_iter_with_nullable(iter).unwrap() + } + + #[test] + fn test_writer_of_large_columns() -> Result<()> { + let runtime = Builder::new_multi_thread().enable_all().build().unwrap(); + let num_batch = 39; + let num_rows = 100; + let num_columns = 2000; + let str_len = 4; + + let to_write = create_batch(num_columns, num_rows, str_len); + let temp_dir = tempfile::tempdir()?; + let path = temp_dir + .into_path() + .join("test.parquet") + .into_os_string() + .into_string() + .unwrap(); + dbg!(&path); + let writer_conf = LakeSoulIOConfigBuilder::new() + .with_files(vec![path.clone()]) + .with_thread_num(2) + .with_batch_size(num_rows) + .with_max_row_group_size(2000) + // .with_max_row_group_num_values(4_00_000) + .with_schema(to_write.schema()) + // .with_primary_keys(vec!["col".to_string()]) + // .with_aux_sort_column("col2".to_string()) + .build(); + + let mut writer = SyncSendableMutableLakeSoulWriter::try_new(writer_conf, runtime)?; + + let start = Instant::now(); + for _ in 0..num_batch { + let once_start = Instant::now(); + writer.write_batch(create_batch(num_columns, num_rows, str_len))?; + println!("write batch once cost: {}", once_start.elapsed().as_millis()); + } + let flush_start = Instant::now(); + writer.flush_and_close()?; + println!("flush cost: {}", flush_start.elapsed().as_millis()); + println!( + "num_batch={}, num_columns={}, num_rows={}, str_len={}, cost_mills={}", + num_batch, + num_columns, + num_rows, + str_len, + start.elapsed().as_millis() + ); + + let file = File::open(path.clone())?; + let mut record_batch_reader = ParquetRecordBatchReader::try_new(file, 100_000).unwrap(); + + let actual_batch = record_batch_reader + .next() + .expect("No batch found") + .expect("Unable to get batch"); + + assert_eq!(to_write.schema(), actual_batch.schema()); + assert_eq!(num_columns, actual_batch.num_columns()); + assert_eq!(num_rows * num_batch, actual_batch.num_rows()); + Ok(()) + } + + #[cfg(feature = "dhat-heap")] + #[global_allocator] + static ALLOC: dhat::Alloc = dhat::Alloc; + + + #[tracing::instrument] + #[test] + fn writer_profiling() -> Result<()> { + use tracing_subscriber::fmt; + + tracing_subscriber::fmt::init(); + + let subscriber = fmt::layer() + .event_format(fmt::format::Format::default().with_level(true).with_source_location(true).with_file(true)); + // .with_max_level(Level::TRACE); + tracing_subscriber::registry().with(subscriber); + + #[cfg(feature = "dhat-heap")] + let _profiler = dhat::Profiler::new_heap(); + + let runtime = Builder::new_multi_thread().enable_all().build().unwrap(); + let num_batch = 100; + let num_rows = 1000; + let num_columns = 100; + let str_len = 4; + + let to_write = create_batch(num_columns, num_rows, str_len); + let temp_dir = tempfile::tempdir()?; + let path = temp_dir + .into_path() + .join("test.parquet") + .into_os_string() + .into_string() + .unwrap(); + let writer_conf = LakeSoulIOConfigBuilder::new() + .with_files(vec![path.clone()]) + .with_prefix(tempfile::tempdir()?.into_path().into_os_string().into_string().unwrap()) + .with_thread_num(2) + .with_batch_size(num_rows) + // .with_max_row_group_size(2000) + // .with_max_row_group_num_values(4_00_000) + .with_schema(to_write.schema()) + .with_primary_keys( + // (0..num_columns - 1) + (0..3) + .into_iter() + .map(|i| format!("col_{}", i)) + .collect::>(), + ) + // .with_aux_sort_column("col2".to_string()) + .with_option(OPTION_KEY_MEM_LIMIT, format!("{}", 1024 * 1024 * 48)) + .set_dynamic_partition(true) + .with_hash_bucket_num(4) + // .with_max_file_size(1024 * 1024 * 32) + .build(); + + let mut writer = SyncSendableMutableLakeSoulWriter::try_new(writer_conf, runtime)?; + + let start = Instant::now(); + for _ in 0..num_batch { + // let once_start = Instant::now(); + writer.write_batch(create_batch(num_columns, num_rows, str_len))?; + // println!("write batch once cost: {}", once_start.elapsed().as_millis()); + } + let flush_start = Instant::now(); + writer.flush_and_close()?; + println!("flush cost: {}", flush_start.elapsed().as_millis()); + println!( + "num_batch={}, num_columns={}, num_rows={}, str_len={}, cost_mills={}", + num_batch, + num_columns, + num_rows, + str_len, + start.elapsed().as_millis() + ); + + // let file = File::open(path.clone())?; + // let mut record_batch_reader = ParquetRecordBatchReader::try_new(file, 100_000).unwrap(); + + // let actual_batch = record_batch_reader + // .next() + // .expect("No batch found") + // .expect("Unable to get batch"); + + // assert_eq!(to_write.schema(), actual_batch.schema()); + // assert_eq!(num_columns, actual_batch.num_columns()); + // assert_eq!(num_rows * num_batch, actual_batch.num_rows()); + Ok(()) + } } diff --git a/rust/lakesoul-io/src/lib.rs b/rust/lakesoul-io/src/lib.rs index 91a605f91..78a2cde41 100644 --- a/rust/lakesoul-io/src/lib.rs +++ b/rust/lakesoul-io/src/lib.rs @@ -2,6 +2,7 @@ // // SPDX-License-Identifier: Apache-2.0 +pub mod async_writer; pub mod datasource; pub mod filter; pub mod hash_utils; @@ -21,7 +22,6 @@ mod default_column_stream; mod transform; pub use arrow; -pub use datafusion; -pub use datafusion::arrow::error::Result; +pub use datafusion::{self, arrow::error::Result}; pub use serde_json; pub use tokio; diff --git a/rust/lakesoul-io/src/transform.rs b/rust/lakesoul-io/src/transform.rs index 60bc8d526..600e1edf8 100644 --- a/rust/lakesoul-io/src/transform.rs +++ b/rust/lakesoul-io/src/transform.rs @@ -19,7 +19,7 @@ use crate::constant::{ ARROW_CAST_OPTIONS, FLINK_TIMESTAMP_FORMAT, LAKESOUL_EMPTY_STRING, LAKESOUL_NULL_STRING, TIMESTAMP_MICROSECOND_FORMAT, TIMESTAMP_MILLSECOND_FORMAT, TIMESTAMP_NANOSECOND_FORMAT, TIMESTAMP_SECOND_FORMAT, }; -use crate::helpers::{date_str_to_epoch_days, into_scalar_value, timestamp_str_to_unix_time}; +use crate::helpers::{column_with_name_and_name2index, date_str_to_epoch_days, timestamp_str_to_unix_time, into_scalar_value}; /// adjust time zone to UTC pub fn uniform_field(orig_field: &FieldRef) -> FieldRef { @@ -83,6 +83,18 @@ pub fn transform_record_batch( ) -> Result { let num_rows = batch.num_rows(); let orig_schema = batch.schema(); + let name_to_index = + if orig_schema.fields().len() > crate::constant::NUM_COLUMN_OPTIMIZE_THRESHOLD { + Some(HashMap::::from_iter( + orig_schema + .fields() + .iter() + .enumerate() + .map(|(idx, field)| (field.name().clone(), idx)) + )) + } else { + None + }; let mut transform_arrays = Vec::new(); let mut fields = vec![]; // O(nm) n = orig_schema.fields().len(), m = target_schema.fields().len() @@ -91,7 +103,7 @@ pub fn transform_record_batch( .iter() .enumerate() .try_for_each(|(_, target_field)| -> Result<()> { - match orig_schema.column_with_name(target_field.name()) { + match column_with_name_and_name2index(&orig_schema, target_field.name(), &name_to_index) { Some((idx, _)) => { let data_type = target_field.data_type(); let transformed_array = transform_array(