cshuo commented on code in PR #18083:
URL: https://github.com/apache/hudi/pull/18083#discussion_r2985775692


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunctionWithContinuousSort.java:
##########
@@ -0,0 +1,325 @@
+/*
+ * 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.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.bulk.sort.SortOperatorGen;
+
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.memory.MemorySegment;
+import org.apache.flink.core.memory.MemorySegmentFactory;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator;
+import org.apache.flink.table.runtime.typeutils.RowDataSerializer;
+import org.apache.flink.table.runtime.generated.GeneratedNormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.GeneratedRecordComparator;
+import org.apache.flink.table.runtime.generated.NormalizedKeyComputer;
+import org.apache.flink.table.runtime.generated.RecordComparator;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.util.Collector;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * Sink function to write data with continuous sorting for improved 
compression.
+ *
+ * <p>Unlike {@link AppendWriteFunctionWithBIMBufferSort} which uses batch 
sorting,
+ * this function maintains sorted order continuously using a TreeMap, 
providing:
+ * <ul>
+ *   <li>Non-blocking inserts (O(log n) vs O(1) + periodic O(n log n))</li>
+ *   <li>Incremental draining without re-sorting</li>
+ *   <li>Predictable latency (no sort spikes)</li>
+ * </ul>
+ *
+ * <p>Strategy:
+ * <ol>
+ *   <li>Records are inserted in sorted order (TreeMap)</li>
+ *   <li>When buffer reaches max capacity, oldest record(s) are drained 
synchronously</li>
+ *   <li>Drain size is configurable to balance latency vs. throughput vs 
compression ratio</li>
+ * </ol>
+ *
+ * @param <T> Type of the input record
+ * @see StreamWriteOperatorCoordinator
+ */
+public class AppendWriteFunctionWithContinuousSort<T> extends 
AppendWriteFunction<T> {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(AppendWriteFunctionWithContinuousSort.class);
+
+  private final long maxCapacity;
+  private final int drainSize;
+
+  private transient TreeMap<SortKey, RowData> sortedRecords;
+  private transient long insertionSequence;
+
+  // Sort key computation
+  private transient NormalizedKeyComputer normalizedKeyComputer;
+  private transient RecordComparator recordComparator;
+  private transient MemorySegment reusableKeySegment;
+  private transient int normalizedKeySize;
+  private transient boolean objectReuseEnabled;
+  private transient RowDataSerializer rowDataSerializer;
+
+  // Metrics
+  private transient long totalDrainOperations;
+  private transient long totalDrainedRecords;
+  private transient long totalInserted;
+
+  public AppendWriteFunctionWithContinuousSort(Configuration config, RowType 
rowType) {
+    super(config, rowType);
+
+    // Configuration
+    this.maxCapacity = config.get(FlinkOptions.WRITE_BUFFER_SIZE);
+    this.drainSize = 
config.get(FlinkOptions.WRITE_BUFFER_SORT_CONTINUOUS_DRAIN_SIZE);
+
+    LOG.info("AppendWriteFunctionWithContinuousSort created: maxCapacity={}, 
drainSize={}",
+        maxCapacity, drainSize);
+  }
+
+  @Override
+  public void open(Configuration parameters) throws Exception {
+    // Validate configuration before calling super.open() which requires Flink 
runtime context
+    if (maxCapacity <= 0) {
+      throw new IllegalArgumentException(
+          String.format("Buffer capacity must be positive, got: %d", 
maxCapacity));
+    }
+
+    if (drainSize <= 0) {
+      throw new IllegalArgumentException(
+          String.format("Drain size must be positive, got: %d", drainSize));
+    }
+
+    // Resolve sort keys, falling back to record key if not specified
+    List<String> sortKeyList = AppendWriteFunctions.resolveSortKeys(config);
+
+    super.open(parameters);
+
+    LOG.info("Initializing continuous sort with keys: {}", sortKeyList);
+
+    // Create sort code generator for normalized key computation and record 
comparison
+    SortOperatorGen sortOperatorGen = new SortOperatorGen(rowType, 
sortKeyList.toArray(new String[0]));
+    SortCodeGenerator codeGenerator = 
sortOperatorGen.createSortCodeGenerator();
+    GeneratedNormalizedKeyComputer generatedKeyComputer = 
codeGenerator.generateNormalizedKeyComputer("ContinuousSortKeyComputer");
+    GeneratedRecordComparator generatedComparator = 
codeGenerator.generateRecordComparator("ContinuousSortComparator");
+
+    // Instantiate code-generated components
+    ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+    this.normalizedKeyComputer = generatedKeyComputer.newInstance(classLoader);
+    this.recordComparator = generatedComparator.newInstance(classLoader);
+    this.normalizedKeySize = normalizedKeyComputer.getNumKeyBytes();
+
+    // Initialize TreeMap with comparator that uses normalized keys for fast 
comparison
+    // and falls back to RecordComparator for full comparison when normalized 
keys are equal
+    this.sortedRecords = new TreeMap<>((k1, k2) -> {
+      int cmp = normalizedKeyComputer.compareKey(k1.keySegment, 0, 
k2.keySegment, 0);
+      if (cmp != 0) {
+        return cmp;
+      }
+      // Normalized keys are equal - use full record comparison for correct 
ordering
+      cmp = recordComparator.compare(k1.record, k2.record);
+      if (cmp != 0) {
+        return cmp;
+      }
+      // Records are equal by sort keys - use insertion order for stability
+      return Long.compare(k1.insertionOrder, k2.insertionOrder);
+    });
+    this.insertionSequence = 0L;
+
+    // Allocate reusable on-heap buffer for computing keys
+    byte[] reusableKeyBuffer = new byte[normalizedKeySize];
+    this.reusableKeySegment = MemorySegmentFactory.wrap(reusableKeyBuffer);
+
+    // Detect object reuse mode and create serializer for copying if needed
+    this.objectReuseEnabled = 
getRuntimeContext().getExecutionConfig().isObjectReuseEnabled();

Review Comment:
   There is some compile error for higher flink versions.
   
   ```
   Caused by: org.apache.maven.plugin.compiler.CompilationFailureException: 
Compilation failure
   
/home/runner/work/hudi/hudi/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/append/AppendWriteFunctionWithContinuousSort.java:[158,50]
 cannot find symbol
     symbol:   method getExecutionConfig()
   ```
   You can refer to `RuntimeContextUtils` to adapt object reuse check for 
different flink versions.



-- 
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]

Reply via email to