diff --git a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java index afe4c50208f7..abc161608730 100644 --- a/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java +++ b/paimon-common/src/main/java/org/apache/paimon/CoreOptions.java @@ -3226,4 +3226,37 @@ public String toString() { return value; } } + + /** The order type of table sort. */ + public enum OrderType { + ORDER("order"), + ZORDER("zorder"), + HILBERT("hilbert"), + NONE("none"); + + private final String orderType; + + OrderType(String orderType) { + this.orderType = orderType; + } + + @Override + public String toString() { + return "order type: " + orderType; + } + + public static OrderType of(String orderType) { + if (ORDER.orderType.equalsIgnoreCase(orderType)) { + return ORDER; + } else if (ZORDER.orderType.equalsIgnoreCase(orderType)) { + return ZORDER; + } else if (HILBERT.orderType.equalsIgnoreCase(orderType)) { + return HILBERT; + } else if (NONE.orderType.equalsIgnoreCase(orderType)) { + return NONE; + } + + throw new IllegalArgumentException("cannot match type: " + orderType + " for ordering"); + } + } } diff --git a/paimon-core/src/main/java/org/apache/paimon/sort/zorder/ZIndexer.java b/paimon-common/src/main/java/org/apache/paimon/sort/zorder/ZIndexer.java similarity index 98% rename from paimon-core/src/main/java/org/apache/paimon/sort/zorder/ZIndexer.java rename to paimon-common/src/main/java/org/apache/paimon/sort/zorder/ZIndexer.java index 14af2a95a7b2..c06ba8fa0632 100644 --- a/paimon-core/src/main/java/org/apache/paimon/sort/zorder/ZIndexer.java +++ b/paimon-common/src/main/java/org/apache/paimon/sort/zorder/ZIndexer.java @@ -47,7 +47,6 @@ import org.apache.paimon.types.VarBinaryType; import org.apache.paimon.types.VarCharType; import org.apache.paimon.types.VariantType; -import org.apache.paimon.utils.ZOrderByteUtils; import java.io.Serializable; import java.nio.ByteBuffer; @@ -57,8 +56,8 @@ import java.util.Set; import java.util.function.BiFunction; -import static org.apache.paimon.utils.ZOrderByteUtils.NULL_BYTES; -import static org.apache.paimon.utils.ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE; +import static org.apache.paimon.sort.zorder.ZOrderByteUtils.NULL_BYTES; +import static org.apache.paimon.sort.zorder.ZOrderByteUtils.PRIMITIVE_BUFFER_SIZE; /** Z-indexer for responsibility to generate z-index. */ public class ZIndexer implements Serializable { diff --git a/paimon-core/src/main/java/org/apache/paimon/utils/ZOrderByteUtils.java b/paimon-common/src/main/java/org/apache/paimon/sort/zorder/ZOrderByteUtils.java similarity index 99% rename from paimon-core/src/main/java/org/apache/paimon/utils/ZOrderByteUtils.java rename to paimon-common/src/main/java/org/apache/paimon/sort/zorder/ZOrderByteUtils.java index 3baff77eded6..04bf863a3f65 100644 --- a/paimon-core/src/main/java/org/apache/paimon/utils/ZOrderByteUtils.java +++ b/paimon-common/src/main/java/org/apache/paimon/sort/zorder/ZOrderByteUtils.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.paimon.utils; +package org.apache.paimon.sort.zorder; import java.nio.ByteBuffer; import java.nio.CharBuffer; diff --git a/paimon-core/src/test/java/org/apache/paimon/utils/TestZOrderByteUtil.java b/paimon-common/src/test/java/org/apache/paimon/sort/zorder/TestZOrderByteUtil.java similarity index 99% rename from paimon-core/src/test/java/org/apache/paimon/utils/TestZOrderByteUtil.java rename to paimon-common/src/test/java/org/apache/paimon/sort/zorder/TestZOrderByteUtil.java index 92786b701c4f..72ffc1712159 100644 --- a/paimon-core/src/test/java/org/apache/paimon/utils/TestZOrderByteUtil.java +++ b/paimon-common/src/test/java/org/apache/paimon/sort/zorder/TestZOrderByteUtil.java @@ -16,7 +16,7 @@ * limitations under the License. */ -package org.apache.paimon.utils; +package org.apache.paimon.sort.zorder; import org.junit.Assert; import org.junit.Test; diff --git a/paimon-core/src/test/java/org/apache/paimon/sort/zorder/ZIndexerTest.java b/paimon-core/src/test/java/org/apache/paimon/sort/zorder/ZIndexerTest.java index c94327817b36..fbaf9f90aa9b 100644 --- a/paimon-core/src/test/java/org/apache/paimon/sort/zorder/ZIndexerTest.java +++ b/paimon-core/src/test/java/org/apache/paimon/sort/zorder/ZIndexerTest.java @@ -25,7 +25,6 @@ import org.apache.paimon.types.IntType; import org.apache.paimon.types.RowType; import org.apache.paimon.types.VarCharType; -import org.apache.paimon.utils.ZOrderByteUtils; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/SortCompactAction.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/SortCompactAction.java index eb545ea4377e..2724a5da3ab5 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/SortCompactAction.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/action/SortCompactAction.java @@ -19,11 +19,11 @@ package org.apache.paimon.flink.action; import org.apache.paimon.CoreOptions; +import org.apache.paimon.CoreOptions.OrderType; import org.apache.paimon.flink.FlinkConnectorOptions; import org.apache.paimon.flink.sink.SortCompactSinkBuilder; import org.apache.paimon.flink.sorter.TableSortInfo; import org.apache.paimon.flink.sorter.TableSorter; -import org.apache.paimon.flink.sorter.TableSorter.OrderType; import org.apache.paimon.flink.source.FlinkSourceBuilder; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java index ecaa5678dd0b..4fd0edf6f058 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sink/FlinkSinkBuilder.java @@ -18,6 +18,7 @@ package org.apache.paimon.flink.sink; +import org.apache.paimon.CoreOptions.OrderType; import org.apache.paimon.annotation.Public; import org.apache.paimon.data.InternalRow; import org.apache.paimon.flink.FlinkConnectorOptions; @@ -25,7 +26,6 @@ import org.apache.paimon.flink.sink.index.GlobalDynamicBucketSink; import org.apache.paimon.flink.sorter.TableSortInfo; import org.apache.paimon.flink.sorter.TableSorter; -import org.apache.paimon.flink.sorter.TableSorter.OrderType; import org.apache.paimon.table.BucketMode; import org.apache.paimon.table.FileStoreTable; import org.apache.paimon.table.Table; @@ -51,14 +51,14 @@ import java.util.List; import java.util.Map; +import static org.apache.paimon.CoreOptions.OrderType.HILBERT; +import static org.apache.paimon.CoreOptions.OrderType.ORDER; +import static org.apache.paimon.CoreOptions.OrderType.ZORDER; import static org.apache.paimon.flink.FlinkConnectorOptions.CLUSTERING_SAMPLE_FACTOR; import static org.apache.paimon.flink.FlinkConnectorOptions.CLUSTERING_STRATEGY; import static org.apache.paimon.flink.FlinkConnectorOptions.MIN_CLUSTERING_SAMPLE_FACTOR; import static org.apache.paimon.flink.sink.FlinkSink.isStreaming; import static org.apache.paimon.flink.sink.FlinkStreamPartitioner.partition; -import static org.apache.paimon.flink.sorter.TableSorter.OrderType.HILBERT; -import static org.apache.paimon.flink.sorter.TableSorter.OrderType.ORDER; -import static org.apache.paimon.flink.sorter.TableSorter.OrderType.ZORDER; import static org.apache.paimon.table.BucketMode.BUCKET_UNAWARE; import static org.apache.paimon.utils.Preconditions.checkArgument; import static org.apache.paimon.utils.Preconditions.checkState; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/TableSortInfo.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/TableSortInfo.java index 9b359e232df5..e8948c80275a 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/TableSortInfo.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/TableSortInfo.java @@ -18,8 +18,8 @@ package org.apache.paimon.flink.sorter; +import org.apache.paimon.CoreOptions.OrderType; import org.apache.paimon.flink.FlinkConnectorOptions; -import org.apache.paimon.flink.sorter.TableSorter.OrderType; import java.util.Collections; import java.util.List; diff --git a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/TableSorter.java b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/TableSorter.java index a0d4b6af2639..1d51ca4ebee6 100644 --- a/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/TableSorter.java +++ b/paimon-flink/paimon-flink-common/src/main/java/org/apache/paimon/flink/sorter/TableSorter.java @@ -18,6 +18,7 @@ package org.apache.paimon.flink.sorter; +import org.apache.paimon.CoreOptions.OrderType; import org.apache.paimon.flink.action.SortCompactAction; import org.apache.paimon.table.FileStoreTable; @@ -83,34 +84,4 @@ public static TableSorter getSorter( throw new IllegalArgumentException("cannot match order type: " + sortStrategy); } } - - /** The order type of table sort. */ - public enum OrderType { - ORDER("order"), - ZORDER("zorder"), - HILBERT("hilbert"); - - private final String orderType; - - OrderType(String orderType) { - this.orderType = orderType; - } - - @Override - public String toString() { - return "order type: " + orderType; - } - - public static OrderType of(String orderType) { - if (ORDER.orderType.equalsIgnoreCase(orderType)) { - return ORDER; - } else if (ZORDER.orderType.equalsIgnoreCase(orderType)) { - return ZORDER; - } else if (HILBERT.orderType.equalsIgnoreCase(orderType)) { - return HILBERT; - } - - throw new IllegalArgumentException("cannot match type: " + orderType + " for ordering"); - } - } } diff --git a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sorter/TableSortInfoTest.java b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sorter/TableSortInfoTest.java index e9e01baead2c..eae05cd4c873 100644 --- a/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sorter/TableSortInfoTest.java +++ b/paimon-flink/paimon-flink-common/src/test/java/org/apache/paimon/flink/sorter/TableSortInfoTest.java @@ -18,8 +18,8 @@ package org.apache.paimon.flink.sorter; +import org.apache.paimon.CoreOptions.OrderType; import org.apache.paimon.flink.FlinkConnectorOptions; -import org.apache.paimon.flink.sorter.TableSorter.OrderType; import org.junit.jupiter.api.Test; diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactProcedure.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactProcedure.java index 4a43e39c31ba..689e2cabd3d2 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactProcedure.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/CompactProcedure.java @@ -19,6 +19,7 @@ package org.apache.paimon.spark.procedure; import org.apache.paimon.CoreOptions; +import org.apache.paimon.CoreOptions.OrderType; import org.apache.paimon.annotation.VisibleForTesting; import org.apache.paimon.append.UnawareAppendCompactionTask; import org.apache.paimon.append.UnawareAppendTableCompactionCoordinator; @@ -144,7 +145,7 @@ public InternalRow[] call(InternalRow args) { String partitions = blank(args, 1) ? null : args.getString(1); // make full compact strategy as default. String compactStrategy = blank(args, 2) ? FULL : args.getString(2); - String sortType = blank(args, 3) ? TableSorter.OrderType.NONE.name() : args.getString(3); + String sortType = blank(args, 3) ? OrderType.NONE.name() : args.getString(3); List sortColumns = blank(args, 4) ? Collections.emptyList() @@ -153,11 +154,11 @@ public InternalRow[] call(InternalRow args) { String options = args.isNullAt(6) ? null : args.getString(6); Duration partitionIdleTime = blank(args, 7) ? null : TimeUtils.parseDuration(args.getString(7)); - if (TableSorter.OrderType.NONE.name().equals(sortType) && !sortColumns.isEmpty()) { + if (OrderType.NONE.name().equals(sortType) && !sortColumns.isEmpty()) { throw new IllegalArgumentException( "order_strategy \"none\" cannot work with order_by columns."); } - if (partitionIdleTime != null && (!TableSorter.OrderType.NONE.name().equals(sortType))) { + if (partitionIdleTime != null && (!OrderType.NONE.name().equals(sortType))) { throw new IllegalArgumentException( "sort compact do not support 'partition_idle_time'."); } @@ -234,7 +235,7 @@ private boolean execute( @Nullable Expression condition, @Nullable Duration partitionIdleTime) { BucketMode bucketMode = table.bucketMode(); - TableSorter.OrderType orderType = TableSorter.OrderType.of(sortType); + OrderType orderType = OrderType.of(sortType); boolean fullCompact = compactStrategy.equalsIgnoreCase(FULL); Predicate filter = condition == null @@ -245,7 +246,7 @@ private boolean execute( table.rowType(), false) .getOrElse(null); - if (orderType.equals(TableSorter.OrderType.NONE)) { + if (orderType.equals(OrderType.NONE)) { JavaSparkContext javaSparkContext = new JavaSparkContext(spark().sparkContext()); switch (bucketMode) { case HASH_FIXED: @@ -474,7 +475,7 @@ private Set getHistoryPartition( private void sortCompactUnAwareBucketTable( FileStoreTable table, - TableSorter.OrderType orderType, + OrderType orderType, List sortColumns, DataSourceV2Relation relation, @Nullable Predicate filter) { diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/sort/SparkZOrderUDF.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/sort/SparkZOrderUDF.java index c41c75cd85b8..45858fe62ab8 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/sort/SparkZOrderUDF.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/sort/SparkZOrderUDF.java @@ -18,7 +18,7 @@ package org.apache.paimon.spark.sort; -import org.apache.paimon.utils.ZOrderByteUtils; +import org.apache.paimon.sort.zorder.ZOrderByteUtils; import org.apache.spark.sql.Column; import org.apache.spark.sql.expressions.UserDefinedFunction; diff --git a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/sort/TableSorter.java b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/sort/TableSorter.java index b76d56a9aaff..a96724fad6de 100644 --- a/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/sort/TableSorter.java +++ b/paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/sort/TableSorter.java @@ -18,6 +18,7 @@ package org.apache.paimon.spark.sort; +import org.apache.paimon.CoreOptions.OrderType; import org.apache.paimon.table.FileStoreTable; import org.apache.spark.sql.Dataset; @@ -62,7 +63,7 @@ private void checkColNames() { public abstract Dataset sort(Dataset input); public static TableSorter getSorter( - FileStoreTable table, TableSorter.OrderType orderType, List orderColumns) { + FileStoreTable table, OrderType orderType, List orderColumns) { switch (orderType) { case ORDER: return new OrderSorter(table, orderColumns); @@ -81,37 +82,4 @@ public Dataset sort(Dataset input) { throw new IllegalArgumentException("cannot match order type: " + orderType); } } - - /** order type for sorting. */ - public enum OrderType { - ORDER("order"), - ZORDER("zorder"), - HILBERT("hilbert"), - NONE("none"); - - private final String orderType; - - OrderType(String orderType) { - this.orderType = orderType; - } - - @Override - public String toString() { - return "order type: " + orderType; - } - - public static OrderType of(String orderType) { - if (ORDER.orderType.equalsIgnoreCase(orderType)) { - return ORDER; - } else if (ZORDER.orderType.equalsIgnoreCase(orderType)) { - return ZORDER; - } else if (HILBERT.orderType.equalsIgnoreCase(orderType)) { - return HILBERT; - } else if (NONE.orderType.equalsIgnoreCase(orderType)) { - return NONE; - } - - throw new IllegalArgumentException("cannot match type: " + orderType + " for ordering"); - } - } }