yihua commented on code in PR #13223:
URL: https://github.com/apache/hudi/pull/13223#discussion_r2065629229


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java:
##########
@@ -127,13 +129,23 @@ private Object getFieldValueFromInternalRow(InternalRow 
row, Schema recordSchema
   @Override
   public UnaryOperator<InternalRow> projectRecord(Schema from, Schema to, 
Map<String, String> renamedColumns) {
     Function1<InternalRow, UnsafeRow> unsafeRowWriter =
-        HoodieInternalRowUtils.getCachedUnsafeRowWriter(getCachedSchema(from), 
getCachedSchema(to), renamedColumns);
+        HoodieInternalRowUtils.getCachedUnsafeRowWriter(getCachedSchema(from), 
getCachedSchema(to), renamedColumns, Collections.emptyMap());
     return row -> (InternalRow) unsafeRowWriter.apply(row);
-
   }
 
-  protected UnaryOperator<InternalRow> getIdentityProjection() {
-    return row -> row;
+  protected UnaryOperator<InternalRow> getBootstrapProjection(Schema from, 
Schema to, Option<String[]> partitionFields, Object[] partitionValues) {
+    Map<Integer, Object> partitionValuesByIndex = new HashMap<>();
+    if (partitionFields.isPresent()) {
+      String[] partitionFieldNames = partitionFields.get();
+      for (int i = 0; i < partitionFieldNames.length; i++) {
+        int index = to.getField(partitionFieldNames[i]).pos();

Review Comment:
   Based on the logic it is assumed that the schema `to` contains all the 
fields in `partitionFields`.  Would be good to add javadocs on that assumption.



##########
hudi-common/src/main/java/org/apache/hudi/common/table/PartitionPathParser.java:
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.hudi.common.table;
+
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.PartitionPathEncodeUtils;
+import org.apache.hudi.common.util.VisibleForTesting;
+
+import org.apache.avro.LogicalType;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+
+import java.nio.charset.StandardCharsets;
+import java.sql.Date;
+import java.sql.Timestamp;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
+
+import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema;
+
+public class PartitionPathParser {
+  public static final String DEPRECATED_DEFAULT_PARTITION_PATH = "default";
+  public static final String DEFAULT_PARTITION_PATH = 
"__HIVE_DEFAULT_PARTITION__";
+  private static final String EQUALS_SIGN = "=";
+  private static final String DASH = "-";
+  private static final String SLASH = "/";
+
+  public Object[] getPartitionFieldVals(Option<String[]> partitionFields,
+                                        String partitionPath,
+                                        Schema writerSchema) {
+    if (!partitionFields.isPresent()) {
+      return new Object[0];
+    }
+    return getPartitionValues(partitionFields.get(), partitionPath, 
writerSchema);
+  }
+
+  private static Object[] getPartitionValues(String[] partitionFields,
+                                             String partitionPath,
+                                             Schema schema) {
+    String[] parts = partitionPath.split("/");
+    int pathSegment = 0;
+    boolean hasDateField = false;
+    Object[] partitionValues = new Object[partitionFields.length];
+    for (int i = 0; i < partitionFields.length; i++) {
+      String partitionField = partitionFields[i];
+      Schema.Field field = schema.getField(partitionField);
+      // if the field is not present in the schema, we assume it is a string
+      Schema fieldSchema = field == null ? Schema.create(Schema.Type.STRING) : 
resolveNullableSchema(field.schema());
+      LogicalType logicalType = fieldSchema.getLogicalType();
+      if (isTimeBasedLogicalType(logicalType)) {
+        if (hasDateField) {
+          throw new IllegalArgumentException("Only one date field based 
partition is supported");
+        }
+        hasDateField = true;
+        int numDateDirs = parts.length - partitionFields.length + 1;
+        partitionValues[i] = inferDateValue(partitionPath, parts, pathSegment, 
numDateDirs, fieldSchema);
+        pathSegment += numDateDirs;
+      } else {
+        String segment = parts[pathSegment];
+        String[] segmentParts = segment.split(EQUALS_SIGN);
+        partitionValues[i] = parseValue(segmentParts[segmentParts.length - 1], 
fieldSchema);
+        pathSegment++;
+      }
+    }
+    return partitionValues;
+  }
+
+  @VisibleForTesting
+  static Object parseValue(String partitionValue, Schema fieldSchema) {
+    if (partitionValue.equals(DEFAULT_PARTITION_PATH) || 
partitionValue.equals(DEPRECATED_DEFAULT_PARTITION_PATH)) {
+      return null;
+    }
+
+    switch (fieldSchema.getType()) {
+      case STRING:
+        return PartitionPathEncodeUtils.unescapePathName(partitionValue);
+      case INT:
+        return Integer.parseInt(partitionValue);
+      case LONG:
+        return Long.parseLong(partitionValue);
+      case FLOAT:
+        return Float.parseFloat(partitionValue);
+      case DOUBLE:
+        return Double.parseDouble(partitionValue);
+      case BOOLEAN:
+        return Boolean.parseBoolean(partitionValue);
+      case BYTES:
+      case FIXED:
+        if (fieldSchema.getLogicalType() instanceof LogicalTypes.Decimal) {
+          return new java.math.BigDecimal(partitionValue);
+        } else {
+          return partitionValue.getBytes(StandardCharsets.UTF_8);
+        }
+      default:
+        throw new IllegalArgumentException("Unexpected type " + 
fieldSchema.getType());
+    }
+  }
+
+  private static Object inferDateValue(

Review Comment:
   The date-typed partition field is not supported for bootstrap table.  So it 
is OK to keep the parser simple without this.



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/BaseSparkInternalRowReaderContext.java:
##########
@@ -146,4 +158,17 @@ public Comparable convertValueToEngineType(Comparable 
value) {
     }
     return value;
   }
+
+  protected static UnaryOperator<InternalRow> addPartitionFields(Schema 
schema, String[] partitionFields, Object[] partitionValues) {

Review Comment:
   This method seems not to be used.



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

Reply via email to