Zouxxyy commented on code in PR #2296:
URL: https://github.com/apache/incubator-paimon/pull/2296#discussion_r1388107790


##########
paimon-spark/paimon-spark-common/src/main/java/org/apache/paimon/spark/procedure/SortCompactProcedure.java:
##########
@@ -0,0 +1,164 @@
+/*
+ * 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.paimon.spark.procedure;
+
+import org.apache.paimon.CoreOptions;
+import org.apache.paimon.annotation.VisibleForTesting;
+import org.apache.paimon.options.Options;
+import org.apache.paimon.spark.SaveMode;
+import org.apache.paimon.spark.commands.WriteIntoPaimonTable;
+import org.apache.paimon.spark.sort.TableSorter;
+import org.apache.paimon.table.AppendOnlyFileStoreTable;
+import org.apache.paimon.table.FileStoreTable;
+import org.apache.paimon.utils.ParameterUtils;
+import org.apache.paimon.utils.Preconditions;
+import org.apache.paimon.utils.StringUtils;
+
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import javax.annotation.Nullable;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/** Sort compact procedure for sort unaware-bucket table. */
+public class SortCompactProcedure extends BaseProcedure {
+
+    private static final ProcedureParameter[] PARAMETERS =
+            new ProcedureParameter[] {
+                ProcedureParameter.required("table", StringType),
+                ProcedureParameter.required("order_type", StringType),
+                ProcedureParameter.required("columns", StringType),
+                ProcedureParameter.optional("conditions", StringType),
+            };
+
+    private static final StructType OUTPUT_TYPE =
+            new StructType(
+                    new StructField[] {
+                        new StructField("result", DataTypes.BooleanType, true, 
Metadata.empty())
+                    });
+
+    protected SortCompactProcedure(TableCatalog tableCatalog) {
+        super(tableCatalog);
+    }
+
+    @Override
+    public ProcedureParameter[] parameters() {
+        return PARAMETERS;
+    }
+
+    @Override
+    public StructType outputType() {
+        return OUTPUT_TYPE;
+    }
+
+    @Override
+    public InternalRow[] call(InternalRow args) {
+        Preconditions.checkArgument(args.numFields() >= 3);
+        Identifier tableIdent = toIdentifier(args.getString(0), 
PARAMETERS[0].name());
+        String sortType = args.getString(1);
+        List<String> sortColumns = Arrays.asList(args.getString(2).split(","));
+
+        String partitionFilter = args.isNullAt(3) ? null : 
toWhere(args.getString(3));
+
+        return modifyPaimonTable(
+                tableIdent,
+                table -> {
+                    Preconditions.checkArgument(table instanceof 
FileStoreTable);
+                    InternalRow internalRow =
+                            newInternalRow(
+                                    execute(
+                                            (FileStoreTable) table,
+                                            sortType,
+                                            sortColumns,
+                                            partitionFilter));
+                    return new InternalRow[] {internalRow};
+                });
+    }
+
+    @Override
+    public String description() {
+        return "This procedure execute sort compact action on unaware-bucket 
table.";
+    }
+
+    private boolean execute(
+            FileStoreTable table,
+            String sortType,
+            List<String> sortColumns,
+            @Nullable String filter) {
+        CoreOptions coreOptions = table.store().options();
+
+        if (!(table instanceof AppendOnlyFileStoreTable) || 
coreOptions.bucket() != -1) {
+            throw new UnsupportedOperationException(
+                    "Spark sort compact only support unaware-bucket 
append-only table yet.");
+        }
+
+        Dataset<Row> row = 
spark().read().format("paimon").load(coreOptions.path().getPath());
+        row = StringUtils.isBlank(filter) ? row : row.where(filter);
+        new WriteIntoPaimonTable(
+                        table,
+                        SaveMode.dynamic(),
+                        TableSorter.getSorter(table, sortType, 
sortColumns).sort(row),
+                        new Options())
+                .run(spark());
+        return true;
+    }
+
+    @VisibleForTesting
+    static String toWhere(String partitions) {
+        if (StringUtils.isBlank(partitions)) {
+            return null;
+        }
+
+        List<Map<String, String>> maps = 
ParameterUtils.getPartitions(partitions.split(";"));
+
+        return maps.stream()

Review Comment:
   We can directly pass a where condition string (such as p1 = a and p2 > b) 
and then let spark analyze it by itself.



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