RussellSpitzer commented on code in PR #3983:
URL: https://github.com/apache/iceberg/pull/3983#discussion_r851496982


##########
spark/v3.2/spark/src/main/java/org/apache/iceberg/spark/actions/Spark3ZOrderStrategy.java:
##########
@@ -0,0 +1,302 @@
+/*
+ * 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.iceberg.spark.actions;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.NullOrder;
+import org.apache.iceberg.PartitionField;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SortDirection;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.spark.FileRewriteCoordinator;
+import org.apache.iceberg.spark.FileScanTaskSetManager;
+import org.apache.iceberg.spark.SparkDistributionAndOrderingUtil;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.SparkWriteOptions;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.NestedField;
+import org.apache.iceberg.util.SortOrderUtil;
+import org.apache.iceberg.util.ZOrderByteUtils;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan;
+import org.apache.spark.sql.connector.distributions.Distribution;
+import org.apache.spark.sql.connector.distributions.Distributions;
+import org.apache.spark.sql.connector.expressions.SortOrder;
+import org.apache.spark.sql.expressions.UserDefinedFunction;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.BinaryType;
+import org.apache.spark.sql.types.BooleanType;
+import org.apache.spark.sql.types.ByteType;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.DateType;
+import org.apache.spark.sql.types.DoubleType;
+import org.apache.spark.sql.types.FloatType;
+import org.apache.spark.sql.types.IntegerType;
+import org.apache.spark.sql.types.LongType;
+import org.apache.spark.sql.types.ShortType;
+import org.apache.spark.sql.types.StringType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.TimestampType;
+import scala.collection.Seq;
+
+public class Spark3ZOrderStrategy extends Spark3SortStrategy {
+
+  private static final String Z_COLUMN = "ICEZVALUE";
+  private static final Schema Z_SCHEMA = new Schema(NestedField.required(0, 
Z_COLUMN, Types.BinaryType.get()));
+  private static final org.apache.iceberg.SortOrder Z_SORT_ORDER = 
org.apache.iceberg.SortOrder.builderFor(Z_SCHEMA)
+      .sortBy(Z_COLUMN, SortDirection.ASC, NullOrder.NULLS_LAST)
+      .build();
+  private static final int STRING_KEY_LENGTH = 60;
+
+  private final List<String> zOrderColNames;
+  private final FileScanTaskSetManager manager = FileScanTaskSetManager.get();
+  private final FileRewriteCoordinator rewriteCoordinator = 
FileRewriteCoordinator.get();
+
+  public Spark3ZOrderStrategy(Table table, SparkSession spark, List<String> 
zOrderColNames) {
+    super(table, spark);
+
+    Stream<String> identityPartitionColumns = table.spec().fields().stream()
+        .filter(f -> f.transform().isIdentity())
+        .map(PartitionField::name);
+    List<String> partZOrderCols = identityPartitionColumns
+        .filter(zOrderColNames::contains)
+        .collect(Collectors.toList());
+    Preconditions.checkArgument(
+        partZOrderCols.isEmpty(),
+        "Cannot ZOrder on an Identity partition column as these values are 
constant within a partition, " +
+            "ZOrdering requested on %s",
+        partZOrderCols);

Review Comment:
   Switched this to logging and added an additional check to make sure we 
haven't removed all of the ZOrder named columns. Personally I don't like 
changing user parameters if we know they are no-ops but I think this is fine 
too.



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


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to