private static Set<Integer> identitySourceIds(PartitionSpec spec) { Set<Integer> sourceIds = Sets.newHashSet(); List<PartitionField> fields = spec.fields(); for (int i = 0; i < fields.size(); i += 1) { PartitionField field = fields.get(i); if ("identity".equals(field.transform().toString())) { sourceIds.add(field.sourceId()); } } return sourceIds; }
static void toJsonFields(PartitionSpec spec, JsonGenerator generator) throws IOException { generator.writeStartArray(); for (PartitionField field : spec.fields()) { generator.writeStartObject(); generator.writeStringField(NAME, field.name()); generator.writeStringField(TRANSFORM, field.transform().toString()); generator.writeNumberField(SOURCE_ID, field.sourceId()); generator.writeEndObject(); } generator.writeEndArray(); }
/** * Returns the source field ids for identity partitions. * * @return a set of source ids for the identity partitions. */ public Set<Integer> identitySourceIds() { Set<Integer> sourceIds = Sets.newHashSet(); List<PartitionField> fields = this.fields(); for (PartitionField field : fields) { if ("identity".equals(field.transform().toString())) { sourceIds.add(field.sourceId()); } } return sourceIds; }
private static PartitionData fillFromPath(PartitionSpec spec, String partitionPath, PartitionData reuse) { PartitionData data = reuse; if (data == null) { data = newPartitionData(spec); } String[] partitions = partitionPath.split("/", -1); Preconditions.checkArgument(partitions.length <= spec.fields().size(), "Invalid partition data, too many fields (expecting %s): %s", spec.fields().size(), partitionPath); Preconditions.checkArgument(partitions.length >= spec.fields().size(), "Invalid partition data, not enough fields (expecting %s): %s", spec.fields().size(), partitionPath); for (int i = 0; i < partitions.length; i += 1) { PartitionField field = spec.fields().get(i); String[] parts = partitions[i].split("=", 2); Preconditions.checkArgument( parts.length == 2 && parts[0] != null && field.name().equals(parts[0]), "Invalid partition: " + partitions[i]); data.set(i, Conversions.fromPartitionString(data.getType(i), parts[1])); } return data; }
private static PartitionData copyPartitionData(PartitionSpec spec, StructLike partitionData, PartitionData reuse) { PartitionData data = reuse; if (data == null) { data = newPartitionData(spec); } Class<?>[] javaClasses = spec.javaClasses(); List<PartitionField> fields = spec.fields(); for (int i = 0; i < fields.size(); i += 1) { data.set(i, partitionData.get(i, javaClasses[i])); } return data; }
@SuppressWarnings("unchecked") PartitionKey(PartitionSpec spec) { this.spec = spec; List<PartitionField> fields = spec.fields(); this.size = fields.size(); this.partitionTuple = new Object[size]; this.transforms = new Transform[size]; this.accessors = (Accessor<InternalRow>[]) Array.newInstance(Accessor.class, size); Schema schema = spec.schema(); Map<Integer, Accessor<InternalRow>> accessors = buildAccessors(schema); for (int i = 0; i < size; i += 1) { PartitionField field = fields.get(i); Accessor<InternalRow> accessor = accessors.get(field.sourceId()); if (accessor == null) { throw new RuntimeException( "Cannot build accessor for field: " + schema.findField(field.sourceId())); } this.accessors[i] = accessor; this.transforms[i] = field.transform(); } }
PartitionRowConverter(Schema partitionSchema, PartitionSpec spec) { StructType partitionType = SparkSchemaUtil.convert(partitionSchema); StructField[] fields = partitionType.fields(); this.types = new DataType[fields.length]; this.positions = new int[types.length]; this.javaTypes = new Class<?>[types.length]; this.reusedRow = new GenericInternalRow(types.length); List<PartitionField> partitionFields = spec.fields(); for (int rowIndex = 0; rowIndex < fields.length; rowIndex += 1) { this.types[rowIndex] = fields[rowIndex].dataType(); int sourceId = partitionSchema.columns().get(rowIndex).fieldId(); for (int specIndex = 0; specIndex < partitionFields.size(); specIndex += 1) { PartitionField field = spec.fields().get(specIndex); if (field.sourceId() == sourceId && "identity".equals(field.transform().toString())) { positions[rowIndex] = specIndex; javaTypes[rowIndex] = spec.javaClasses()[specIndex]; break; } } } }
partitionId, attemptNumber, uuid)); AppenderFactory<InternalRow> factory = new SparkAppenderFactory(); if (spec.fields().isEmpty()) { return new UnpartitionedWriter(lazyDataPath(), filename, format, conf.value(), factory);
public static TableMetadata newTableMetadata(TableOperations ops, Schema schema, PartitionSpec spec, String location, Map<String, String> properties) { // reassign all column ids to ensure consistency AtomicInteger lastColumnId = new AtomicInteger(0); Schema freshSchema = TypeUtil.assignFreshIds(schema, lastColumnId::incrementAndGet); // rebuild the partition spec using the new column ids PartitionSpec.Builder specBuilder = PartitionSpec.builderFor(freshSchema) .withSpecId(INITIAL_SPEC_ID); for (PartitionField field : spec.fields()) { // look up the name of the source field in the old schema to get the new schema's id String sourceName = schema.findColumnName(field.sourceId()); specBuilder.add( freshSchema.findField(sourceName).fieldId(), field.name(), field.transform().toString()); } PartitionSpec freshSpec = specBuilder.build(); return new TableMetadata(ops, null, location, System.currentTimeMillis(), lastColumnId.get(), freshSchema, INITIAL_SPEC_ID, ImmutableList.of(freshSpec), ImmutableMap.copyOf(properties), -1, ImmutableList.of(), ImmutableList.of()); }
@Override public List<ManifestFile> apply(TableMetadata base) { if (writeSpec().fields().size() <= 0) { // replace all data in an unpartitioned table deleteByRowFilter(Expressions.alwaysTrue()); } try { return super.apply(base); } catch (DeleteException e) { throw new ValidationException( "Cannot commit file that conflicts with existing partition: %s", e.partition()); } } }
static <R> List<R> visit(Schema schema, PartitionSpec spec, PartitionSpecVisitor<R> visitor) { List<R> 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(sourceName, field.sourceId())); } else if (transform instanceof Bucket) { results.add(visitor.bucket(sourceName, field.sourceId(), ((Bucket<?>) transform).numBuckets())); } else if (transform instanceof Truncate) { results.add(visitor.truncate(sourceName, field.sourceId(), ((Truncate<?>) transform).width())); } else if (transform == Dates.YEAR || transform == Timestamps.YEAR) { results.add(visitor.year(sourceName, field.sourceId())); } else if (transform == Dates.MONTH || transform == Timestamps.MONTH) { results.add(visitor.month(sourceName, field.sourceId())); } else if (transform == Dates.DAY || transform == Timestamps.DAY) { results.add(visitor.day(sourceName, field.sourceId())); } else if (transform == Timestamps.HOUR) { results.add(visitor.hour(sourceName, field.sourceId())); } } return results; } }
private static PartitionSpec freshSpec(int specId, Schema schema, PartitionSpec partitionSpec) { PartitionSpec.Builder specBuilder = PartitionSpec.builderFor(schema) .withSpecId(specId); for (PartitionField field : partitionSpec.fields()) { // look up the name of the source field in the old schema to get the new schema's id String sourceName = partitionSpec.schema().findColumnName(field.sourceId()); specBuilder.add( schema.findField(sourceName).fieldId(), field.name(), field.transform().toString()); } return specBuilder.build(); }
/** * Converts IcebergTable to TableDto. * * @param name qualified name * @param table iceberg table object * @param tableLoc iceberg table metadata location * @param auditInfo audit information * @return Metacat table Info */ public TableInfo fromIcebergTableToTableInfo(final QualifiedName name, final com.netflix.iceberg.Table table, final String tableLoc, final AuditInfo auditInfo) { final List<FieldInfo> allFields = this.hiveTypeConverter.icebergeSchemaTofieldDtos(table.schema(), table.spec().fields()); final Map<String, String> tableParameters = new HashMap<>(); tableParameters.put(DirectSqlTable.PARAM_TABLE_TYPE, DirectSqlTable.ICEBERG_TABLE_TYPE); tableParameters.put(DirectSqlTable.PARAM_METADATA_LOCATION, tableLoc); //adding iceberg table properties tableParameters.putAll(table.properties()); return TableInfo.builder().fields(allFields) .metadata(tableParameters) .serde(StorageInfo.builder().uri(table.location()).build()) .name(name).auditInfo(auditInfo) .build(); }
/** * Converts IcebergTable to TableDto. * * @param name qualified name * @param table iceberg table object * @param tableLoc iceberg table metadata location * @param auditInfo audit information * @return Metacat table Info */ public TableInfo fromIcebergTableToTableInfo(final QualifiedName name, final com.netflix.iceberg.Table table, final String tableLoc, final AuditInfo auditInfo) { final List<FieldInfo> allFields = this.hiveTypeConverter.icebergeSchemaTofieldDtos(table.schema(), table.spec().fields()); final Map<String, String> tableParameters = new HashMap<>(); tableParameters.put(DirectSqlTable.PARAM_TABLE_TYPE, DirectSqlTable.ICEBERG_TABLE_TYPE); tableParameters.put(DirectSqlTable.PARAM_METADATA_LOCATION, tableLoc); //adding iceberg table properties tableParameters.putAll(table.properties()); return TableInfo.builder().fields(allFields) .metadata(tableParameters) .serde(StorageInfo.builder().uri(table.location()).build()) .name(name).auditInfo(auditInfo) .build(); }
@Test public void testReplaceWithNewPartitionSpec() { PartitionSpec newSpec = PartitionSpec.unpartitioned(); Snapshot start = table.currentSnapshot(); Schema schema = table.schema(); table.newAppend() .appendFile(FILE_A) .commit(); Assert.assertEquals("Version should be 1", 1L, (long) version()); validateSnapshot(start, table.currentSnapshot(), FILE_A); Transaction replace = TestTables.beginReplace(tableDir, "test", table.schema(), newSpec); replace.commitTransaction(); table.refresh(); Assert.assertEquals("Version should be 2", 2L, (long) version()); Assert.assertNull("Table should not have a current snapshot", table.currentSnapshot()); Assert.assertEquals("Schema should use new schema, not compatible with previous", schema.asStruct(), table.schema().asStruct()); Assert.assertEquals("Table should have new unpartitioned spec", 0, table.spec().fields().size()); }
@Test public void testReplaceTransaction() { Schema newSchema = new Schema( required(4, "id", Types.IntegerType.get()), required(5, "data", Types.StringType.get())); Snapshot start = table.currentSnapshot(); Schema schema = table.schema(); table.newAppend() .appendFile(FILE_A) .commit(); Assert.assertEquals("Version should be 1", 1L, (long) version()); validateSnapshot(start, table.currentSnapshot(), FILE_A); Transaction replace = TestTables.beginReplace(tableDir, "test", newSchema, unpartitioned()); replace.commitTransaction(); table.refresh(); Assert.assertEquals("Version should be 2", 2L, (long) version()); Assert.assertNull("Table should not have a current snapshot", table.currentSnapshot()); Assert.assertEquals("Schema should match previous schema", schema.asStruct(), table.schema().asStruct()); Assert.assertEquals("Partition spec should have no fields", 0, table.spec().fields().size()); }