public List toXTable()

in xtable-core/src/main/java/org/apache/xtable/iceberg/IcebergPartitionValueConverter.java [75:154]


  public List<PartitionValue> toXTable(
      InternalTable internalTable, StructLike structLike, PartitionSpec partitionSpec) {
    if (!partitionSpec.isPartitioned()) {
      return Collections.emptyList();
    }
    List<PartitionValue> partitionValues = new ArrayList<>(partitionSpec.fields().size());
    Map<InternalField, Map<PartitionTransformType, InternalPartitionField>> partitionFieldMap =
        getInternalPartitionFieldMap(internalTable);
    IndexedRecord partitionData = ((IndexedRecord) structLike);
    for (PartitionField partitionField : partitionSpec.fields()) {
      Object value;
      PartitionTransformType transformType;
      int fieldPosition =
          partitionData.getSchema().getField(escapeFieldName(partitionField.name())).pos();
      // Convert date based partitions into millis since epoch
      switch (partitionField.transform().toString()) {
        case YEAR:
          value =
              EPOCH
                  .plusYears(structLike.get(fieldPosition, Integer.class))
                  .toInstant()
                  .toEpochMilli();
          transformType = PartitionTransformType.YEAR;
          break;
        case MONTH:
          value =
              EPOCH
                  .plusMonths(structLike.get(fieldPosition, Integer.class))
                  .toInstant()
                  .toEpochMilli();
          transformType = PartitionTransformType.MONTH;
          break;
        case DAY:
          value =
              EPOCH
                  .plusDays(structLike.get(fieldPosition, Integer.class))
                  .toInstant()
                  .toEpochMilli();
          transformType = PartitionTransformType.DAY;
          break;
        case HOUR:
          value =
              EPOCH
                  .plusHours(structLike.get(fieldPosition, Integer.class))
                  .toInstant()
                  .toEpochMilli();
          transformType = PartitionTransformType.HOUR;
          break;
        case IDENTITY:
          value = structLike.get(fieldPosition, Object.class);
          transformType = PartitionTransformType.VALUE;
          break;
        default:
          if (partitionField.transform().toString().startsWith(BUCKET)) {
            value = structLike.get(fieldPosition, Integer.class);
            transformType = PartitionTransformType.BUCKET;
          } else if (partitionField.transform().isVoid()) {
            // skip void type
            continue;
          } else {
            throw new NotSupportedException(
                "Partition transform not supported: " + partitionField.transform().toString());
          }
      }
      Types.NestedField partitionSourceField =
          partitionSpec.schema().findField(partitionField.sourceId());
      InternalField sourceField =
          SchemaFieldFinder.getInstance()
              .findFieldByPath(internalTable.getReadSchema(), partitionSourceField.name());
      // This helps reduce creating these objects for each file processed and re-using them.
      InternalPartitionField internalPartitionField =
          getFromInternalPartitionFieldMap(partitionFieldMap, sourceField, transformType);
      partitionValues.add(
          PartitionValue.builder()
              .partitionField(internalPartitionField)
              .range(Range.scalar(value))
              .build());
    }
    return partitionValues;
  }