-
Notifications
You must be signed in to change notification settings - Fork 3k
Add persistent IDs to partition fields #845
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
b702aff
89d0fc6
fdbd57d
cd75cbd
9a176d2
c5b2221
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -33,6 +33,7 @@ | |
| import java.util.List; | ||
| import java.util.Map; | ||
| import java.util.Set; | ||
| import java.util.concurrent.atomic.AtomicInteger; | ||
| import org.apache.iceberg.exceptions.ValidationException; | ||
| import org.apache.iceberg.transforms.Transforms; | ||
| import org.apache.iceberg.transforms.UnknownTransform; | ||
|
|
@@ -47,7 +48,7 @@ | |
| * represented by a named {@link PartitionField}. | ||
| */ | ||
| public class PartitionSpec implements Serializable { | ||
| // start assigning IDs for partition fields at 1000 | ||
| // IDs for partition fields start at 1000 | ||
| private static final int PARTITION_DATA_ID_START = 1000; | ||
|
|
||
| private final Schema schema; | ||
|
|
@@ -58,14 +59,16 @@ public class PartitionSpec implements Serializable { | |
| private transient volatile ListMultimap<Integer, PartitionField> fieldsBySourceId = null; | ||
| private transient volatile Class<?>[] lazyJavaClasses = null; | ||
| private transient volatile List<PartitionField> fieldList = null; | ||
| private final int lastAssignedFieldId; | ||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we want to use TypeUtil::NextID?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This is tracking something different. Here, this is the highest ID assigned to any partition, so that the next ID assigned will be unique. |
||
|
|
||
| private PartitionSpec(Schema schema, int specId, List<PartitionField> fields) { | ||
| private PartitionSpec(Schema schema, int specId, List<PartitionField> fields, int lastAssignedFieldId) { | ||
| this.schema = schema; | ||
| this.specId = specId; | ||
| this.fields = new PartitionField[fields.size()]; | ||
| for (int i = 0; i < this.fields.length; i += 1) { | ||
| this.fields[i] = fields.get(i); | ||
| } | ||
| this.lastAssignedFieldId = lastAssignedFieldId; | ||
| } | ||
|
|
||
| /** | ||
|
|
@@ -89,6 +92,10 @@ public List<PartitionField> fields() { | |
| return lazyFieldList(); | ||
| } | ||
|
|
||
| int lastAssignedFieldId() { | ||
| return lastAssignedFieldId; | ||
| } | ||
|
|
||
| /** | ||
| * @param fieldId a field id from the source schema | ||
| * @return the {@link PartitionField field} that partitions the given source field | ||
|
|
@@ -107,9 +114,8 @@ public StructType partitionType() { | |
| PartitionField field = fields[i]; | ||
| Type sourceType = schema.findType(field.sourceId()); | ||
| Type resultType = field.transform().getResultType(sourceType); | ||
| // assign ids for partition fields starting at PARTITION_DATA_ID_START to leave room for data file's other fields | ||
| structFields.add( | ||
| Types.NestedField.optional(PARTITION_DATA_ID_START + i, field.name(), resultType)); | ||
| Types.NestedField.optional(field.fieldId(), field.name(), resultType)); | ||
| } | ||
|
|
||
| return Types.StructType.of(structFields); | ||
|
|
@@ -168,8 +174,8 @@ public String partitionToPath(StructLike data) { | |
| } | ||
|
|
||
| /** | ||
| * Returns true if this spec is equivalent to the other, with field names ignored. That is, if | ||
| * both specs have the same number of fields, field order, source columns, and transforms. | ||
| * Returns true if this spec is equivalent to the other, with field names and partition field ids ignored. | ||
| * That is, if both specs have the same number of fields, field order, source columns, and transforms. | ||
| * | ||
| * @param other another PartitionSpec | ||
| * @return true if the specs have the same fields, source columns, and transforms. | ||
|
|
@@ -275,7 +281,7 @@ public String toString() { | |
| } | ||
|
|
||
| private static final PartitionSpec UNPARTITIONED_SPEC = | ||
| new PartitionSpec(new Schema(), 0, ImmutableList.of()); | ||
| new PartitionSpec(new Schema(), 0, ImmutableList.of(), PARTITION_DATA_ID_START - 1); | ||
|
|
||
| /** | ||
| * Returns a spec for unpartitioned tables. | ||
|
|
@@ -307,11 +313,16 @@ public static class Builder { | |
| private final Set<String> partitionNames = Sets.newHashSet(); | ||
| private Map<Integer, PartitionField> timeFields = Maps.newHashMap(); | ||
| private int specId = 0; | ||
| private final AtomicInteger lastAssignedFieldId = new AtomicInteger(PARTITION_DATA_ID_START - 1); | ||
|
|
||
| private Builder(Schema schema) { | ||
| this.schema = schema; | ||
| } | ||
|
|
||
| private int nextFieldId() { | ||
| return lastAssignedFieldId.incrementAndGet(); | ||
| } | ||
|
|
||
| private void checkAndAddPartitionName(String name) { | ||
| checkAndAddPartitionName(name, null); | ||
| } | ||
|
|
@@ -357,7 +368,7 @@ Builder identity(String sourceName, String targetName) { | |
| Types.NestedField sourceColumn = findSourceColumn(sourceName); | ||
| checkAndAddPartitionName(targetName, sourceColumn.fieldId()); | ||
| fields.add(new PartitionField( | ||
| sourceColumn.fieldId(), targetName, Transforms.identity(sourceColumn.type()))); | ||
| sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.identity(sourceColumn.type()))); | ||
| return this; | ||
| } | ||
|
|
||
|
|
@@ -369,7 +380,7 @@ public Builder year(String sourceName, String targetName) { | |
| checkAndAddPartitionName(targetName); | ||
| Types.NestedField sourceColumn = findSourceColumn(sourceName); | ||
| PartitionField field = new PartitionField( | ||
| sourceColumn.fieldId(), targetName, Transforms.year(sourceColumn.type())); | ||
| sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.year(sourceColumn.type())); | ||
| checkForRedundantPartitions(field); | ||
| fields.add(field); | ||
| return this; | ||
|
|
@@ -383,7 +394,7 @@ public Builder month(String sourceName, String targetName) { | |
| checkAndAddPartitionName(targetName); | ||
| Types.NestedField sourceColumn = findSourceColumn(sourceName); | ||
| PartitionField field = new PartitionField( | ||
| sourceColumn.fieldId(), targetName, Transforms.month(sourceColumn.type())); | ||
| sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.month(sourceColumn.type())); | ||
| checkForRedundantPartitions(field); | ||
| fields.add(field); | ||
| return this; | ||
|
|
@@ -397,7 +408,7 @@ public Builder day(String sourceName, String targetName) { | |
| checkAndAddPartitionName(targetName); | ||
| Types.NestedField sourceColumn = findSourceColumn(sourceName); | ||
| PartitionField field = new PartitionField( | ||
| sourceColumn.fieldId(), targetName, Transforms.day(sourceColumn.type())); | ||
| sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.day(sourceColumn.type())); | ||
| checkForRedundantPartitions(field); | ||
| fields.add(field); | ||
| return this; | ||
|
|
@@ -411,7 +422,7 @@ public Builder hour(String sourceName, String targetName) { | |
| checkAndAddPartitionName(targetName); | ||
| Types.NestedField sourceColumn = findSourceColumn(sourceName); | ||
| PartitionField field = new PartitionField( | ||
| sourceColumn.fieldId(), targetName, Transforms.hour(sourceColumn.type())); | ||
| sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.hour(sourceColumn.type())); | ||
| checkForRedundantPartitions(field); | ||
| fields.add(field); | ||
| return this; | ||
|
|
@@ -425,7 +436,7 @@ public Builder bucket(String sourceName, int numBuckets, String targetName) { | |
| checkAndAddPartitionName(targetName); | ||
| Types.NestedField sourceColumn = findSourceColumn(sourceName); | ||
| fields.add(new PartitionField( | ||
| sourceColumn.fieldId(), targetName, Transforms.bucket(sourceColumn.type(), numBuckets))); | ||
| sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.bucket(sourceColumn.type(), numBuckets))); | ||
| return this; | ||
| } | ||
|
|
||
|
|
@@ -437,24 +448,30 @@ public Builder truncate(String sourceName, int width, String targetName) { | |
| checkAndAddPartitionName(targetName); | ||
| Types.NestedField sourceColumn = findSourceColumn(sourceName); | ||
| fields.add(new PartitionField( | ||
| sourceColumn.fieldId(), targetName, Transforms.truncate(sourceColumn.type(), width))); | ||
| sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.truncate(sourceColumn.type(), width))); | ||
| return this; | ||
| } | ||
|
|
||
| public Builder truncate(String sourceName, int width) { | ||
| return truncate(sourceName, width, sourceName + "_trunc"); | ||
| } | ||
|
|
||
| // add a partition field with an auto-increment partition field id starting from PARTITION_DATA_ID_START | ||
| Builder add(int sourceId, String name, String transform) { | ||
| return add(sourceId, nextFieldId(), name, 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, name, Transforms.fromString(column.type(), transform))); | ||
| fields.add(new PartitionField(sourceId, fieldId, name, Transforms.fromString(column.type(), transform))); | ||
| lastAssignedFieldId.getAndAccumulate(fieldId, Math::max); | ||
| return this; | ||
| } | ||
|
|
||
| public PartitionSpec build() { | ||
| PartitionSpec spec = new PartitionSpec(schema, specId, fields); | ||
| PartitionSpec spec = new PartitionSpec(schema, specId, fields, lastAssignedFieldId.get()); | ||
| checkCompatibility(spec, schema); | ||
| return spec; | ||
| } | ||
|
|
@@ -473,4 +490,13 @@ static void checkCompatibility(PartitionSpec spec, Schema schema) { | |
| sourceType, field.transform()); | ||
| } | ||
| } | ||
|
|
||
| static boolean hasSequentialIds(PartitionSpec spec) { | ||
| for (int i = 0; i < spec.fields.length; i += 1) { | ||
| if (spec.fields[i].fieldId() != PARTITION_DATA_ID_START + i) { | ||
| return false; | ||
| } | ||
| } | ||
| return true; | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need to check
fieldIdis larger than 1000?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I don't think so. That is a convention that we use, but not strictly required by the spec.