diff --git a/api/src/main/java/org/apache/iceberg/PartitionSpec.java b/api/src/main/java/org/apache/iceberg/PartitionSpec.java
index 4448189601f9..fc152fc142eb 100644
--- a/api/src/main/java/org/apache/iceberg/PartitionSpec.java
+++ b/api/src/main/java/org/apache/iceberg/PartitionSpec.java
@@ -35,6 +35,7 @@
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.collect.Multimaps;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.transforms.Transform;
import org.apache.iceberg.transforms.Transforms;
import org.apache.iceberg.transforms.UnknownTransform;
import org.apache.iceberg.types.Type;
@@ -333,12 +334,12 @@ private void checkAndAddPartitionName(String name) {
checkAndAddPartitionName(name, null);
}
- private void checkAndAddPartitionName(String name, Integer identitySourceColumnId) {
+ private void checkAndAddPartitionName(String name, Integer sourceColumnId) {
Types.NestedField schemaField = schema.findField(name);
- if (identitySourceColumnId != null) {
+ if (sourceColumnId != null) {
// for identity transform case we allow conflicts between partition and schema field name as
// long as they are sourced from the same schema field
- Preconditions.checkArgument(schemaField == null || schemaField.fieldId() == identitySourceColumnId,
+ Preconditions.checkArgument(schemaField == null || schemaField.fieldId() == sourceColumnId,
"Cannot create identity partition sourced from different field in schema: %s", name);
} else {
// for all other transforms we don't allow conflicts between partition name and schema field name
@@ -463,8 +464,8 @@ public Builder truncate(String sourceName, int width) {
}
public Builder alwaysNull(String sourceName, String targetName) {
- checkAndAddPartitionName(targetName);
Types.NestedField sourceColumn = findSourceColumn(sourceName);
+ checkAndAddPartitionName(targetName, sourceColumn.fieldId()); // can duplicate a source column name
fields.add(new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.alwaysNull()));
return this;
}
@@ -480,9 +481,13 @@ Builder add(int sourceId, String name, String transform) {
Builder add(int sourceId, int fieldId, String name, String transform) {
Types.NestedField column = schema.findField(sourceId);
- checkAndAddPartitionName(name, column.fieldId());
Preconditions.checkNotNull(column, "Cannot find source column: %s", sourceId);
- fields.add(new PartitionField(sourceId, fieldId, name, Transforms.fromString(column.type(), transform)));
+ return add(sourceId, fieldId, name, Transforms.fromString(column.type(), transform));
+ }
+
+ Builder add(int sourceId, int fieldId, String name, Transform, ?> transform) {
+ checkAndAddPartitionName(name, sourceId);
+ fields.add(new PartitionField(sourceId, fieldId, name, transform));
lastAssignedFieldId.getAndAccumulate(fieldId, Math::max);
return this;
}
diff --git a/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java b/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java
new file mode 100644
index 000000000000..d15c272e164c
--- /dev/null
+++ b/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+import org.apache.iceberg.exceptions.CommitFailedException;
+import org.apache.iceberg.expressions.Term;
+
+/**
+ * API for partition spec evolution.
+ *
+ * When committing, these changes will be applied to the current table metadata. Commit conflicts
+ * will not be resolved and will result in a {@link CommitFailedException}.
+ */
+public interface UpdatePartitionSpec extends PendingUpdate {
+ UpdatePartitionSpec addField(String sourceName);
+
+ UpdatePartitionSpec addField(Term term);
+
+ UpdatePartitionSpec addField(String name, Term term);
+
+ UpdatePartitionSpec removeField(String name);
+
+ UpdatePartitionSpec removeField(Term term);
+
+ UpdatePartitionSpec renameField(String name, String newName);
+}
diff --git a/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java b/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java
index cba2738fda15..1770a22a35c4 100644
--- a/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java
+++ b/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java
@@ -113,37 +113,44 @@ static List visit(PartitionSpec spec, PartitionSpecVisitor visitor) {
* @deprecated this will be removed in 0.11.0; use {@link #visit(PartitionSpec, PartitionSpecVisitor)} instead.
*/
@Deprecated
- @SuppressWarnings("checkstyle:CyclomaticComplexity")
static List visit(Schema schema, PartitionSpec spec, PartitionSpecVisitor visitor) {
List results = Lists.newArrayListWithExpectedSize(spec.fields().size());
for (PartitionField field : spec.fields()) {
- String sourceName = schema.findColumnName(field.sourceId());
- Transform, ?> transform = field.transform();
-
- if (transform instanceof Identity) {
- results.add(visitor.identity(field.fieldId(), sourceName, field.sourceId()));
- } else if (transform instanceof Bucket) {
- int numBuckets = ((Bucket>) transform).numBuckets();
- results.add(visitor.bucket(field.fieldId(), sourceName, field.sourceId(), numBuckets));
- } else if (transform instanceof Truncate) {
- int width = ((Truncate>) transform).width();
- results.add(visitor.truncate(field.fieldId(), sourceName, field.sourceId(), width));
- } else if (transform == Dates.YEAR || transform == Timestamps.YEAR) {
- results.add(visitor.year(field.fieldId(), sourceName, field.sourceId()));
- } else if (transform == Dates.MONTH || transform == Timestamps.MONTH) {
- results.add(visitor.month(field.fieldId(), sourceName, field.sourceId()));
- } else if (transform == Dates.DAY || transform == Timestamps.DAY) {
- results.add(visitor.day(field.fieldId(), sourceName, field.sourceId()));
- } else if (transform == Timestamps.HOUR) {
- results.add(visitor.hour(field.fieldId(), sourceName, field.sourceId()));
- } else if (transform instanceof VoidTransform) {
- results.add(visitor.alwaysNull(field.fieldId(), sourceName, field.sourceId()));
- } else if (transform instanceof UnknownTransform) {
- results.add(visitor.unknown(field.fieldId(), sourceName, field.sourceId(), transform.toString()));
- }
+ results.add(visit(schema, field, visitor));
}
return results;
}
+
+ @SuppressWarnings("checkstyle:CyclomaticComplexity")
+ static R visit(Schema schema, PartitionField field, PartitionSpecVisitor visitor) {
+ String sourceName = schema.findColumnName(field.sourceId());
+ Transform, ?> transform = field.transform();
+
+ if (transform instanceof Identity) {
+ return visitor.identity(field.fieldId(), sourceName, field.sourceId());
+ } else if (transform instanceof Bucket) {
+ int numBuckets = ((Bucket>) transform).numBuckets();
+ return visitor.bucket(field.fieldId(), sourceName, field.sourceId(), numBuckets);
+ } else if (transform instanceof Truncate) {
+ int width = ((Truncate>) transform).width();
+ return visitor.truncate(field.fieldId(), sourceName, field.sourceId(), width);
+ } else if (transform == Dates.YEAR || transform == Timestamps.YEAR) {
+ return visitor.year(field.fieldId(), sourceName, field.sourceId());
+ } else if (transform == Dates.MONTH || transform == Timestamps.MONTH) {
+ return visitor.month(field.fieldId(), sourceName, field.sourceId());
+ } else if (transform == Dates.DAY || transform == Timestamps.DAY) {
+ return visitor.day(field.fieldId(), sourceName, field.sourceId());
+ } else if (transform == Timestamps.HOUR) {
+ return visitor.hour(field.fieldId(), sourceName, field.sourceId());
+ } else if (transform instanceof VoidTransform) {
+ return visitor.alwaysNull(field.fieldId(), sourceName, field.sourceId());
+ } else if (transform instanceof UnknownTransform) {
+ return visitor.unknown(field.fieldId(), sourceName, field.sourceId(), transform.toString());
+ }
+
+ throw new UnsupportedOperationException(
+ String.format("Unknown transform class %s", field.transform().getClass().getName()));
+ }
}
diff --git a/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java b/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java
new file mode 100644
index 000000000000..efcf0894abb3
--- /dev/null
+++ b/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java
@@ -0,0 +1,395 @@
+/*
+ * 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;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.BoundReference;
+import org.apache.iceberg.expressions.BoundTerm;
+import org.apache.iceberg.expressions.BoundTransform;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.expressions.Term;
+import org.apache.iceberg.expressions.UnboundTerm;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.transforms.PartitionSpecVisitor;
+import org.apache.iceberg.transforms.Transform;
+import org.apache.iceberg.transforms.Transforms;
+import org.apache.iceberg.transforms.UnknownTransform;
+import org.apache.iceberg.util.Pair;
+
+class BaseUpdatePartitionSpec implements UpdatePartitionSpec {
+ private final TableOperations ops;
+ private final boolean caseSensitive;
+ private final TableMetadata base;
+ private final int formatVersion;
+ private final PartitionSpec spec;
+ private final Schema schema;
+ private final Map nameToField;
+ private final Map, PartitionField> transformToField;
+
+ private final List adds = Lists.newArrayList();
+ private final Map addedTimeFields = Maps.newHashMap();
+ private final Map, PartitionField> transformToAddedField = Maps.newHashMap();
+ private final Map nameToAddedField = Maps.newHashMap();
+ private final Set