cshuo commented on code in PR #17864: URL: https://github.com/apache/hudi/pull/17864#discussion_r2690116921
########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunctionWithDisruptorBufferSort.java: ########## @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.sink.append; + +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.queue.DisruptorMessageQueue; +import org.apache.hudi.common.util.queue.HoodieConsumer; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.sink.StreamWriteOperatorCoordinator; +import org.apache.hudi.sink.buffer.BufferType; +import org.apache.hudi.sink.buffer.MemorySegmentPoolFactory; +import org.apache.hudi.sink.bulk.sort.SortOperatorGen; +import org.apache.hudi.sink.utils.BufferUtils; +import org.apache.hudi.util.MutableIteratorWrapperIterator; + +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.sort.QuickSort; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator; +import org.apache.flink.table.runtime.generated.GeneratedNormalizedKeyComputer; +import org.apache.flink.table.runtime.generated.GeneratedRecordComparator; +import org.apache.flink.table.runtime.operators.sort.BinaryInMemorySortBuffer; +import org.apache.flink.table.runtime.util.MemorySegmentPool; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Sink function to write the data to the underneath filesystem using LMAX Disruptor + * as a lock-free ring buffer for better throughput. + * + * <p>Uses Flink's native {@link BinaryInMemorySortBuffer} with code-generated comparators + * for efficient sorting. + * + * <p>The function writes base files directly for each checkpoint, + * the file may roll over when its size hits the configured threshold. + * + * @param <T> Type of the input record + * @see StreamWriteOperatorCoordinator + * @see BufferType#DISRUPTOR + */ +@Slf4j +public class AppendWriteFunctionWithDisruptorBufferSort<T> extends AppendWriteFunction<T> { + + // writeBufferSize: record count threshold for flushing sort buffer to disk + private final long writeBufferSize; + // ringBufferSize: Disruptor ring buffer capacity (queue between producer and consumer threads) + private final int ringBufferSize; + private final String waitStrategy; + + private transient MemorySegmentPool memorySegmentPool; + private transient GeneratedNormalizedKeyComputer keyComputer; + private transient GeneratedRecordComparator recordComparator; + private transient DisruptorMessageQueue<RowData, RowData> disruptorQueue; + private transient BinaryInMemorySortBuffer sortBuffer; + private transient SortingConsumer sortingConsumer; + + public AppendWriteFunctionWithDisruptorBufferSort(Configuration config, RowType rowType) { + super(config, rowType); + this.writeBufferSize = config.get(FlinkOptions.WRITE_BUFFER_SIZE); + this.ringBufferSize = config.get(FlinkOptions.WRITE_BUFFER_DISRUPTOR_RING_SIZE); + this.waitStrategy = config.get(FlinkOptions.WRITE_BUFFER_DISRUPTOR_WAIT_STRATEGY); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + // Resolve sort keys (defaults to record key if not specified) + String sortKeys = AppendWriteFunctions.resolveSortKeys(config); + if (StringUtils.isNullOrEmpty(sortKeys)) { + throw new IllegalArgumentException("Sort keys can't be null or empty for append write with disruptor sort. " + + "Either set write.buffer.sort.keys or ensure record key field is configured."); + } + + List<String> sortKeyList = Arrays.stream(sortKeys.split(",")) + .map(String::trim) + .collect(Collectors.toList()); + + // Create Flink-native sort components + SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, sortKeyList.toArray(new String[0])); + SortCodeGenerator codeGenerator = sortOperatorGen.createSortCodeGenerator(); + this.keyComputer = codeGenerator.generateNormalizedKeyComputer("SortComputer"); + this.recordComparator = codeGenerator.generateRecordComparator("SortComparator"); + this.memorySegmentPool = MemorySegmentPoolFactory.createMemorySegmentPool(config); + + initDisruptorBuffer(); + + log.info("{} initialized with disruptor buffer, sort keys: {}, ring size: {}", + getClass().getSimpleName(), sortKeys, ringBufferSize); + } + + private void initDisruptorBuffer() throws Exception { + this.sortBuffer = BufferUtils.createBuffer(rowType, + memorySegmentPool, + keyComputer.newInstance(Thread.currentThread().getContextClassLoader()), + recordComparator.newInstance(Thread.currentThread().getContextClassLoader())); + + this.sortingConsumer = new SortingConsumer(); + + this.disruptorQueue = new DisruptorMessageQueue<>( + ringBufferSize, + Function.identity(), + waitStrategy, + 1, // single producer (Flink operator thread) + null + ); + disruptorQueue.setHandlers(sortingConsumer); + disruptorQueue.start(); + } + + @Override + public void processElement(T value, Context ctx, Collector<RowData> out) throws Exception { + disruptorQueue.insertRecord((RowData) value); + } + + @Override + public void snapshotState() { + try { + flushDisruptor(); + reinitDisruptorAfterCheckpoint(); + } catch (Exception e) { + throw new HoodieException("Fail to flush data during snapshot state.", e); + } + super.snapshotState(); + } + + @Override + public void endInput() { + try { + flushDisruptor(); + } catch (Exception e) { + throw new HoodieException("Fail to flush data during endInput.", e); + } + super.endInput(); + } + + private void flushDisruptor() { + disruptorQueue.close(); + // Check if any errors occurred during event processing + Throwable error = disruptorQueue.getThrowable(); + if (error != null) { + throw new HoodieException("Error processing records in disruptor buffer", error); + } + sortingConsumer.finish(); + } + + private void reinitDisruptorAfterCheckpoint() throws Exception { + this.sortBuffer = BufferUtils.createBuffer(rowType, + memorySegmentPool, + keyComputer.newInstance(Thread.currentThread().getContextClassLoader()), + recordComparator.newInstance(Thread.currentThread().getContextClassLoader())); + + this.sortingConsumer = new SortingConsumer(); + + this.disruptorQueue = new DisruptorMessageQueue<>( + ringBufferSize, + Function.identity(), + waitStrategy, + 1, + null + ); + disruptorQueue.setHandlers(sortingConsumer); + disruptorQueue.start(); + } + + private void sortAndSend(BinaryInMemorySortBuffer buffer) throws IOException { + if (buffer.isEmpty()) { + return; + } + if (this.writerHelper == null) { + initWriterHelper(); + } + new QuickSort().sort(buffer); + Iterator<BinaryRowData> iterator = new MutableIteratorWrapperIterator<>( + buffer.getIterator(), + () -> new BinaryRowData(rowType.getFieldCount())); + while (iterator.hasNext()) { + writerHelper.write(iterator.next()); + } + buffer.reset(); + } + + @Override + public void close() throws Exception { + try { + if (disruptorQueue != null) { + disruptorQueue.close(); + } + } finally { + super.close(); + } + } + + /** + * Consumer for disruptor buffer that uses Flink's native sorting. + * Accumulates records into a sort buffer and flushes when full. + */ + private class SortingConsumer implements HoodieConsumer<RowData, Void> { + + @Override + public void consume(RowData record) throws Exception { + boolean success = sortBuffer.write(record); + if (!success) { + flushSortBuffer(); + success = sortBuffer.write(record); + if (!success) { + throw new HoodieException("Sort buffer is too small to hold a single record."); + } + } + + if (sortBuffer.size() >= writeBufferSize) { + flushSortBuffer(); + } + } + + private void flushSortBuffer() throws IOException { Review Comment: `flushSortBuffer` seems not needed, use `sortAndSend(sortBuffer)` directly? ########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunctionWithDisruptorBufferSort.java: ########## @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.sink.append; + +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.queue.DisruptorMessageQueue; +import org.apache.hudi.common.util.queue.HoodieConsumer; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.sink.StreamWriteOperatorCoordinator; +import org.apache.hudi.sink.buffer.BufferType; +import org.apache.hudi.sink.buffer.MemorySegmentPoolFactory; +import org.apache.hudi.sink.bulk.sort.SortOperatorGen; +import org.apache.hudi.sink.utils.BufferUtils; +import org.apache.hudi.util.MutableIteratorWrapperIterator; + +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.sort.QuickSort; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator; +import org.apache.flink.table.runtime.generated.GeneratedNormalizedKeyComputer; +import org.apache.flink.table.runtime.generated.GeneratedRecordComparator; +import org.apache.flink.table.runtime.operators.sort.BinaryInMemorySortBuffer; +import org.apache.flink.table.runtime.util.MemorySegmentPool; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Sink function to write the data to the underneath filesystem using LMAX Disruptor + * as a lock-free ring buffer for better throughput. + * + * <p>Uses Flink's native {@link BinaryInMemorySortBuffer} with code-generated comparators + * for efficient sorting. + * + * <p>The function writes base files directly for each checkpoint, + * the file may roll over when its size hits the configured threshold. + * + * @param <T> Type of the input record + * @see StreamWriteOperatorCoordinator + * @see BufferType#DISRUPTOR + */ +@Slf4j +public class AppendWriteFunctionWithDisruptorBufferSort<T> extends AppendWriteFunction<T> { + + // writeBufferSize: record count threshold for flushing sort buffer to disk + private final long writeBufferSize; + // ringBufferSize: Disruptor ring buffer capacity (queue between producer and consumer threads) + private final int ringBufferSize; + private final String waitStrategy; + + private transient MemorySegmentPool memorySegmentPool; + private transient GeneratedNormalizedKeyComputer keyComputer; + private transient GeneratedRecordComparator recordComparator; + private transient DisruptorMessageQueue<RowData, RowData> disruptorQueue; + private transient BinaryInMemorySortBuffer sortBuffer; + private transient SortingConsumer sortingConsumer; + + public AppendWriteFunctionWithDisruptorBufferSort(Configuration config, RowType rowType) { + super(config, rowType); + this.writeBufferSize = config.get(FlinkOptions.WRITE_BUFFER_SIZE); + this.ringBufferSize = config.get(FlinkOptions.WRITE_BUFFER_DISRUPTOR_RING_SIZE); Review Comment: validate ring buffer size is a power of 2, which is required by Disruptor. ########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunctionWithDisruptorBufferSort.java: ########## @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.sink.append; + +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.queue.DisruptorMessageQueue; +import org.apache.hudi.common.util.queue.HoodieConsumer; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.sink.StreamWriteOperatorCoordinator; +import org.apache.hudi.sink.buffer.BufferType; +import org.apache.hudi.sink.buffer.MemorySegmentPoolFactory; +import org.apache.hudi.sink.bulk.sort.SortOperatorGen; +import org.apache.hudi.sink.utils.BufferUtils; +import org.apache.hudi.util.MutableIteratorWrapperIterator; + +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.sort.QuickSort; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator; +import org.apache.flink.table.runtime.generated.GeneratedNormalizedKeyComputer; +import org.apache.flink.table.runtime.generated.GeneratedRecordComparator; +import org.apache.flink.table.runtime.operators.sort.BinaryInMemorySortBuffer; +import org.apache.flink.table.runtime.util.MemorySegmentPool; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Sink function to write the data to the underneath filesystem using LMAX Disruptor + * as a lock-free ring buffer for better throughput. + * + * <p>Uses Flink's native {@link BinaryInMemorySortBuffer} with code-generated comparators + * for efficient sorting. + * + * <p>The function writes base files directly for each checkpoint, + * the file may roll over when its size hits the configured threshold. + * + * @param <T> Type of the input record + * @see StreamWriteOperatorCoordinator + * @see BufferType#DISRUPTOR + */ +@Slf4j +public class AppendWriteFunctionWithDisruptorBufferSort<T> extends AppendWriteFunction<T> { + + // writeBufferSize: record count threshold for flushing sort buffer to disk + private final long writeBufferSize; + // ringBufferSize: Disruptor ring buffer capacity (queue between producer and consumer threads) + private final int ringBufferSize; + private final String waitStrategy; + + private transient MemorySegmentPool memorySegmentPool; + private transient GeneratedNormalizedKeyComputer keyComputer; + private transient GeneratedRecordComparator recordComparator; + private transient DisruptorMessageQueue<RowData, RowData> disruptorQueue; + private transient BinaryInMemorySortBuffer sortBuffer; + private transient SortingConsumer sortingConsumer; + + public AppendWriteFunctionWithDisruptorBufferSort(Configuration config, RowType rowType) { + super(config, rowType); + this.writeBufferSize = config.get(FlinkOptions.WRITE_BUFFER_SIZE); + this.ringBufferSize = config.get(FlinkOptions.WRITE_BUFFER_DISRUPTOR_RING_SIZE); + this.waitStrategy = config.get(FlinkOptions.WRITE_BUFFER_DISRUPTOR_WAIT_STRATEGY); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + // Resolve sort keys (defaults to record key if not specified) + String sortKeys = AppendWriteFunctions.resolveSortKeys(config); + if (StringUtils.isNullOrEmpty(sortKeys)) { + throw new IllegalArgumentException("Sort keys can't be null or empty for append write with disruptor sort. " + + "Either set write.buffer.sort.keys or ensure record key field is configured."); + } + + List<String> sortKeyList = Arrays.stream(sortKeys.split(",")) + .map(String::trim) + .collect(Collectors.toList()); + + // Create Flink-native sort components + SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, sortKeyList.toArray(new String[0])); + SortCodeGenerator codeGenerator = sortOperatorGen.createSortCodeGenerator(); + this.keyComputer = codeGenerator.generateNormalizedKeyComputer("SortComputer"); + this.recordComparator = codeGenerator.generateRecordComparator("SortComparator"); + this.memorySegmentPool = MemorySegmentPoolFactory.createMemorySegmentPool(config); + + initDisruptorBuffer(); + + log.info("{} initialized with disruptor buffer, sort keys: {}, ring size: {}", + getClass().getSimpleName(), sortKeys, ringBufferSize); + } + + private void initDisruptorBuffer() throws Exception { + this.sortBuffer = BufferUtils.createBuffer(rowType, + memorySegmentPool, + keyComputer.newInstance(Thread.currentThread().getContextClassLoader()), + recordComparator.newInstance(Thread.currentThread().getContextClassLoader())); + + this.sortingConsumer = new SortingConsumer(); + + this.disruptorQueue = new DisruptorMessageQueue<>( + ringBufferSize, + Function.identity(), + waitStrategy, + 1, // single producer (Flink operator thread) + null + ); + disruptorQueue.setHandlers(sortingConsumer); + disruptorQueue.start(); + } + + @Override + public void processElement(T value, Context ctx, Collector<RowData> out) throws Exception { + disruptorQueue.insertRecord((RowData) value); + } + + @Override + public void snapshotState() { + try { + flushDisruptor(); + reinitDisruptorAfterCheckpoint(); Review Comment: is it possible to reuse the `disruptorQueue`. `sortBuffer` can be reused actually by invoke `sortBuffer#reset()`. ########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunctionWithDisruptorBufferSort.java: ########## @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.sink.append; + +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.queue.DisruptorMessageQueue; +import org.apache.hudi.common.util.queue.HoodieConsumer; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.sink.StreamWriteOperatorCoordinator; +import org.apache.hudi.sink.buffer.BufferType; +import org.apache.hudi.sink.buffer.MemorySegmentPoolFactory; +import org.apache.hudi.sink.bulk.sort.SortOperatorGen; +import org.apache.hudi.sink.utils.BufferUtils; +import org.apache.hudi.util.MutableIteratorWrapperIterator; + +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.sort.QuickSort; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator; +import org.apache.flink.table.runtime.generated.GeneratedNormalizedKeyComputer; +import org.apache.flink.table.runtime.generated.GeneratedRecordComparator; +import org.apache.flink.table.runtime.operators.sort.BinaryInMemorySortBuffer; +import org.apache.flink.table.runtime.util.MemorySegmentPool; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Sink function to write the data to the underneath filesystem using LMAX Disruptor + * as a lock-free ring buffer for better throughput. + * + * <p>Uses Flink's native {@link BinaryInMemorySortBuffer} with code-generated comparators + * for efficient sorting. + * + * <p>The function writes base files directly for each checkpoint, + * the file may roll over when its size hits the configured threshold. + * + * @param <T> Type of the input record + * @see StreamWriteOperatorCoordinator + * @see BufferType#DISRUPTOR + */ +@Slf4j +public class AppendWriteFunctionWithDisruptorBufferSort<T> extends AppendWriteFunction<T> { + + // writeBufferSize: record count threshold for flushing sort buffer to disk + private final long writeBufferSize; + // ringBufferSize: Disruptor ring buffer capacity (queue between producer and consumer threads) + private final int ringBufferSize; + private final String waitStrategy; + + private transient MemorySegmentPool memorySegmentPool; + private transient GeneratedNormalizedKeyComputer keyComputer; + private transient GeneratedRecordComparator recordComparator; + private transient DisruptorMessageQueue<RowData, RowData> disruptorQueue; + private transient BinaryInMemorySortBuffer sortBuffer; + private transient SortingConsumer sortingConsumer; + + public AppendWriteFunctionWithDisruptorBufferSort(Configuration config, RowType rowType) { + super(config, rowType); + this.writeBufferSize = config.get(FlinkOptions.WRITE_BUFFER_SIZE); + this.ringBufferSize = config.get(FlinkOptions.WRITE_BUFFER_DISRUPTOR_RING_SIZE); + this.waitStrategy = config.get(FlinkOptions.WRITE_BUFFER_DISRUPTOR_WAIT_STRATEGY); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + // Resolve sort keys (defaults to record key if not specified) + String sortKeys = AppendWriteFunctions.resolveSortKeys(config); + if (StringUtils.isNullOrEmpty(sortKeys)) { + throw new IllegalArgumentException("Sort keys can't be null or empty for append write with disruptor sort. " + + "Either set write.buffer.sort.keys or ensure record key field is configured."); + } + + List<String> sortKeyList = Arrays.stream(sortKeys.split(",")) + .map(String::trim) + .collect(Collectors.toList()); + + // Create Flink-native sort components + SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, sortKeyList.toArray(new String[0])); + SortCodeGenerator codeGenerator = sortOperatorGen.createSortCodeGenerator(); + this.keyComputer = codeGenerator.generateNormalizedKeyComputer("SortComputer"); + this.recordComparator = codeGenerator.generateRecordComparator("SortComparator"); + this.memorySegmentPool = MemorySegmentPoolFactory.createMemorySegmentPool(config); + + initDisruptorBuffer(); + + log.info("{} initialized with disruptor buffer, sort keys: {}, ring size: {}", + getClass().getSimpleName(), sortKeys, ringBufferSize); + } + + private void initDisruptorBuffer() throws Exception { + this.sortBuffer = BufferUtils.createBuffer(rowType, + memorySegmentPool, + keyComputer.newInstance(Thread.currentThread().getContextClassLoader()), + recordComparator.newInstance(Thread.currentThread().getContextClassLoader())); + + this.sortingConsumer = new SortingConsumer(); + + this.disruptorQueue = new DisruptorMessageQueue<>( + ringBufferSize, + Function.identity(), + waitStrategy, + 1, // single producer (Flink operator thread) + null + ); + disruptorQueue.setHandlers(sortingConsumer); + disruptorQueue.start(); + } + + @Override + public void processElement(T value, Context ctx, Collector<RowData> out) throws Exception { + disruptorQueue.insertRecord((RowData) value); + } + + @Override + public void snapshotState() { + try { + flushDisruptor(); + reinitDisruptorAfterCheckpoint(); + } catch (Exception e) { + throw new HoodieException("Fail to flush data during snapshot state.", e); + } + super.snapshotState(); + } + + @Override + public void endInput() { + try { + flushDisruptor(); + } catch (Exception e) { + throw new HoodieException("Fail to flush data during endInput.", e); + } + super.endInput(); + } + + private void flushDisruptor() { + disruptorQueue.close(); Review Comment: `DisruptorMessageQueue#close` will wait for a fixed duration of `TIMEOUT_WAITING_SECS` (10) seconds before all events being processed. When the writer is slow, timeout exception is likely to occur then, is it possible to configure the wait time? ########## hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/append/ITTestAppendWriteFunctionWithBIMBufferSort.java: ########## @@ -52,9 +52,9 @@ import static org.junit.jupiter.api.Assertions.assertTrue; /** - * Test cases for {@link AppendWriteFunctionWithBufferSort}. + * Test cases for {@link AppendWriteFunctionWithBIMBufferSort} (BOUNDED_IN_MEMORY buffer type). */ -public class ITTestAppendWriteFunctionWithBufferSort extends TestWriteBase { +public class ITTestAppendWriteFunctionWithBIMBufferSort extends TestWriteBase { Review Comment: -> TestAppendWriteFunctionWithBIMBufferSort ########## hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/append/ITTestAppendWriteFunctionWithDisruptorBufferSort.java: ########## @@ -0,0 +1,292 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the "License"); + * you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.sink.append; + +import org.apache.hudi.sink.buffer.BufferType; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.sink.utils.TestWriteBase; +import org.apache.hudi.utils.TestConfigurations; +import org.apache.hudi.utils.TestData; + +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.VarCharType; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Test cases for {@link AppendWriteFunctionWithDisruptorBufferSort} (DISRUPTOR buffer type). + */ +public class ITTestAppendWriteFunctionWithDisruptorBufferSort extends TestWriteBase { Review Comment: can we combine `ITTestAppendWriteFunctionWithDisruptorBufferSort` and `ITTestAppendWriteFunctionWithBIMBufferSort` into one, like `TestAppendWriteFunctionWithBufferSort`, and update test cases using `ParameterizedTest` to cover these two types write functions. ########## hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunctionWithDisruptorBufferSort.java: ########## @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.sink.append; + +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.queue.DisruptorMessageQueue; +import org.apache.hudi.common.util.queue.HoodieConsumer; +import org.apache.hudi.configuration.FlinkOptions; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.sink.StreamWriteOperatorCoordinator; +import org.apache.hudi.sink.buffer.BufferType; +import org.apache.hudi.sink.buffer.MemorySegmentPoolFactory; +import org.apache.hudi.sink.bulk.sort.SortOperatorGen; +import org.apache.hudi.sink.utils.BufferUtils; +import org.apache.hudi.util.MutableIteratorWrapperIterator; + +import lombok.extern.slf4j.Slf4j; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.operators.sort.QuickSort; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.binary.BinaryRowData; +import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator; +import org.apache.flink.table.runtime.generated.GeneratedNormalizedKeyComputer; +import org.apache.flink.table.runtime.generated.GeneratedRecordComparator; +import org.apache.flink.table.runtime.operators.sort.BinaryInMemorySortBuffer; +import org.apache.flink.table.runtime.util.MemorySegmentPool; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.util.Collector; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.function.Function; +import java.util.stream.Collectors; + +/** + * Sink function to write the data to the underneath filesystem using LMAX Disruptor + * as a lock-free ring buffer for better throughput. + * + * <p>Uses Flink's native {@link BinaryInMemorySortBuffer} with code-generated comparators + * for efficient sorting. + * + * <p>The function writes base files directly for each checkpoint, + * the file may roll over when its size hits the configured threshold. + * + * @param <T> Type of the input record + * @see StreamWriteOperatorCoordinator + * @see BufferType#DISRUPTOR + */ +@Slf4j +public class AppendWriteFunctionWithDisruptorBufferSort<T> extends AppendWriteFunction<T> { + + // writeBufferSize: record count threshold for flushing sort buffer to disk + private final long writeBufferSize; + // ringBufferSize: Disruptor ring buffer capacity (queue between producer and consumer threads) + private final int ringBufferSize; + private final String waitStrategy; + + private transient MemorySegmentPool memorySegmentPool; + private transient GeneratedNormalizedKeyComputer keyComputer; + private transient GeneratedRecordComparator recordComparator; + private transient DisruptorMessageQueue<RowData, RowData> disruptorQueue; + private transient BinaryInMemorySortBuffer sortBuffer; + private transient SortingConsumer sortingConsumer; + + public AppendWriteFunctionWithDisruptorBufferSort(Configuration config, RowType rowType) { + super(config, rowType); + this.writeBufferSize = config.get(FlinkOptions.WRITE_BUFFER_SIZE); + this.ringBufferSize = config.get(FlinkOptions.WRITE_BUFFER_DISRUPTOR_RING_SIZE); + this.waitStrategy = config.get(FlinkOptions.WRITE_BUFFER_DISRUPTOR_WAIT_STRATEGY); + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + // Resolve sort keys (defaults to record key if not specified) + String sortKeys = AppendWriteFunctions.resolveSortKeys(config); + if (StringUtils.isNullOrEmpty(sortKeys)) { + throw new IllegalArgumentException("Sort keys can't be null or empty for append write with disruptor sort. " + + "Either set write.buffer.sort.keys or ensure record key field is configured."); + } + + List<String> sortKeyList = Arrays.stream(sortKeys.split(",")) + .map(String::trim) + .collect(Collectors.toList()); + + // Create Flink-native sort components + SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, sortKeyList.toArray(new String[0])); + SortCodeGenerator codeGenerator = sortOperatorGen.createSortCodeGenerator(); + this.keyComputer = codeGenerator.generateNormalizedKeyComputer("SortComputer"); + this.recordComparator = codeGenerator.generateRecordComparator("SortComparator"); + this.memorySegmentPool = MemorySegmentPoolFactory.createMemorySegmentPool(config); + + initDisruptorBuffer(); + + log.info("{} initialized with disruptor buffer, sort keys: {}, ring size: {}", + getClass().getSimpleName(), sortKeys, ringBufferSize); + } + + private void initDisruptorBuffer() throws Exception { + this.sortBuffer = BufferUtils.createBuffer(rowType, + memorySegmentPool, + keyComputer.newInstance(Thread.currentThread().getContextClassLoader()), + recordComparator.newInstance(Thread.currentThread().getContextClassLoader())); + + this.sortingConsumer = new SortingConsumer(); + + this.disruptorQueue = new DisruptorMessageQueue<>( + ringBufferSize, + Function.identity(), + waitStrategy, + 1, // single producer (Flink operator thread) + null + ); + disruptorQueue.setHandlers(sortingConsumer); + disruptorQueue.start(); + } + + @Override + public void processElement(T value, Context ctx, Collector<RowData> out) throws Exception { + disruptorQueue.insertRecord((RowData) value); + } + + @Override + public void snapshotState() { + try { + flushDisruptor(); + reinitDisruptorAfterCheckpoint(); + } catch (Exception e) { + throw new HoodieException("Fail to flush data during snapshot state.", e); + } + super.snapshotState(); + } + + @Override + public void endInput() { + try { + flushDisruptor(); + } catch (Exception e) { + throw new HoodieException("Fail to flush data during endInput.", e); + } + super.endInput(); + } + + private void flushDisruptor() { + disruptorQueue.close(); + // Check if any errors occurred during event processing + Throwable error = disruptorQueue.getThrowable(); + if (error != null) { + throw new HoodieException("Error processing records in disruptor buffer", error); + } + sortingConsumer.finish(); + } + + private void reinitDisruptorAfterCheckpoint() throws Exception { Review Comment: duplicate code, same as `initDisruptorBuffer` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
