You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/05/19 02:27:04 UTC

[GitHub] [hudi] minihippo opened a new pull request, #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

minihippo opened a new pull request, #5629:
URL: https://github.com/apache/hudi/pull/5629

   ## What is the purpose of the pull request
   
   RFC-46 spark specific file reader/writer based on internal row
   
   ## Brief change log
   
   add spark file reader of parquet/orc/HFile
   add spark file writer of parquet/orc/HFile
   
   ## Verify this pull request
   
   *(Please pick either of the following options)*
   
   This pull request is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This pull request is already covered by existing tests, such as *(please describe tests)*.
   
   (or)
   
   This change added tests and can be verified as follows:
   
   *(example:)*
   
     - *Added integration tests for end-to-end.*
     - *Added HoodieClientWriteTest to verify the change.*
     - *Manually verified the change by running a job locally.*
   
   ## Committer checklist
   
    - [ ] Has a corresponding JIRA in PR title & commit
    
    - [ ] Commit message is descriptive of the change
    
    - [ ] CI is green
   
    - [ ] Necessary doc changes done or have another open PR
          
    - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA.
   


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r943064220


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -20,62 +20,33 @@
 package org.apache.hudi.common.table.log;
 
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
-import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.stream.StreamSupport;
+import java.util.Properties;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
-public class HoodieFileSliceReader<T extends HoodieRecordPayload> implements Iterator<HoodieRecord<T>> {
+public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().getKeyWithParams(schema, props,

Review Comment:
   Yes. What you said is possible. 
   
   One thing needs to be added, in addition to HoodieKey, getKeyWithParams func also converts avro data to HoodieRecordPayload. I think the HoodieRecord returned by FileReader should be the original data in file rather than the processed data. Because the callers of FileReader need to process the HoodieRecord differently or not need to process it. We do not need to converge the process logic to FileReader.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] xushiyan commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
xushiyan commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r919594262


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.io.storage;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.row.HoodieRowParquetConfig;
+import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.IOException;
+
+public class HoodieSparkFileWriterFactory extends HoodieFileWriterFactory {
+
+  private static class SingletonHolder {
+
+    private static final HoodieSparkFileWriterFactory INSTANCE = new HoodieSparkFileWriterFactory();
+  }
+
+  public static HoodieFileWriterFactory getFileWriterFactory() {
+    return HoodieSparkFileWriterFactory.SingletonHolder.INSTANCE;
+  }
+
+  @Override
+  protected HoodieFileWriter newParquetFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS);
+    boolean enableBloomFilter = populateMetaFields;
+    Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
+    String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME);
+    // Support PARQUET_COMPRESSION_CODEC_NAME is ""
+    if (compressionCodecName.isEmpty()) {
+      compressionCodecName = null;
+    }
+    HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf,
+        HoodieInternalRowUtils.getCachedSchema(schema), filter.get(),
+        HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build());
+    HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport,
+        CompressionCodecName.fromConf(compressionCodecName),
+        config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE),
+        config.getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE),
+        config.getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE),
+        conf,
+        config.getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION),
+        config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED));
+    parquetConfig.getHadoopConf().addResource(writeSupport.getHadoopConf());
+
+    return new HoodieSparkParquetWriter(path, parquetConfig, instantTime, taskContextSupplier, config.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS));
+  }
+
+  protected HoodieFileWriter newParquetFileWriter(
+      FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException {
+    boolean populateMetaFields = true;
+    boolean enableBloomFilter = false;
+    BloomFilter filter = enableBloomFilter ? createBloomFilter(config) : null;
+    HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf,
+        HoodieInternalRowUtils.getCachedSchema(schema), filter,
+        HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build());
+    String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME);
+    // Support PARQUET_COMPRESSION_CODEC_NAME is ""
+    if (compressionCodecName.isEmpty()) {
+      compressionCodecName = null;
+    }
+    HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport,
+        CompressionCodecName.fromConf(compressionCodecName),
+        config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE),
+        config.getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE),
+        config.getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE),
+        writeSupport.getHadoopConf(), config.getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION),
+        config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED));
+    parquetConfig.getHadoopConf().addResource(writeSupport.getHadoopConf());

Review Comment:
   looks duplicate from the other newParquetFileWriter()



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.io.storage.row;
+
+import org.apache.hudi.io.storage.HoodieParquetConfig;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+/**
+ * ParquetConfig for datasource implementation with {@link org.apache.hudi.client.model.HoodieInternalRow}.
+ */
+public class HoodieRowParquetConfig extends HoodieParquetConfig<HoodieRowParquetWriteSupport> {
+
+  public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName,
+                                int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
+                                double compressionRatio) {
+    super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio);
+  }
+
+  public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName,
+      int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
+      double compressionRatio, boolean dictionEnable) {

Review Comment:
   enableDictionary



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.io.storage;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.util.BaseFileUtils;
+import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.common.util.ParquetReaderIterator;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.io.InputFile;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+public class HoodieSparkParquetReader implements HoodieSparkFileReader {
+
+  private final Path path;
+  private final Configuration conf;
+  private final BaseFileUtils parquetUtils;
+  private List<ParquetReaderIterator> readerIterators = new ArrayList<>();
+
+  public HoodieSparkParquetReader(Configuration conf, Path path) {
+    this.path = path;
+    this.conf = conf;
+    this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    return parquetUtils.readMinMaxRecordKeys(conf, path);
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    return parquetUtils.readBloomFilterFromMetadata(conf, path);
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set<String> candidateRowKeys) {
+    return parquetUtils.filterRowKeys(conf, path, candidateRowKeys);
+  }
+
+  @Override
+  public ClosableIterator<InternalRow> getInternalRowIterator(Schema schema) throws IOException {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA(), structType.json());
+    // todo: get it from spark context
+    conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING().key(),false);
+    conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), true);

Review Comment:
   pls fix this by setting it from environment



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieInternalRowUtils.scala:
##########
@@ -188,29 +211,82 @@ object HoodieInternalRowUtils {
     schemaMap.get(schema)
   }
 
+  def getProjection(from: Schema, to: Schema): Projection = {
+    getCachedUnsafeProjection(getCachedSchema(from), getCachedSchema(to))
+  }
+
   private def getCachedProjection(from: StructType, to: StructType): Projection = {
     val schemaPair = (from, to)
     if (!projectionMap.contains(schemaPair)) {
       projectionMap.synchronized {
         if (!projectionMap.contains(schemaPair)) {
-          val projection = generateMutableProjection(from, to)
+          val utilsClazz = ReflectionUtils.getClass("org.apache.hudi.HoodieSparkProjectionUtils")

Review Comment:
   @wzx140 i thought we made changes to avoid per-record reflection from previous step. can we stick to the no-reflection principle here?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.io.storage;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.row.HoodieRowParquetConfig;
+import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.IOException;
+
+public class HoodieSparkFileWriterFactory extends HoodieFileWriterFactory {
+
+  private static class SingletonHolder {
+
+    private static final HoodieSparkFileWriterFactory INSTANCE = new HoodieSparkFileWriterFactory();
+  }
+
+  public static HoodieFileWriterFactory getFileWriterFactory() {
+    return HoodieSparkFileWriterFactory.SingletonHolder.INSTANCE;
+  }
+
+  @Override
+  protected HoodieFileWriter newParquetFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS);
+    boolean enableBloomFilter = populateMetaFields;
+    Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
+    String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME);
+    // Support PARQUET_COMPRESSION_CODEC_NAME is ""
+    if (compressionCodecName.isEmpty()) {
+      compressionCodecName = null;
+    }
+    HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf,
+        HoodieInternalRowUtils.getCachedSchema(schema), filter.get(),
+        HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build());
+    HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport,
+        CompressionCodecName.fromConf(compressionCodecName),
+        config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE),
+        config.getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE),
+        config.getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE),
+        conf,
+        config.getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION),
+        config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED));
+    parquetConfig.getHadoopConf().addResource(writeSupport.getHadoopConf());
+
+    return new HoodieSparkParquetWriter(path, parquetConfig, instantTime, taskContextSupplier, config.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS));
+  }
+
+  protected HoodieFileWriter newParquetFileWriter(
+      FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException {
+    boolean populateMetaFields = true;
+    boolean enableBloomFilter = false;
+    BloomFilter filter = enableBloomFilter ? createBloomFilter(config) : null;
+    HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf,
+        HoodieInternalRowUtils.getCachedSchema(schema), filter,
+        HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build());
+    String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME);
+    // Support PARQUET_COMPRESSION_CODEC_NAME is ""
+    if (compressionCodecName.isEmpty()) {
+      compressionCodecName = null;
+    }

Review Comment:
   ditto



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.io.storage;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.row.HoodieRowParquetConfig;
+import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.IOException;
+
+public class HoodieSparkFileWriterFactory extends HoodieFileWriterFactory {
+
+  private static class SingletonHolder {
+
+    private static final HoodieSparkFileWriterFactory INSTANCE = new HoodieSparkFileWriterFactory();
+  }
+
+  public static HoodieFileWriterFactory getFileWriterFactory() {
+    return HoodieSparkFileWriterFactory.SingletonHolder.INSTANCE;
+  }
+
+  @Override
+  protected HoodieFileWriter newParquetFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS);
+    boolean enableBloomFilter = populateMetaFields;
+    Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
+    String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME);
+    // Support PARQUET_COMPRESSION_CODEC_NAME is ""
+    if (compressionCodecName.isEmpty()) {
+      compressionCodecName = null;
+    }

Review Comment:
   this is weird.. can we get rid of this check and validate the values set to PARQUET_COMPRESSION_CODEC_NAME at config level?



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1191165931

   @xushiyan @vinothchandar Please take a moment to look at this pr. I have revised it according to the feedback.


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r928059133


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -21,64 +21,46 @@
 
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
 import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
+import java.util.Properties;
 import java.util.stream.StreamSupport;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
 public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieAvroFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().expansion(props, simpleKeyGenFieldsOpt,
+            scanner.isWithOperationField(), scanner.getPartitionName(), false));
       }
       return new HoodieFileSliceReader(scanner.iterator());
     } else {
       Iterable<HoodieRecord> iterable = () -> scanner.iterator();
-      HoodiePayloadConfig payloadConfig = HoodiePayloadConfig.newBuilder().withPayloadOrderingField(preCombineField).build();
       return new HoodieFileSliceReader(StreamSupport.stream(iterable.spliterator(), false)
           .map(e -> {
             try {
-              GenericRecord record = (GenericRecord) e.toIndexedRecord(schema, payloadConfig.getProps()).get();
-              return transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
+              return e.expansion(props, simpleKeyGenFieldsOpt, scanner.isWithOperationField(), scanner.getPartitionName(), false);

Review Comment:
   Let's pull back a little bit here:  i fully appreciate that might be missing operational context that you have acquired performing this refactoring and seeing these nuanced interconnections. Zooming out, my primary issue with this API is   that when i'm looking at this method in the interface -- i can't understand what it does and can't understand why it even should be here (even knowing what it does)



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1225809494

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 819824dcc83e97a7a36dab27cb2f877c113de4c6 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920) 
   * 5165092d2dca99c4e684d76811ff3d38ca0ee049 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1225830848

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 5165092d2dca99c4e684d76811ff3d38ca0ee049 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1131042547

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c34256ac98d03c787d264e56e35a7058d4273442 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755) 
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1131037264

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c34256ac98d03c787d264e56e35a7058d4273442 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] yuzhaojing merged pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
yuzhaojing merged PR #5629:
URL: https://github.com/apache/hudi/pull/5629


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1180689993

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 5f47e8e08a77e72c0916868e296e80d72fcd7d18 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845) 
   * d12f5116645664470119b8c79787c2a7da5feb87 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1250959163

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097",
       "triggerID" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11342",
       "triggerID" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33d151a1041104ecfbeeb7b749e82643e1abb11b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11493",
       "triggerID" : "33d151a1041104ecfbeeb7b749e82643e1abb11b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 33d151a1041104ecfbeeb7b749e82643e1abb11b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11493) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1230658407

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a149effad0c1992bd717e65208fbcef601deeb97 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1246302869

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097",
       "triggerID" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11342",
       "triggerID" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 6cd6761a77f288c15508f286a5294491026ffb81 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11342) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r938412210


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.RowKeyGeneratorHelper;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+
+import scala.Tuple2;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(InternalRow data, StructType structType) {
+    return new HoodieSparkRecord(data, structType);
+  }
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();
+    final String partitionPath = (partitionName.isPresent() ? partitionName.get() :
+        getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString());
+
+    Object preCombineVal = getPreCombineVal(structType, data, preCombineField);

Review Comment:
   @alexeykudinkin Thank you for your advice, I will study it carefully. This sounds reasonable.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1204972611

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * c4f8afd6d63100feba2a218ee4331fad5790ef69 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524) 
   * a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1154992293

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1131139052

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c34256ac98d03c787d264e56e35a7058d4273442 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755) 
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1155265080

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1228007439

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 64ddff55a9f3083be754e0951bf5f082fecca9e5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953) 
   * 858a47a5b106462a5089ecf77278196bc7c7a0a8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1228073114

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 858a47a5b106462a5089ecf77278196bc7c7a0a8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1207590068

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579) 
   * 96701a705ec8c271438ff696c1451364b9b4398d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r938408127


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -30,9 +34,19 @@
  * It can implement the merging logic of HoodieRecord of different engines
  * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
  */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
+public interface HoodieRecordMerger extends Serializable {
+
+  /**
+   * This method converges combineAndGetUpdateValue and precombine from HoodiePayload.
+   * It'd be associative operation: f(a, f(b, c)) = f(f(a, b), c) (which we can translate as having 3 versions A, B, C
+   * of the single record, both orders of operations applications have to yield the same result)
+   */
+  Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;

Review Comment:
   @alexeykudinkin Maybe vc means merge function will deduplicate the records for insertion.  Do you think we should put the shouldCombine marker in the record? cc @vinothchandar 



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r942112597


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -20,62 +20,33 @@
 package org.apache.hudi.common.table.log;
 
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
-import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.stream.StreamSupport;
+import java.util.Properties;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
-public class HoodieFileSliceReader<T extends HoodieRecordPayload> implements Iterator<HoodieRecord<T>> {
+public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().getKeyWithParams(schema, props,

Review Comment:
   For old records read from FileReader and passing to HoodieMergeHanle, we do not need to extract HoodieKey. 
   HoodieMergeHanle use keyGeneratorOpt to extract recordKey directly.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1191106976

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 10b42e4cc2af26b5eba6d62fe89a184ede548d72 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101) 
   * 7fbce260c6b883c33fd982a3aa600e53f54916be Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121) 
   * 5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1189383220

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 2130d320fe85d766d5b939eee2541fe7e1a00e96 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1190544110

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 11cc4bffb3c49d2f6580e72c2a599bb9a174824e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290) 
   * 10b42e4cc2af26b5eba6d62fe89a184ede548d72 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1230570654

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017) 
   * a149effad0c1992bd717e65208fbcef601deeb97 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1230297260

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 858a47a5b106462a5089ecf77278196bc7c7a0a8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966) 
   * a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231239891

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a149effad0c1992bd717e65208fbcef601deeb97 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019) 
   * 4ab845a4c27a678876f33abf8196bf760df53408 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1232454827

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * f3330b66baa3e3abd2d2c5b238443359d707955b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058) 
   * f5cbb2bc74556f86f8d86b79e359823645067bd8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068) 
   * 1af6f84fe01384f08f05a8cc95fb711855c37eb6 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1251151122

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097",
       "triggerID" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11342",
       "triggerID" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33d151a1041104ecfbeeb7b749e82643e1abb11b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11493",
       "triggerID" : "33d151a1041104ecfbeeb7b749e82643e1abb11b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8888f85e9b3eefd14a098a1bc5b277fd5989ef8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e8888f85e9b3eefd14a098a1bc5b277fd5989ef8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 33d151a1041104ecfbeeb7b749e82643e1abb11b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11493) 
   * e8888f85e9b3eefd14a098a1bc5b277fd5989ef8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] vinothchandar commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r969133391


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -30,9 +34,19 @@
  * It can implement the merging logic of HoodieRecord of different engines
  * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
  */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
+public interface HoodieRecordMerger extends Serializable {
+
+  /**
+   * This method converges combineAndGetUpdateValue and precombine from HoodiePayload.
+   * It'd be associative operation: f(a, f(b, c)) = f(f(a, b), c) (which we can translate as having 3 versions A, B, C
+   * of the single record, both orders of operations applications have to yield the same result)
+   */
+  Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;

Review Comment:
   [sorry been out of GH with some fam stuff I have been dealing with in India.]
   
   yes wonder if we should make this API more generic and provide ways for user to transform if needed for inserts/deletes as well. (there could be use-cases today where some additional metadata is computed here, e.g arrival time etc.). Users who currently wrote some custom code inside `HoodieRecordPaylod::getInsertValue()` have no way of translating into the new API right
   
    Default can just return null/empty for deletes and newer as-is for inserts. I do feel it makes for a more streamlined experience. 
   
   



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r903758999


##########
hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java:
##########
@@ -436,14 +444,18 @@ public static GenericRecord removeFields(GenericRecord record, List<String> fiel
 
   private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) {
     Schema oldSchema = oldRecord.getSchema();
-    Object fieldValue = oldSchema.getField(field.name()) == null ? null : oldRecord.get(field.name());
+    Field oldSchemaField = oldSchema.getField(field.name());
+    Object fieldValue = oldSchemaField == null ? null : oldRecord.get(field.name());

Review Comment:
   Not possible. Value name in Schema is final.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r917627789


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);

Review Comment:
   Maybe, it should change signature to InternalRow instead of SparkRecord. The org.apache.hudi.util.HoodieSparkRecordUtils#getRecordColumnValues is the same as org.apache.hudi.avro.HoodieAvroUtils#getRecordColumnValues.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1160106390

   @hudi-bot run azure


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r943808765


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -20,62 +20,33 @@
 package org.apache.hudi.common.table.log;
 
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
-import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.stream.StreamSupport;
+import java.util.Properties;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
-public class HoodieFileSliceReader<T extends HoodieRecordPayload> implements Iterator<HoodieRecord<T>> {
+public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().getKeyWithParams(schema, props,

Review Comment:
   > One thing needs to be added, in addition to HoodieKey, getKeyWithParams func also converts avro data to HoodieRecordPayload.
   
   That's exactly what i'm referring to: this method is very cryptic with an unclear contract and expectation what is being done by it in the end.
   
   > I think the HoodieRecord returned by FileReader should be the original data in file rather than the processed data. Because the callers of FileReader need to process the HoodieRecord differently or not need to process it. We do not need to converge the process logic to FileReader.
   
   Can you please elaborate what you're referring to as "processed data"? I'm still a little bit unclear what this "processing" means.
   



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r936365784


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -124,12 +126,17 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. "
           + "This will render any value set for PRECOMBINE_FIELD_OPT_VAL in-effective");
 
-  public static final ConfigProperty<String> MERGE_CLASS_NAME = ConfigProperty
+  public static final ConfigProperty<String> MERGER_CLASS_NAME = ConfigProperty

Review Comment:
   I have discussed this with Raymond. RecordMerger should not be persisted in table config. It is up to the user to specify what to choose. If not specified, we provide a engine-default RecordMerger.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1154628024

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1180806229

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 02f8bdabc35820fa3765e77b154de6478440739d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r920369408


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {

Review Comment:
   We should make `HoodieRecord` a non-generic interface just providing the APIs and move base implementation into `HoodieBaseRecord<T>` so that all the code relying on `HoodieRecord` is not exposed to internal implementation details



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {

Review Comment:
   Let's keep this method where it's today outside of the Record class (w/in utils), and instead let's provide an API
   ```
   getFieldValue(fieldName) 
   ```



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.RowKeyGeneratorHelper;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+
+import scala.Tuple2;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(InternalRow data, StructType structType) {
+    return new HoodieSparkRecord(data, structType);
+  }
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();
+    final String partitionPath = (partitionName.isPresent() ? partitionName.get() :
+        getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString());
+
+    Object preCombineVal = getPreCombineVal(structType, data, preCombineField);

Review Comment:
   We should not be unpacking the row here -- it should only happen when we actually perform the merge



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieInternalRowUtils.scala:
##########
@@ -16,43 +16,47 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.hudi
+package org.apache.hudi
 
 import java.nio.charset.StandardCharsets
-import java.util
+import java.util.List
 import java.util.concurrent.ConcurrentHashMap
 import org.apache.avro.Schema
-import org.apache.hudi.AvroConversionUtils
-import org.apache.hudi.avro.HoodieAvroUtils
-import org.apache.hudi.avro.HoodieAvroUtils.{createFullName, fromJavaDate, toJavaDate}
+import org.apache.avro.generic.IndexedRecord
+import org.apache.hudi.HoodieSparkUtils.sparkAdapter
+import org.apache.hudi.avro.HoodieAvroUtils.{createFullName, toJavaDate}
 import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField
-import org.apache.hudi.common.util.ValidationUtils
+import org.apache.hudi.common.util.ReflectionUtils
 import org.apache.hudi.exception.HoodieException
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, MutableProjection, Projection}
+import org.apache.spark.sql.avro.{HoodieAvroDeserializer, HoodieAvroSerializer}
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, JoinedRow, MutableProjection, Projection, UnsafeProjection}
 import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData}
-import org.apache.spark.sql.hudi.ColumnStatsExpressionUtils.AllowedTransformationExpression.exprUtils.generateMutableProjection
+import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
 import org.apache.spark.sql.types._
-
 import scala.collection.mutable
 
-/**
- * Helper class to do common stuff across Spark InternalRow.
- * Provides common methods similar to {@link HoodieAvroUtils}.
- */
 object HoodieInternalRowUtils {
 
+  val unsafeProjectionMap = new ConcurrentHashMap[(StructType, StructType), UnsafeProjection]
   val projectionMap = new ConcurrentHashMap[(StructType, StructType), MutableProjection]
   val schemaMap = new ConcurrentHashMap[Schema, StructType]
-  val SchemaPosMap = new ConcurrentHashMap[StructType, Map[String, (StructField, Int)]]
+  val schemaPosMap = new ConcurrentHashMap[StructType, Map[String, (StructField, Int)]]
+  val rowConverterMap = new ConcurrentHashMap[Schema, HoodieAvroDeserializer]
+  val avroConverterMap = new ConcurrentHashMap[Schema, HoodieAvroSerializer]
 
+  /**
+   * @see org.apache.hudi.avro.HoodieAvroUtils#stitchRecords(org.apache.avro.generic.GenericRecord, org.apache.avro.generic.GenericRecord, org.apache.avro.Schema)
+   */
   def stitchRecords(left: InternalRow, leftSchema: StructType, right: InternalRow, rightSchema: StructType, stitchedSchema: StructType): InternalRow = {
     val mergeSchema = StructType(leftSchema.fields ++ rightSchema.fields)
     val row = new JoinedRow(left, right)
     val projection = getCachedProjection(mergeSchema, stitchedSchema)

Review Comment:
   There's no need to do projection in that case



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java:
##########
@@ -155,13 +165,52 @@ public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<
   }
 
   @Override
-  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties props, int pos, String newValue) throws IOException {
     data.put(pos, newValue);
     return this;
   }
 
   @Override
-  public boolean shouldIgnore(Schema schema, Properties prop) throws IOException {
+  public HoodieRecord expansion(
+      Schema schema,
+      Properties props,
+      String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    return HoodieAvroUtils.createHoodieRecordFromAvro(data, payloadClass, preCombineField, simpleKeyGenFieldsOpt, withOperation, partitionNameOp, populateMetaFieldsOp);
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties props, boolean useKeyGen) {
+    GenericRecord record = (GenericRecord) data;
+    String key;
+    String partition;
+    if (useKeyGen && !Boolean.parseBoolean(props.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        Class<?> clazz = ReflectionUtils.getClass("org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory");
+        Method createKeyGenerator = clazz.getMethod("createKeyGenerator", TypedProperties.class);
+        BaseKeyGenerator keyGeneratorOpt = (BaseKeyGenerator) createKeyGenerator.invoke(null, new TypedProperties(props));
+        key = keyGeneratorOpt.getRecordKey(record);
+        partition = keyGeneratorOpt.getPartitionPath(record);
+      } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) {
+        throw new HoodieException("Only BaseKeyGenerators are supported when meta columns are disabled ", e);
+      }
+    } else {
+      key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+      partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
+    }
+    HoodieKey hoodieKey = new HoodieKey(key, partition);
+
+    HoodieRecordPayload avroPayload = new RewriteAvroPayload(record);

Review Comment:
   We should not be using `RecordPayload` abstractions anymore -- whole idea of RFC-46 is to get rid of it.
   
   1. HoodieAvroRecord should hold a reference on the Avro payload directly
   2. RecordPayload abstraction should only be used w/in backward-compatible `HoodieMerge` implementation (during transitionary period) to merge the records and nowhere else 



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,

Review Comment:
   What's this method needed for? It seems like it's trying to do too many things 



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecordMerge.java:
##########
@@ -31,6 +34,8 @@
 public class HoodieAvroRecordMerge implements HoodieMerge {
   @Override
   public HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer) {
+    ValidationUtils.checkArgument(older.getRecordType() == HoodieRecordType.AVRO);
+    ValidationUtils.checkArgument(newer.getRecordType() == HoodieRecordType.AVRO);
     HoodieRecordPayload picked = unsafeCast(((HoodieAvroRecord) newer).getData().preCombine(((HoodieAvroRecord) older).getData()));

Review Comment:
   Building on my previous comment regarding use of RecordPayload: we should only resort to using it within this context when we actually do the merging, and we should get rid of its usages anywhere else.
   
   Here we should just lookup configured a) payload-class b) pre-combine field and use it to perform the merging



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {

Review Comment:
   What's this method needed for?



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java:
##########
@@ -189,8 +189,13 @@ protected void processNextDeletedRecord(DeleteRecord deleteRecord) {
       }
     }
     // Put the DELETE record
-    records.put(key, SpillableMapUtils.generateEmptyPayload(key,
-        deleteRecord.getPartitionPath(), deleteRecord.getOrderingValue(), getPayloadClassFQN()));
+    if (recordType == HoodieRecordType.AVRO) {

Review Comment:
   Why are we handling Avro differently here? There are only a few places where we can have this bifurcation (reader, writer, merger) but we should def not have it here



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieAvroDataBlock.java:
##########
@@ -115,13 +117,13 @@ protected byte[] serializeRecords(List<HoodieRecord> records) throws IOException
     output.writeInt(records.size());
 
     // 3. Write the records
-    for (HoodieRecord s : records) {
+    for (HoodieRecord<?> s : records) {
       ByteArrayOutputStream temp = new ByteArrayOutputStream();
       BinaryEncoder encoder = EncoderFactory.get().binaryEncoder(temp, encoderCache.get());
       encoderCache.set(encoder);
       try {
         // Encode the record into bytes
-        IndexedRecord data = (IndexedRecord) s.toIndexedRecord(schema, new Properties()).get();
+        IndexedRecord data = s.toIndexedRecord(schema, new Properties()).get();

Review Comment:
   See my comment above regarding deprecation of `toIndexedRecord`: we shouldn't be using `toIndexedRecord` here -- instead we should cast the record to `HoodieAvroRecord` and access the payload directly



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    String key;
+    String partition;
+    if (useKeygen && !Boolean.parseBoolean(prop.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        SparkKeyGeneratorInterface keyGeneratorOpt = (SparkKeyGeneratorInterface) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(prop));
+        key = keyGeneratorOpt.getRecordKey(data, structType);
+        partition = keyGeneratorOpt.getPartitionPath(data, structType);
+      } catch (IOException e) {
+        throw new HoodieException("Only SparkKeyGeneratorInterface are supported when meta columns are disabled ", e);
+      }
+    } else {
+      key = data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString();
+      partition = data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString();
+    }
+    this.key = new HoodieKey(key, partition);
+
+    return this;
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+
+  @Override
+  public boolean isPresent(Schema schema, Properties prop) throws IOException {
+    if (null == data) {
+      return false;
+    }
+    if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) == null) {
+      return true;
+    }
+    Object deleteMarker = data.get(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).pos(), BooleanType);
+    return !(deleteMarker instanceof Boolean && (boolean) deleteMarker);
+  }
+
+  @Override
+  public boolean shouldIgnore(Schema schema, Properties prop) throws IOException {
+    // TODO SENTINEL should refactor SENTINEL without Avro(GenericRecord)
+    if (data != null && data.equals(SENTINEL)) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Option<IndexedRecord> toIndexedRecord(Schema schema, Properties prop) throws IOException {

Review Comment:
   This should be a transitory API that we remove at the completion of the RFC-46. Let's mark it as deprecated 



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);

Review Comment:
   We should only be unpacking the rows and extracting `orderingVal` only when we do merging. We should avoid storing this field in the record itself



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();

Review Comment:
   This is not necessary: you can just do `structType.fieldIndex(name)`



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {

Review Comment:
   Why do we need 5 rewriting method? 1 should be enough, right?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema) throws IOException {

Review Comment:
   We should not be passing current Record's schema



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieInternalRowUtils.scala:
##########
@@ -188,29 +211,82 @@ object HoodieInternalRowUtils {
     schemaMap.get(schema)
   }
 
+  def getProjection(from: Schema, to: Schema): Projection = {
+    getCachedUnsafeProjection(getCachedSchema(from), getCachedSchema(to))
+  }
+
   private def getCachedProjection(from: StructType, to: StructType): Projection = {
     val schemaPair = (from, to)
     if (!projectionMap.contains(schemaPair)) {
       projectionMap.synchronized {
         if (!projectionMap.contains(schemaPair)) {
-          val projection = generateMutableProjection(from, to)
+          val utilsClazz = ReflectionUtils.getClass("org.apache.hudi.HoodieSparkProjectionUtils")

Review Comment:
   We can address this later, but we should avoid reflection at all costs -- instead we can just move this class into spark-client (in fact it's already moved on master actually)



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {

Review Comment:
   A few notes
   1. All Records should hold its own schema 
   2. I don't think we're looking to support merging records of different types (say Avro w/ Spark), 
   therefore we should be able to assume other record is also Spark Record. 
   
   That means there's no need to pass schemas as params here, we should extract it from the record itself.
   



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {

Review Comment:
   There's no need for both `overrideMetadataFieldValue` and `addMetadataValues`, 1 should be enough



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroIndexedRecord.java:
##########
@@ -155,13 +165,52 @@ public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<
   }
 
   @Override
-  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties props, int pos, String newValue) throws IOException {
     data.put(pos, newValue);
     return this;
   }
 
   @Override
-  public boolean shouldIgnore(Schema schema, Properties prop) throws IOException {
+  public HoodieRecord expansion(
+      Schema schema,
+      Properties props,
+      String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    return HoodieAvroUtils.createHoodieRecordFromAvro(data, payloadClass, preCombineField, simpleKeyGenFieldsOpt, withOperation, partitionNameOp, populateMetaFieldsOp);
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties props, boolean useKeyGen) {
+    GenericRecord record = (GenericRecord) data;
+    String key;
+    String partition;
+    if (useKeyGen && !Boolean.parseBoolean(props.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        Class<?> clazz = ReflectionUtils.getClass("org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory");

Review Comment:
   This is not going to work -- we can't load key generator using reflection for every record



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java:
##########
@@ -91,13 +87,13 @@ public abstract class AbstractHoodieLogRecordReader {
   // Latest valid instant time
   // Log-Blocks belonging to inflight delta-instants are filtered-out using this high-watermark.
   private final String latestInstantTime;
-  private final HoodieTableMetaClient hoodieTableMetaClient;
+  protected final HoodieTableMetaClient hoodieTableMetaClient;
   // Merge strategy to use when combining records from log
   private final String payloadClassFQN;
   // preCombine field
   private final String preCombineField;
   // Stateless component for merging records
-  private final String mergeClassFQN;
+  private final String mergeClass;

Review Comment:
   Folks, let's make sure we're not losing the context of the previous discussion: i've highlighted on a previous PR (step 2) that `HoodieMerge` is not a good name for this abstraction, and this had not been addressed.
   
   Let's make sure we follow-up on it here



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java:
##########
@@ -84,8 +86,14 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute(String instantTime,
         dedupedKeys = keys.repartition(parallelism);
       }
 
-      HoodieData<HoodieRecord<T>> dedupedRecords =
-          dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      HoodieData dedupedRecords;
+      if (config.getRecordType() == HoodieRecordType.AVRO) {

Review Comment:
   Same comment as below: there should be no need for this bifurcation



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r936357691


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -30,9 +34,19 @@
  * It can implement the merging logic of HoodieRecord of different engines
  * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
  */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
+public interface HoodieRecordMerger extends Serializable {
+
+  /**
+   * This method converges combineAndGetUpdateValue and precombine from HoodiePayload.
+   * It'd be associative operation: f(a, f(b, c)) = f(f(a, b), c) (which we can translate as having 3 versions A, B, C
+   * of the single record, both orders of operations applications have to yield the same result)
+   */
+  Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;

Review Comment:
   Maybe insert a marker in HoodieRecord?



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r945404845


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.RowKeyGeneratorHelper;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+
+import scala.Tuple2;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(InternalRow data, StructType structType) {
+    return new HoodieSparkRecord(data, structType);
+  }
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();
+    final String partitionPath = (partitionName.isPresent() ? partitionName.get() :
+        getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString());
+
+    Object preCombineVal = getPreCombineVal(structType, data, preCombineField);

Review Comment:
   I took a closer look and there is no doubt that the ordering val will be used multiple times. I think it's better to keep this ordering val in HoodieRecord just like HoodieRecordPayload. Since this value will not change during writing and we can reduce the multiply extract ordering val time.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231323764

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a149effad0c1992bd717e65208fbcef601deeb97 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019) 
   * 4ab845a4c27a678876f33abf8196bf760df53408 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038) 
   * faf16a7e76db8efc6b23be7a8362c4fe35496556 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r959827795


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -290,59 +286,59 @@ public void checkState() {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Rewrite record into new schema(add meta columns)
+   */
+  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException;
 
   /**
-   * Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields.
+   * Support schema evolution.
    */
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException;
+  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map<String, String> renameCols) throws IOException;
 
-  public abstract HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException;
+  public abstract HoodieRecord updateValues(Schema recordSchema, Properties props, Map<String, String> metadataValues) throws IOException;

Review Comment:
   Yeah, found it already. Let's annotate this method as temporary (we'd likely revisit it in the future) -- we should be wary exposing API that allows mutations.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1230283023

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 858a47a5b106462a5089ecf77278196bc7c7a0a8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966) 
   * a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231834669

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 4ee2969401d4db019780389b2898526f6c99fd86 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044) 
   * a7a1ff3ce32f1e7281eed9d42679cbff70a7d523 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050) 
   * 756a4a94fa739de42fd722a7e036563f3df60d27 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231234383

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a149effad0c1992bd717e65208fbcef601deeb97 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019) 
   * 4ab845a4c27a678876f33abf8196bf760df53408 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1251158308

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097",
       "triggerID" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11342",
       "triggerID" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33d151a1041104ecfbeeb7b749e82643e1abb11b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11493",
       "triggerID" : "33d151a1041104ecfbeeb7b749e82643e1abb11b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8888f85e9b3eefd14a098a1bc5b277fd5989ef8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11504",
       "triggerID" : "e8888f85e9b3eefd14a098a1bc5b277fd5989ef8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 33d151a1041104ecfbeeb7b749e82643e1abb11b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11493) 
   * e8888f85e9b3eefd14a098a1bc5b277fd5989ef8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11504) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1251566082

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097",
       "triggerID" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11342",
       "triggerID" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33d151a1041104ecfbeeb7b749e82643e1abb11b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11493",
       "triggerID" : "33d151a1041104ecfbeeb7b749e82643e1abb11b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e8888f85e9b3eefd14a098a1bc5b277fd5989ef8",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11504",
       "triggerID" : "e8888f85e9b3eefd14a098a1bc5b277fd5989ef8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * e8888f85e9b3eefd14a098a1bc5b277fd5989ef8 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11504) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r909635425


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java:
##########
@@ -194,6 +194,8 @@ public List<WriteStatus> compact(HoodieCompactionHandler compactionHandler,
         .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
         .withOperationField(config.allowOperationMetadataField())
         .withPartition(operation.getPartitionPath())
+        .withRecordType(config.getRecordType())
+        .withCombiningEngineClassFQN(config.getMergeClass())

Review Comment:
   Fixed



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java:
##########
@@ -140,29 +136,31 @@ public abstract class AbstractHoodieLogRecordReader {
   private Option<String> partitionName;
   // Populate meta fields for the records
   private boolean populateMetaFields = true;
+  // Record type read from log block
+  protected final HoodieRecordType recordType;
 
   protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List<String> logFilePaths,
                                           Schema readerSchema,
                                           String latestInstantTime, boolean readBlocksLazily, boolean reverseReader,
                                           int bufferSize, Option<InstantRange> instantRange,
-                                          boolean withOperationField) {
+                                          boolean withOperationField, HoodieRecordType recordType, String combiningEngineClassFQN) {
     this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize,
-        instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema());
+        instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema(), recordType, combiningEngineClassFQN);
   }
 
   protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List<String> logFilePaths,
                                           Schema readerSchema, String latestInstantTime, boolean readBlocksLazily,
                                           boolean reverseReader, int bufferSize, Option<InstantRange> instantRange,
                                           boolean withOperationField, boolean forceFullScan,
-                                          Option<String> partitionName, InternalSchema internalSchema) {
+                                          Option<String> partitionName, InternalSchema internalSchema, HoodieRecordType recordType, String combiningEngineClassFQN) {
     this.readerSchema = readerSchema;
     this.latestInstantTime = latestInstantTime;
     this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build();
     // load class from the payload fully qualified class name
     HoodieTableConfig tableConfig = this.hoodieTableMetaClient.getTableConfig();
     this.payloadClassFQN = tableConfig.getPayloadClass();
     this.preCombineField = tableConfig.getPreCombineField();
-    this.mergeClassFQN = tableConfig.getMergeClass();
+    this.mergeClassFQN = combiningEngineClassFQN;

Review Comment:
   Fixed



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r924560320


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.io.storage;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.row.HoodieRowParquetConfig;
+import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.IOException;
+
+public class HoodieSparkFileWriterFactory extends HoodieFileWriterFactory {
+
+  private static class SingletonHolder {
+
+    private static final HoodieSparkFileWriterFactory INSTANCE = new HoodieSparkFileWriterFactory();
+  }
+
+  public static HoodieFileWriterFactory getFileWriterFactory() {
+    return HoodieSparkFileWriterFactory.SingletonHolder.INSTANCE;
+  }
+
+  @Override
+  protected HoodieFileWriter newParquetFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS);
+    boolean enableBloomFilter = populateMetaFields;
+    Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
+    String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME);
+    // Support PARQUET_COMPRESSION_CODEC_NAME is ""
+    if (compressionCodecName.isEmpty()) {
+      compressionCodecName = null;
+    }

Review Comment:
   Because this check exists in HoodieWriteConfig. But we pass HoodieConfig from HoodieParquetDataBlock(hoodie-common) to WriteFactory



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r924543589


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.RowKeyGeneratorHelper;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+
+import scala.Tuple2;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(InternalRow data, StructType structType) {
+    return new HoodieSparkRecord(data, structType);
+  }
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();
+    final String partitionPath = (partitionName.isPresent() ? partitionName.get() :
+        getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString());
+
+    Object preCombineVal = getPreCombineVal(structType, data, preCombineField);

Review Comment:
   fixed. Moved getOrderingVal in HoodieRecordMerger. 



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();

Review Comment:
   fixed



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1189447202

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 2130d320fe85d766d5b939eee2541fe7e1a00e96 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062) 
   * 0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r926305993


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -21,64 +21,46 @@
 
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
 import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
+import java.util.Properties;
 import java.util.stream.StreamSupport;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
 public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieAvroFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().expansion(props, simpleKeyGenFieldsOpt,
+            scanner.isWithOperationField(), scanner.getPartitionName(), false));
       }
       return new HoodieFileSliceReader(scanner.iterator());
     } else {
       Iterable<HoodieRecord> iterable = () -> scanner.iterator();
-      HoodiePayloadConfig payloadConfig = HoodiePayloadConfig.newBuilder().withPayloadOrderingField(preCombineField).build();
       return new HoodieFileSliceReader(StreamSupport.stream(iterable.spliterator(), false)
           .map(e -> {
             try {
-              GenericRecord record = (GenericRecord) e.toIndexedRecord(schema, payloadConfig.getProps()).get();
-              return transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
+              return e.expansion(props, simpleKeyGenFieldsOpt, scanner.isWithOperationField(), scanner.getPartitionName(), false);

Review Comment:
   1. Why use expansion not transform func?  
   A: Expansion method used to extract HoodieKey not through keyGenerator. Different data(InternalRow, avro) use different extract logic. Keeping two or more wrapping logics in transform func is not reasonable.
   
   2. HoodieRecord initialization is in FileReader. I do not think that extracting key in FileReader is reasonable. Sometimes we use raw record without key.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r926262756


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -21,64 +21,46 @@
 
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
 import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
+import java.util.Properties;
 import java.util.stream.StreamSupport;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
 public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieAvroFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().expansion(props, simpleKeyGenFieldsOpt,
+            scanner.isWithOperationField(), scanner.getPartitionName(), false));
       }
       return new HoodieFileSliceReader(scanner.iterator());
     } else {
       Iterable<HoodieRecord> iterable = () -> scanner.iterator();
-      HoodiePayloadConfig payloadConfig = HoodiePayloadConfig.newBuilder().withPayloadOrderingField(preCombineField).build();
       return new HoodieFileSliceReader(StreamSupport.stream(iterable.spliterator(), false)
           .map(e -> {
             try {
-              GenericRecord record = (GenericRecord) e.toIndexedRecord(schema, payloadConfig.getProps()).get();
-              return transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
+              return e.expansion(props, simpleKeyGenFieldsOpt, scanner.isWithOperationField(), scanner.getPartitionName(), false);

Review Comment:
   I don't think we should carry over existing `transform` into `HoodieRecord.expansion` as is -- 
   while `transform` method was appropriate (was wrapping Avro into HoodieRecord), `expansion` method by itself doesn't make much sense: we already iterate by `HoodieRecord`, why do we need to expand it? We should be able to appropriately initialize `HoodieRecord` (key, partition-path) when we instantiate it during iteration
   
   
   



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r938150358


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -30,9 +34,19 @@
  * It can implement the merging logic of HoodieRecord of different engines
  * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
  */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
+public interface HoodieRecordMerger extends Serializable {
+
+  /**
+   * This method converges combineAndGetUpdateValue and precombine from HoodiePayload.
+   * It'd be associative operation: f(a, f(b, c)) = f(f(a, b), c) (which we can translate as having 3 versions A, B, C
+   * of the single record, both orders of operations applications have to yield the same result)
+   */
+  Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;

Review Comment:
   @vinothchandar not sure i understand the original question? We will retain the same semantic w/ tombstone value of `HoodieRecord` being passed around for deletion



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1227649827

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 5165092d2dca99c4e684d76811ff3d38ca0ee049 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922) 
   * 64ddff55a9f3083be754e0951bf5f082fecca9e5 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r909633457


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -977,6 +988,16 @@ public String getKeyGeneratorClass() {
     return getString(KEYGENERATOR_CLASS_NAME);
   }
 
+  public HoodieRecord.HoodieRecordType getRecordType() {
+    HoodieRecordType recordType = HoodieRecord.HoodieRecordType.valueOf(getString(RECORD_TYPE));
+    String basePath = getString(BASE_PATH);
+    boolean metadataTable = HoodieTableMetadata.isMetadataTable(basePath);
+    if (metadataTable) {
+      recordType = HoodieRecordType.AVRO;
+    }
+    return recordType;

Review Comment:
   Fixed. Cached the HoodieRecordType.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r916796368


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -84,21 +80,21 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
     Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
     HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
 
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
     Schema readSchema;
+    Schema readerSchema;
+    Schema writerSchema;

Review Comment:
   Will fix



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1190492229

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 11cc4bffb3c49d2f6580e72c2a599bb9a174824e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290) 
   * 10b42e4cc2af26b5eba6d62fe89a184ede548d72 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r924542663


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);

Review Comment:
   fixed



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r924541560


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetConfig.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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.io.storage.row;
+
+import org.apache.hudi.io.storage.HoodieParquetConfig;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+/**
+ * ParquetConfig for datasource implementation with {@link org.apache.hudi.client.model.HoodieInternalRow}.
+ */
+public class HoodieRowParquetConfig extends HoodieParquetConfig<HoodieRowParquetWriteSupport> {
+
+  public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName,
+                                int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
+                                double compressionRatio) {
+    super(writeSupport, compressionCodecName, blockSize, pageSize, maxFileSize, hadoopConf, compressionRatio);
+  }
+
+  public HoodieRowParquetConfig(HoodieRowParquetWriteSupport writeSupport, CompressionCodecName compressionCodecName,
+      int blockSize, int pageSize, long maxFileSize, Configuration hadoopConf,
+      double compressionRatio, boolean dictionEnable) {

Review Comment:
   fixed



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1190665270

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 10b42e4cc2af26b5eba6d62fe89a184ede548d72 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1192263404

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133) 
   * c3f5e348484ed6f538113b04dc1e2d7166d43848 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r929142300


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -21,64 +21,46 @@
 
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
 import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
+import java.util.Properties;
 import java.util.stream.StreamSupport;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
 public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieAvroFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().expansion(props, simpleKeyGenFieldsOpt,
+            scanner.isWithOperationField(), scanner.getPartitionName(), false));
       }
       return new HoodieFileSliceReader(scanner.iterator());
     } else {
       Iterable<HoodieRecord> iterable = () -> scanner.iterator();
-      HoodiePayloadConfig payloadConfig = HoodiePayloadConfig.newBuilder().withPayloadOrderingField(preCombineField).build();
       return new HoodieFileSliceReader(StreamSupport.stream(iterable.spliterator(), false)
           .map(e -> {
             try {
-              GenericRecord record = (GenericRecord) e.toIndexedRecord(schema, payloadConfig.getProps()).get();
-              return transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
+              return e.expansion(props, simpleKeyGenFieldsOpt, scanner.isWithOperationField(), scanner.getPartitionName(), false);

Review Comment:
   Great! New APIs i think are much better option. Thank you very much!



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -21,64 +21,46 @@
 
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
 import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
+import java.util.Properties;
 import java.util.stream.StreamSupport;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
 public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieAvroFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().expansion(props, simpleKeyGenFieldsOpt,
+            scanner.isWithOperationField(), scanner.getPartitionName(), false));
       }
       return new HoodieFileSliceReader(scanner.iterator());
     } else {
       Iterable<HoodieRecord> iterable = () -> scanner.iterator();
-      HoodiePayloadConfig payloadConfig = HoodiePayloadConfig.newBuilder().withPayloadOrderingField(preCombineField).build();
       return new HoodieFileSliceReader(StreamSupport.stream(iterable.spliterator(), false)
           .map(e -> {
             try {
-              GenericRecord record = (GenericRecord) e.toIndexedRecord(schema, payloadConfig.getProps()).get();
-              return transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
+              return e.expansion(props, simpleKeyGenFieldsOpt, scanner.isWithOperationField(), scanner.getPartitionName(), false);

Review Comment:
   Great! New APIs seem like a much better option. Thank you very much!



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r938253077


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -20,62 +20,33 @@
 package org.apache.hudi.common.table.log;
 
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
-import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.stream.StreamSupport;
+import java.util.Properties;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
-public class HoodieFileSliceReader<T extends HoodieRecordPayload> implements Iterator<HoodieRecord<T>> {
+public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().getKeyWithParams(schema, props,

Review Comment:
   Suggested transition doesn't make sense: 
   
   1. Previously, file-reader was producing `GenericRecord`s hence we had `transform` method that was wrapping them into `HoodieRecord`
   2. Now, we have file-reader instance that produces `HoodieRecord`, and so calling the method `getKeyWithParams` that returns another `HoodieRecord` doesn't really make sense.
   
   Can you please explain why do we need it and why can't we return proper record from the file-reader in the first place? 



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1250690243

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097",
       "triggerID" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11342",
       "triggerID" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33d151a1041104ecfbeeb7b749e82643e1abb11b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "33d151a1041104ecfbeeb7b749e82643e1abb11b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 6cd6761a77f288c15508f286a5294491026ffb81 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11342) 
   * 33d151a1041104ecfbeeb7b749e82643e1abb11b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r977040996


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkStructTypeSerializer.scala:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.spark.sql.hudi
+
+import com.esotericsoftware.kryo.Kryo
+import com.esotericsoftware.kryo.io.{Input, Output}
+import com.twitter.chill.KSerializer
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+import java.nio.ByteBuffer
+import java.nio.charset.StandardCharsets
+import org.apache.avro.SchemaNormalization
+import org.apache.commons.io.IOUtils
+import org.apache.hudi.commmon.model.HoodieSparkRecord
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+import org.apache.spark.{SparkEnv, SparkException}
+import scala.collection.mutable
+
+/**
+ * Custom serializer used for generic spark records. If the user registers the schemas
+ * ahead of time, then the schema's fingerprint will be sent with each message instead of the actual
+ * schema, as to reduce network IO.
+ * Actions like parsing or compressing schemas are computationally expensive so the serializer
+ * caches all previously seen values as to reduce the amount of work needed to do.
+ * @param schemas a map where the keys are unique IDs for spark schemas and the values are the
+ *                string representation of the Avro schema, used to decrease the amount of data
+ *                that needs to be serialized.
+ */
+class SparkStructTypeSerializer(schemas: Map[Long, StructType]) extends KSerializer[HoodieSparkRecord] {

Review Comment:
   Sorry, my bad i wasn't clear enough -- we will have to
   
   - Implement Registrar to make sure it does register our custom serializer
   - Make sure we update the docs to include it (and make sure to highlight it in the change-log), similarly to how we recommend including `spark.serializer` config



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r976598130


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -291,59 +284,51 @@ public void checkState() {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
-
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;

Review Comment:
   However, this method is strong related to data types. RowData, Arrow also need different implement. So I put it into HoodieRecord interface.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1230486756

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1234243210

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 1af6f84fe01384f08f05a8cc95fb711855c37eb6 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070) 
   * 17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1234322444

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 1af6f84fe01384f08f05a8cc95fb711855c37eb6 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070) 
   * 17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089) 
   * 87eeee945845062be0b1d4d31b26032e7fc0eb5a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1227644964

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 5165092d2dca99c4e684d76811ff3d38ca0ee049 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922) 
   * 64ddff55a9f3083be754e0951bf5f082fecca9e5 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r974403023


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordCompatibilityInterface.java:
##########
@@ -18,21 +18,28 @@
 
 package org.apache.hudi.common.model;
 
+import java.io.IOException;
+import java.util.Properties;
 import org.apache.avro.Schema;
 import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.BaseKeyGenerator;
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.util.Properties;
+public interface HoodieRecordCompatibilityInterface {
 
-/**
- * HoodieMerge defines how to merge two records. It is a stateless component.
- * It can implement the merging logic of HoodieRecord of different engines
- * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
- */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+  /**
+   * This method used to extract HoodieKey not through keyGenerator.
+   */
+  HoodieRecord wrapIntoHoodieRecordPayloadWithParams(

Review Comment:
   👍 



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java:
##########
@@ -147,19 +141,19 @@ public static HoodieMergedLogRecordScanner.Builder newBuilder() {
   }
 
   @Override
-  protected void processNextRecord(HoodieRecord hoodieRecord) throws IOException {
+  protected <T> void processNextRecord(HoodieRecord<T> hoodieRecord) throws IOException {
     String key = hoodieRecord.getRecordKey();
     if (records.containsKey(key)) {
       // Merge and store the merged record. The HoodieRecordPayload implementation is free to decide what should be
       // done when a DELETE (empty payload) is encountered before or after an insert/update.
 
-      HoodieRecord<? extends HoodieRecordPayload> oldRecord = records.get(key);
-      HoodieRecordPayload oldValue = oldRecord.getData();
-      HoodieRecordPayload combinedValue = (HoodieRecordPayload) merge.preCombine(oldRecord, hoodieRecord).getData();
+      HoodieRecord<T> oldRecord = records.get(key);
+      T oldValue = oldRecord.getData();
+      T combinedValue = ((HoodieRecord<T>) recordMerger.merge(oldRecord, hoodieRecord, readerSchema, this.hoodieTableMetaClient.getTableConfig().getProps()).get()).getData();
       // If combinedValue is oldValue, no need rePut oldRecord
       if (combinedValue != oldValue) {
-        HoodieOperation operation = hoodieRecord.getOperation();
-        records.put(key, new HoodieAvroRecord<>(new HoodieKey(key, hoodieRecord.getPartitionPath()), combinedValue, operation));
+        hoodieRecord.setData(combinedValue);

Review Comment:
   Why are we resetting the data instead of using new `HoodieRecord` returned by the Merger?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -126,11 +129,17 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. "
           + "This will render any value set for PRECOMBINE_FIELD_OPT_VAL in-effective");
 
-  public static final ConfigProperty<String> MERGE_CLASS_NAME = ConfigProperty
-      .key("hoodie.datasource.write.merge.class")
-      .defaultValue(HoodieAvroRecordMerge.class.getName())
-      .withDocumentation("Merge class provide stateless component interface for merging records, and support various HoodieRecord "
-          + "types, such as Spark records or Flink records.");
+  public static final ConfigProperty<String> MERGER_IMPLS = ConfigProperty
+      .key("hoodie.datasource.write.merger.impls")
+      .defaultValue(HoodieAvroRecordMerger.class.getName())
+      .withDocumentation("List of HoodieMerger implementations constituting Hudi's merging strategy -- based on the engine used. "
+          + "These merger impls will filter by hoodie.datasource.write.merger.strategy "
+          + "Hudi will pick most efficient implementation to perform merging/combining of the records (during update, reading MOR table, etc)");
+
+  public static final ConfigProperty<String> MERGER_STRATEGY = ConfigProperty
+      .key("hoodie.datasource.write.merger.strategy")
+      .defaultValue(StringUtils.DEFAULT_MERGER_STRATEGY_UUID)

Review Comment:
   Let's move this to HoodieMerger, rather than `StringUtils` (we can do it in a follow-up)



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -156,11 +155,10 @@ public class HoodieTableConfig extends HoodieConfig {
       .withDocumentation("Payload class to use for performing compactions, i.e merge delta logs with current base file and then "
           + " produce a new base file.");
 
-  public static final ConfigProperty<String> MERGE_CLASS_NAME = ConfigProperty
-      .key("hoodie.compaction.merge.class")
-      .defaultValue(HoodieAvroRecordMerge.class.getName())
-      .withDocumentation("Merge class provide stateless component interface for merging records, and support various HoodieRecord "
-          + "types, such as Spark records or Flink records.");
+  public static final ConfigProperty<String> MERGER_STRATEGY = ConfigProperty
+      .key("hoodie.compaction.merger.strategy")
+      .defaultValue(StringUtils.DEFAULT_MERGER_STRATEGY_UUID)
+      .withDocumentation("Id of merger strategy.  Hudi will pick RecordMergers in hoodie.datasource.write.merger.impls which has the same merger strategy id");

Review Comment:
   nit: `HoodieRecordMerger` implementations



##########
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##########
@@ -244,15 +242,24 @@ public class HoodieTableConfig extends HoodieConfig {
 
   private static final String TABLE_CHECKSUM_FORMAT = "%s.%s"; // <database_name>.<table_name>
 
-  public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) {
+  public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName, String mergerStrategy) {

Review Comment:
   nit: `mergerStrategyId`



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.spark.sql.HoodieCatalystExpressionUtils$;
+import org.apache.spark.sql.HoodieUnsafeRowUtils;
+import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();
+    final String partitionPath = (partitionName.isPresent() ? partitionName.get() :
+        getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString());
+
+    HoodieOperation operation = withOperationField
+        ? HoodieOperation.fromName(getNullableValAsString(structType, data, HoodieRecord.OPERATION_METADATA_FIELD)) : null;
+    return new HoodieSparkRecord(new HoodieKey(recKey, partitionPath), data, structType, operation);
+  }
+
+  private static Object getValue(StructType structType, String fieldName, InternalRow row) {
+    NestedFieldPath posList = HoodieInternalRowUtils.getCachedPosList(structType, fieldName);
+    return HoodieUnsafeRowUtils.getNestedInternalRowValue(row, posList);
+  }
+
+  /**
+   * Returns the string value of the given record {@code rec} and field {@code fieldName}. The field and value both could be missing.
+   *
+   * @param row       The record
+   * @param fieldName The field name
+   * @return the string form of the field or empty if the schema does not contain the field name or the value is null
+   */
+  private static Option<String> getNullableValAsString(StructType structType, InternalRow row, String fieldName) {

Review Comment:
   I don't think we need this method (we can use `getValue(...).toString` instead)



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -291,59 +284,51 @@ public void checkState() {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
-
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;
 
   /**
-   * Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields.
+   * Rewrite record into new schema(add meta columns)
    */
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException;
+  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException;
 
-  public abstract HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException;
+  /**
+   * Support schema evolution.
+   */
+  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map<String, String> renameCols) throws IOException;
 
-  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException;
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema) throws IOException {
+    return rewriteRecordWithNewSchema(recordSchema, props, newSchema, Collections.emptyMap());
+  }
 
-  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols, Mapper mapper) throws IOException;
+  /**
+   * This method could change in the future.
+   * @temporary
+   */
+  public abstract HoodieRecord updateValues(Schema recordSchema, Properties props, Map<String, String> metadataValues) throws IOException;
 
-  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException;
+  public abstract boolean isDelete(Schema schema, Properties props) throws IOException;
 
-  public abstract HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException;
+  /**
+   * Is EmptyRecord. Generated by ExpressionPayload.
+   */
+  public abstract boolean shouldIgnore(Schema schema, Properties props) throws IOException;
 
-  public abstract HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException;
+  public abstract Option<HoodieAvroIndexedRecord> toIndexedRecord(Schema schema, Properties props) throws IOException;

Review Comment:
   Let's move this method to `HoodieCompatibilityInterface`



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieAvroRecord.java:
##########
@@ -20,33 +20,27 @@
 package org.apache.hudi.common.model;
 
 import org.apache.hudi.avro.HoodieAvroUtils;
-import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.util.HoodieRecordUtils;
+import org.apache.hudi.common.util.ConfigUtils;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.ValidationUtils;
+import org.apache.hudi.common.util.collection.Pair;
 import org.apache.hudi.keygen.BaseKeyGenerator;
 
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.IndexedRecord;
 
-import javax.annotation.Nonnull;
-
 import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Properties;
 
-import static org.apache.hudi.common.util.TypeUtils.unsafeCast;
-
 public class HoodieAvroRecord<T extends HoodieRecordPayload> extends HoodieRecord<T> {

Review Comment:
   We should rename this to be `HoodieLegacyAvroRecord` (to make it more clear that this will eventually be going away)



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.spark.sql.HoodieCatalystExpressionUtils$;
+import org.apache.spark.sql.HoodieUnsafeRowUtils;
+import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();

Review Comment:
   We can't do that, we need to use key-gen to fetch both record-key and partition-path



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.spark.sql.HoodieCatalystExpressionUtils$;
+import org.apache.spark.sql.HoodieUnsafeRowUtils;
+import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();
+    final String partitionPath = (partitionName.isPresent() ? partitionName.get() :
+        getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString());
+
+    HoodieOperation operation = withOperationField
+        ? HoodieOperation.fromName(getNullableValAsString(structType, data, HoodieRecord.OPERATION_METADATA_FIELD)) : null;
+    return new HoodieSparkRecord(new HoodieKey(recKey, partitionPath), data, structType, operation);
+  }
+
+  private static Object getValue(StructType structType, String fieldName, InternalRow row) {
+    NestedFieldPath posList = HoodieInternalRowUtils.getCachedPosList(structType, fieldName);
+    return HoodieUnsafeRowUtils.getNestedInternalRowValue(row, posList);
+  }
+
+  /**
+   * Returns the string value of the given record {@code rec} and field {@code fieldName}. The field and value both could be missing.
+   *
+   * @param row       The record
+   * @param fieldName The field name
+   * @return the string form of the field or empty if the schema does not contain the field name or the value is null
+   */
+  private static Option<String> getNullableValAsString(StructType structType, InternalRow row, String fieldName) {
+    String fieldVal = !HoodieCatalystExpressionUtils$.MODULE$.existField(structType, fieldName)
+        ? null : StringUtils.objToString(getValue(structType, fieldName, row));
+    return Option.ofNullable(fieldVal);
+  }
+
+  /**
+   * Gets record column values into one object.
+   *
+   * @param row  InternalRow.
+   * @param columns Names of the columns to get values.
+   * @param structType  {@link StructType} instance.
+   * @return Column value if a single column, or concatenated String values by comma.
+   */
+  public static Object getRecordColumnValues(InternalRow row,

Review Comment:
   Please check my other comment regarding making this method return an array of objects, instead of concatenated string



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -291,59 +284,51 @@ public void checkState() {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
-
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;

Review Comment:
   We should not have this method in the `HoodieRecord` -- this method should be implemented w/in `HoodieRecordMerger` itself



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -213,6 +200,10 @@ public HoodieRecord setCurrentLocation(HoodieRecordLocation location) {
     return this;
   }
 
+  public void setData(T data) {

Review Comment:
   We should avoid any mutating methods in the interface



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.client.model.HoodieInternalRow;
+import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.ConfigUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.spark.sql.HoodieCatalystExpressionUtils$;
+import org.apache.spark.sql.HoodieUnsafeRowUtils;
+import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  private StructType structType = null;

Review Comment:
   We should make this `transient` to make sure we don't accidentally serialize schema along w/ every record



##########
hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileReaderFactory.java:
##########
@@ -20,18 +20,41 @@
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.io.hfile.CacheConfig;
+
 import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.exception.HoodieException;
 
 import java.io.IOException;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 
 import static org.apache.hudi.common.model.HoodieFileFormat.HFILE;
 import static org.apache.hudi.common.model.HoodieFileFormat.ORC;
 import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET;
 
 public class HoodieFileReaderFactory {
 
-  public static HoodieAvroFileReader getFileReader(Configuration conf, Path path) throws IOException {
+  public static HoodieFileReaderFactory getReaderFactory(HoodieRecord.HoodieRecordType recordType) {
+    switch (recordType) {
+      case AVRO:
+        return HoodieAvroFileReaderFactory.getFileReaderFactory();
+      case SPARK:
+        try {
+          Class<?> clazz = ReflectionUtils.getClass("org.apache.hudi.io.storage.HoodieSparkFileReaderFactory");
+          Method method = clazz.getMethod("getFileReaderFactory", null);

Review Comment:
   Few nits:
    - We don't need to make FileFactory a singleton, we can instantiate it every time
    - That makes this much simpler we can just call `newInstance`



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.client.model.HoodieInternalRow;
+import org.apache.hudi.common.model.HoodieAvroIndexedRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.ConfigUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.spark.sql.HoodieCatalystExpressionUtils$;
+import org.apache.spark.sql.HoodieUnsafeRowUtils;
+import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.UnsafeProjection;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Properties;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  private StructType structType = null;
+  private Option<Long> schemaFingerPrint = Option.empty();
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    initSchema(schema);
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    initSchema(schema);
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation) {
+    super(key, data, operation);
+    initSchema(schema);
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    initSchema(record.getStructType());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, getStructType(), op);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, getStructType());
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get())
+        .getRecordKey(data, getStructType()).toString() : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    DataType dataType = getStructType().apply(keyFieldName).dataType();
+    int pos = getStructType().fieldIndex(keyFieldName);
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, getStructType(), consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException {
+    StructType otherStructType = ((HoodieSparkRecord) other).getStructType();
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, getStructType(), (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    UTF8String[] metaFields = extractMetaField(targetStructType);
+    if (metaFields.length == 0) {
+      throw new UnsupportedOperationException();
+    }
+
+    InternalRow resultRow;
+    if (extractMetaField(getStructType()).length == 0) {
+      resultRow = new HoodieInternalRow(metaFields, data, false);
+    } else {
+      resultRow = new HoodieInternalRow(metaFields, data, true);
+    }
+
+    return new HoodieSparkRecord(getKey(), resultRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, getStructType(), newStructType, renameCols);
+    UnsafeProjection unsafeConvert = HoodieInternalRowUtils.getCachedUnsafeConvert(newStructType);
+    InternalRow resultRow = unsafeConvert.apply(rewriteRow);
+    UTF8String[] metaFields = extractMetaField(newStructType);
+    if (metaFields.length > 0) {
+      resultRow = new HoodieInternalRow(metaFields, data, true);
+    }
+
+    return new HoodieSparkRecord(getKey(), resultRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord updateValues(Schema recordSchema, Properties props, Map<String, String> metadataValues) throws IOException {
+    metadataValues.forEach((key, value) -> {
+      int pos = getStructType().fieldIndex(key);
+      if (value != null) {
+        data.update(pos, CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+
+    return new HoodieSparkRecord(getKey(), data, getStructType(), getOperation());
+  }
+
+  @Override
+  public boolean isDelete(Schema schema, Properties props) throws IOException {
+    if (null == data) {
+      return true;
+    }
+    if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) == null) {
+      return false;
+    }
+    Object deleteMarker = data.get(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).pos(), BooleanType);
+    return deleteMarker instanceof Boolean && (boolean) deleteMarker;
+  }
+
+  @Override
+  public boolean shouldIgnore(Schema schema, Properties props) throws IOException {
+    if (data != null && data.equals(SENTINEL)) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public HoodieRecord wrapIntoHoodieRecordPayloadWithParams(
+      Schema schema, Properties props,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Boolean populateMetaFields) {
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(getStructType(), data, withOperation);

Review Comment:
   Let's move this methods in this class and make them private to limit their access (they should also be deprecated/removed after we deprecate this method)



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -505,7 +514,9 @@ public class HoodieWriteConfig extends HoodieConfig {
   private HoodieMetadataConfig metadataConfig;
   private HoodieMetastoreConfig metastoreConfig;
   private HoodieCommonConfig commonConfig;
+  private HoodieStorageConfig storageConfig;
   private EngineType engineType;
+  private HoodieRecordMerger recordMerger;

Review Comment:
   I don't think we need to hold `recordMerger` -- we should instantiate it on the fly



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkStructTypeSerializer.scala:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.spark.sql.hudi
+
+import com.esotericsoftware.kryo.Kryo
+import com.esotericsoftware.kryo.io.{Input, Output}
+import com.twitter.chill.KSerializer
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+import java.nio.ByteBuffer
+import java.nio.charset.StandardCharsets
+import org.apache.avro.SchemaNormalization
+import org.apache.commons.io.IOUtils
+import org.apache.hudi.commmon.model.HoodieSparkRecord
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+import org.apache.spark.{SparkEnv, SparkException}
+import scala.collection.mutable
+
+/**
+ * Custom serializer used for generic spark records. If the user registers the schemas
+ * ahead of time, then the schema's fingerprint will be sent with each message instead of the actual
+ * schema, as to reduce network IO.
+ * Actions like parsing or compressing schemas are computationally expensive so the serializer
+ * caches all previously seen values as to reduce the amount of work needed to do.
+ * @param schemas a map where the keys are unique IDs for spark schemas and the values are the
+ *                string representation of the Avro schema, used to decrease the amount of data
+ *                that needs to be serialized.
+ */
+class SparkStructTypeSerializer(schemas: Map[Long, StructType]) extends KSerializer[HoodieSparkRecord] {

Review Comment:
   Let's not forget that we most importantly need this serializer to be registered w/ Spark:
   https://spark.incubator.apache.org/docs/0.6.0/tuning.html#data-serialization



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkStructTypeSerializer.scala:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.spark.sql.hudi
+
+import com.esotericsoftware.kryo.Kryo
+import com.esotericsoftware.kryo.io.{Input, Output}
+import com.twitter.chill.KSerializer
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+import java.nio.ByteBuffer
+import java.nio.charset.StandardCharsets
+import org.apache.avro.SchemaNormalization
+import org.apache.commons.io.IOUtils
+import org.apache.hudi.commmon.model.HoodieSparkRecord
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+import org.apache.spark.{SparkEnv, SparkException}
+import scala.collection.mutable
+
+/**
+ * Custom serializer used for generic spark records. If the user registers the schemas
+ * ahead of time, then the schema's fingerprint will be sent with each message instead of the actual
+ * schema, as to reduce network IO.
+ * Actions like parsing or compressing schemas are computationally expensive so the serializer
+ * caches all previously seen values as to reduce the amount of work needed to do.
+ * @param schemas a map where the keys are unique IDs for spark schemas and the values are the
+ *                string representation of the Avro schema, used to decrease the amount of data
+ *                that needs to be serialized.
+ */
+class SparkStructTypeSerializer(schemas: Map[Long, StructType]) extends KSerializer[HoodieSparkRecord] {

Review Comment:
   This is rather `HoodieSparkRecordSerializer`



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {

Review Comment:
   @wzx140 i think we might have missed this comment



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -30,9 +34,19 @@
  * It can implement the merging logic of HoodieRecord of different engines
  * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
  */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
+public interface HoodieRecordMerger extends Serializable {
+
+  /**
+   * This method converges combineAndGetUpdateValue and precombine from HoodiePayload.
+   * It'd be associative operation: f(a, f(b, c)) = f(f(a, b), c) (which we can translate as having 3 versions A, B, C
+   * of the single record, both orders of operations applications have to yield the same result)
+   */
+  Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;

Review Comment:
   @vinothchandar we've touched upon this w/ @prasannarajaperumal recently:
   
    - Initial take is that we're planning to have this method involved only when 2 records are merged (deletion is sub-type of merge, where second record is sentinel). Insertions will bypass this method
    - Your concern regarding users who have custom logic in `getInsertValue` is valid, but we don't want to overload the API out the gate and want actually to start w/ a simple API and increase complexity as we get more signals in terms of the other ways people are using (if they do)



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1199313520

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 652a0d666fe29487d3ce2c2ce1cef70dc443dd61 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302) 
   * 279857485f18875cab94f72b5bf61522bdaecd31 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] minihippo commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
minihippo commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231814490

   > > How much gains do we get after the patch, feel nervous about this change, let's pay more attention about stability first.
   > 
   > @danny0405 The current benchmark result of RFC-46:
   > 
   > * no overhead introduced of avro log read/write with new merger
   > * Compare with avro record in reading/writing parquet log
   >   
   >   * for read, spark record improves ~33% (after e2e duration-before e2e duration)*before
   >   * for write, spark record improves 7%~9%. In the stage that transforms input to hoodie record, it improves ~25%. But the shuffle data becomes bigger that before. It costs more in the following shuffle read stage.
   > 
   > All based on spark batch write/read
   
   After adding map type to test dataset, we find that the e2e of writing improves 27%.
   - with further analysis,  interalRow2parquet improves 40%, which matches with 50% CPU cost in avro2parquet  got by flame graph.
   - The relationship between performance improvement and complex column num is not nonlinear. That means even if increase the complex column, the income will not increase.
   
   To reproduce the benchmark result, I add the simple one to the test part


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1225644286

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a80d4bdd93c349b09b6e640dd2229379f2173ff0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661) 
   * 819824dcc83e97a7a36dab27cb2f877c113de4c6 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1202718539

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a7e980d34598ccee7262b93b439f92c98e78c0c1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498) 
   * 0ed3f481f20ae3420e2d3d71e320a2f0992b963a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1180750871

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 5f47e8e08a77e72c0916868e296e80d72fcd7d18 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845) 
   * d12f5116645664470119b8c79787c2a7da5feb87 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850) 
   * 02f8bdabc35820fa3765e77b154de6478440739d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r975549775


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.spark.sql.HoodieCatalystExpressionUtils$;
+import org.apache.spark.sql.HoodieUnsafeRowUtils;
+import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();
+    final String partitionPath = (partitionName.isPresent() ? partitionName.get() :
+        getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString());
+
+    HoodieOperation operation = withOperationField
+        ? HoodieOperation.fromName(getNullableValAsString(structType, data, HoodieRecord.OPERATION_METADATA_FIELD)) : null;
+    return new HoodieSparkRecord(new HoodieKey(recKey, partitionPath), data, structType, operation);
+  }
+
+  private static Object getValue(StructType structType, String fieldName, InternalRow row) {
+    NestedFieldPath posList = HoodieInternalRowUtils.getCachedPosList(structType, fieldName);
+    return HoodieUnsafeRowUtils.getNestedInternalRowValue(row, posList);
+  }
+
+  /**
+   * Returns the string value of the given record {@code rec} and field {@code fieldName}. The field and value both could be missing.
+   *
+   * @param row       The record
+   * @param fieldName The field name
+   * @return the string form of the field or empty if the schema does not contain the field name or the value is null
+   */
+  private static Option<String> getNullableValAsString(StructType structType, InternalRow row, String fieldName) {
+    String fieldVal = !HoodieCatalystExpressionUtils$.MODULE$.existField(structType, fieldName)
+        ? null : StringUtils.objToString(getValue(structType, fieldName, row));
+    return Option.ofNullable(fieldVal);
+  }
+
+  /**
+   * Gets record column values into one object.
+   *
+   * @param row  InternalRow.
+   * @param columns Names of the columns to get values.
+   * @param structType  {@link StructType} instance.
+   * @return Column value if a single column, or concatenated String values by comma.
+   */
+  public static Object getRecordColumnValues(InternalRow row,

Review Comment:
   We should also change HoodieAvroUtils#getRecordColumnValues, right?



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r975617095


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.spark.sql.HoodieCatalystExpressionUtils$;
+import org.apache.spark.sql.HoodieUnsafeRowUtils;
+import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();
+    final String partitionPath = (partitionName.isPresent() ? partitionName.get() :
+        getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString());
+
+    HoodieOperation operation = withOperationField
+        ? HoodieOperation.fromName(getNullableValAsString(structType, data, HoodieRecord.OPERATION_METADATA_FIELD)) : null;
+    return new HoodieSparkRecord(new HoodieKey(recKey, partitionPath), data, structType, operation);
+  }
+
+  private static Object getValue(StructType structType, String fieldName, InternalRow row) {
+    NestedFieldPath posList = HoodieInternalRowUtils.getCachedPosList(structType, fieldName);
+    return HoodieUnsafeRowUtils.getNestedInternalRowValue(row, posList);
+  }
+
+  /**
+   * Returns the string value of the given record {@code rec} and field {@code fieldName}. The field and value both could be missing.
+   *
+   * @param row       The record
+   * @param fieldName The field name
+   * @return the string form of the field or empty if the schema does not contain the field name or the value is null
+   */
+  private static Option<String> getNullableValAsString(StructType structType, InternalRow row, String fieldName) {
+    String fieldVal = !HoodieCatalystExpressionUtils$.MODULE$.existField(structType, fieldName)
+        ? null : StringUtils.objToString(getValue(structType, fieldName, row));
+    return Option.ofNullable(fieldVal);
+  }
+
+  /**
+   * Gets record column values into one object.
+   *
+   * @param row  InternalRow.
+   * @param columns Names of the columns to get values.
+   * @param structType  {@link StructType} instance.
+   * @return Column value if a single column, or concatenated String values by comma.
+   */
+  public static Object getRecordColumnValues(InternalRow row,

Review Comment:
   @wzx140 ideally we should, but we don't want to make this PR to fix everything either. Hence i suggest to only focus this for the new API and then we can follow-up and fix it for `HoodieAvroUtils`



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r975768738


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -505,7 +514,9 @@ public class HoodieWriteConfig extends HoodieConfig {
   private HoodieMetadataConfig metadataConfig;
   private HoodieMetastoreConfig metastoreConfig;
   private HoodieCommonConfig commonConfig;
+  private HoodieStorageConfig storageConfig;
   private EngineType engineType;
+  private HoodieRecordMerger recordMerger;

Review Comment:
   Let's avoid premature optimizations -- `RecordMerger` is specifically designed as stateless component and as such is lightweight enough to be initialized on-demand. Only thing we need to take care of is just making sure we don't init it for every record.
   
   The problem with caching it inside `HoodieWriteConfig` is that HoodieWriteConfig has nothing to do w/ RecordMerger and we should not open this can of warm when it's used as a state carriage for other components.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1199233077

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 652a0d666fe29487d3ce2c2ce1cef70dc443dd61 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302) 
   * 279857485f18875cab94f72b5bf61522bdaecd31 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] vinothchandar commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r936192827


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -30,9 +34,19 @@
  * It can implement the merging logic of HoodieRecord of different engines
  * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
  */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
+public interface HoodieRecordMerger extends Serializable {
+
+  /**
+   * This method converges combineAndGetUpdateValue and precombine from HoodiePayload.
+   * It'd be associative operation: f(a, f(b, c)) = f(f(a, b), c) (which we can translate as having 3 versions A, B, C
+   * of the single record, both orders of operations applications have to yield the same result)
+   */
+  Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;

Review Comment:
   For inserts, deletes - we will not be calling merge()? i.e `older` and `newer` will always be non-null? Something to think about is - whether we would want the user to be able to get call backs for those as well



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -19,6 +19,10 @@
 package org.apache.hudi.common.model;
 
 import org.apache.avro.Schema;

Review Comment:
   can we introduce a `HoodieSchema` class hierarchy. and make the API depend on that. Based on context, we would provide HoodieAvroSchema or HoodieSparkSchema etc?



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -30,9 +34,19 @@
  * It can implement the merging logic of HoodieRecord of different engines
  * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
  */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
+public interface HoodieRecordMerger extends Serializable {
+
+  /**
+   * This method converges combineAndGetUpdateValue and precombine from HoodiePayload.
+   * It'd be associative operation: f(a, f(b, c)) = f(f(a, b), c) (which we can translate as having 3 versions A, B, C
+   * of the single record, both orders of operations applications have to yield the same result)
+   */
+  Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
 
-  Option<HoodieRecord> combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
+  /**
+   * The record type handled by the current merger.
+   * SPARK, AVRO, FLINK
+   */
+  HoodieRecordType getRecordType();

Review Comment:
   Imagine I am a Hudi user at Acme Inc. how do I write one merger implementation e.g `AcmeCDCMerger implements HoodieRecordMerger`, and have it be used across engines. IIUC here - we configure one implementation and have it be associated to one record type? 



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -124,12 +126,17 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. "
           + "This will render any value set for PRECOMBINE_FIELD_OPT_VAL in-effective");
 
-  public static final ConfigProperty<String> MERGE_CLASS_NAME = ConfigProperty
+  public static final ConfigProperty<String> MERGER_CLASS_NAME = ConfigProperty

Review Comment:
   I wonder if we can simplify all this by having a table property that takes a list of merger class names.  Then we can call `getRecordType()` to pick the optimal merge implementation? But high level should n't this be a table config like before.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1190050427

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * f482ae2541c0de736c021466bace00ce85ccf625 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083) 
   * 11cc4bffb3c49d2f6580e72c2a599bb9a174824e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r924541311


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriterFactory.java:
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.io.storage;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.config.HoodieConfig;
+import org.apache.hudi.common.config.HoodieStorageConfig;
+import org.apache.hudi.common.engine.TaskContextSupplier;
+import org.apache.hudi.common.table.HoodieTableConfig;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.exception.HoodieIOException;
+import org.apache.hudi.io.storage.row.HoodieRowParquetConfig;
+import org.apache.hudi.io.storage.row.HoodieRowParquetWriteSupport;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.parquet.hadoop.metadata.CompressionCodecName;
+
+import java.io.IOException;
+
+public class HoodieSparkFileWriterFactory extends HoodieFileWriterFactory {
+
+  private static class SingletonHolder {
+
+    private static final HoodieSparkFileWriterFactory INSTANCE = new HoodieSparkFileWriterFactory();
+  }
+
+  public static HoodieFileWriterFactory getFileWriterFactory() {
+    return HoodieSparkFileWriterFactory.SingletonHolder.INSTANCE;
+  }
+
+  @Override
+  protected HoodieFileWriter newParquetFileWriter(
+      String instantTime, Path path, Configuration conf, HoodieConfig config, Schema schema,
+      TaskContextSupplier taskContextSupplier) throws IOException {
+    boolean populateMetaFields = config.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS);
+    boolean enableBloomFilter = populateMetaFields;
+    Option<BloomFilter> filter = enableBloomFilter ? Option.of(createBloomFilter(config)) : Option.empty();
+    String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME);
+    // Support PARQUET_COMPRESSION_CODEC_NAME is ""
+    if (compressionCodecName.isEmpty()) {
+      compressionCodecName = null;
+    }
+    HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf,
+        HoodieInternalRowUtils.getCachedSchema(schema), filter.get(),
+        HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build());
+    HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport,
+        CompressionCodecName.fromConf(compressionCodecName),
+        config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE),
+        config.getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE),
+        config.getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE),
+        conf,
+        config.getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION),
+        config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED));
+    parquetConfig.getHadoopConf().addResource(writeSupport.getHadoopConf());
+
+    return new HoodieSparkParquetWriter(path, parquetConfig, instantTime, taskContextSupplier, config.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS));
+  }
+
+  protected HoodieFileWriter newParquetFileWriter(
+      FSDataOutputStream outputStream, Configuration conf, HoodieConfig config, Schema schema) throws IOException {
+    boolean populateMetaFields = true;
+    boolean enableBloomFilter = false;
+    BloomFilter filter = enableBloomFilter ? createBloomFilter(config) : null;
+    HoodieRowParquetWriteSupport writeSupport = new HoodieRowParquetWriteSupport(conf,
+        HoodieInternalRowUtils.getCachedSchema(schema), filter,
+        HoodieStorageConfig.newBuilder().fromProperties(config.getProps()).build());
+    String compressionCodecName = config.getStringOrDefault(HoodieStorageConfig.PARQUET_COMPRESSION_CODEC_NAME);
+    // Support PARQUET_COMPRESSION_CODEC_NAME is ""
+    if (compressionCodecName.isEmpty()) {
+      compressionCodecName = null;
+    }
+    HoodieRowParquetConfig parquetConfig = new HoodieRowParquetConfig(writeSupport,
+        CompressionCodecName.fromConf(compressionCodecName),
+        config.getInt(HoodieStorageConfig.PARQUET_BLOCK_SIZE),
+        config.getInt(HoodieStorageConfig.PARQUET_PAGE_SIZE),
+        config.getLong(HoodieStorageConfig.PARQUET_MAX_FILE_SIZE),
+        writeSupport.getHadoopConf(), config.getDouble(HoodieStorageConfig.PARQUET_COMPRESSION_RATIO_FRACTION),
+        config.getBooleanOrDefault(HoodieStorageConfig.PARQUET_DICTIONARY_ENABLED));
+    parquetConfig.getHadoopConf().addResource(writeSupport.getHadoopConf());

Review Comment:
   These two are used in writing base files and log files respectively



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.io.storage;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.util.BaseFileUtils;
+import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.common.util.ParquetReaderIterator;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.io.InputFile;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+public class HoodieSparkParquetReader implements HoodieSparkFileReader {
+
+  private final Path path;
+  private final Configuration conf;
+  private final BaseFileUtils parquetUtils;
+  private List<ParquetReaderIterator> readerIterators = new ArrayList<>();
+
+  public HoodieSparkParquetReader(Configuration conf, Path path) {
+    this.path = path;
+    this.conf = conf;
+    this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    return parquetUtils.readMinMaxRecordKeys(conf, path);
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    return parquetUtils.readBloomFilterFromMetadata(conf, path);
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set<String> candidateRowKeys) {
+    return parquetUtils.filterRowKeys(conf, path, candidateRowKeys);
+  }
+
+  @Override
+  public ClosableIterator<InternalRow> getInternalRowIterator(Schema schema) throws IOException {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA(), structType.json());
+    // todo: get it from spark context
+    conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING().key(),false);
+    conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), true);

Review Comment:
   fixed



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r942820533


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -20,62 +20,33 @@
 package org.apache.hudi.common.table.log;
 
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
-import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.stream.StreamSupport;
+import java.util.Properties;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
-public class HoodieFileSliceReader<T extends HoodieRecordPayload> implements Iterator<HoodieRecord<T>> {
+public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().getKeyWithParams(schema, props,

Review Comment:
   > For old records read from FileReader and passing to HoodieMergeHanle, we do not need to extract HoodieKey.
   HoodieMergeHanle use keyGeneratorOpt to extract recordKey directly.
   
   Sorry, not sure i fully understand your point. Can you please elaborate?
   
   But even if we don't need to extract they keys in all cases we can still solve it by configuring the `FileReader` instead of adding this method, right?



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1228004890

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 64ddff55a9f3083be754e0951bf5f082fecca9e5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953) 
   * 858a47a5b106462a5089ecf77278196bc7c7a0a8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1246206686

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097",
       "triggerID" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11342",
       "triggerID" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 5002e131469f9587103c5ca39b322e9470eba79a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097) 
   * 6cd6761a77f288c15508f286a5294491026ffb81 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11342) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r974857657


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -291,59 +284,51 @@ public void checkState() {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
-
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;

Review Comment:
   What this method does is to stitch two record. The logic of bootstrap merge is fixed. We should not let users customize the implementation of this method, right?



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r959122392


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -290,59 +286,59 @@ public void checkState() {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Rewrite record into new schema(add meta columns)
+   */
+  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException;
 
   /**
-   * Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields.
+   * Support schema evolution.
    */
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException;
+  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map<String, String> renameCols) throws IOException;
 
-  public abstract HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException;
+  public abstract HoodieRecord updateValues(Schema recordSchema, Properties props, Map<String, String> metadataValues) throws IOException;

Review Comment:
   HoodieRecord#rewriteRecord add meta field in incoming record and HoodieRecord#updateValues update the meta field value. This seems inevitable.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1230576501

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017) 
   * a149effad0c1992bd717e65208fbcef601deeb97 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231485317

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 4ab845a4c27a678876f33abf8196bf760df53408 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038) 
   * faf16a7e76db8efc6b23be7a8362c4fe35496556 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042) 
   * 4ee2969401d4db019780389b2898526f6c99fd86 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r944089455


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -20,62 +20,33 @@
 package org.apache.hudi.common.table.log;
 
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
-import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.stream.StreamSupport;
+import java.util.Properties;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
-public class HoodieFileSliceReader<T extends HoodieRecordPayload> implements Iterator<HoodieRecord<T>> {
+public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().getKeyWithParams(schema, props,

Review Comment:
   It's hard to determine a good method name. Backward compatibility(Converting avro data to HoodieRecordPayload) makes the meaning of this method seem ambiguous. Do you have any good suggestions for the name of this method?
   
   Sorry, I did not express clearing. "processed data" means "transformed data" contrary to "original data". Some callers of FileReader prefer to use  "original data" that directly read from file. Some callers of FileReader prefer to use "transformed data" processed by getKeyWithParams. 



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1195937698

   @danny0405 
   
   > I agree we should decouple Hudi from Avro, but that does not mean we should lean back to engine-specific data structures which is very hard to maintain as a engine neutral project, see how hard it is for hudi to integrate with a new engine now :),
   i kind of expect hudi's own reader/writer/data structures, which is the right direction we should elaborate with.
   
   I don't think we are aligned on this one: Hudi is and will be staying engine-neutral project. However for the top-of-the-line performance on *any* engine (to stay competitive with other formats) we *have to* use engine-specific representations (think `Row`, `RowData`, `ArrayWritable`, Arrow, etc). There's just no other way -- any intermediate representation will be a tax on performance, and general direction is that we want to provide best possible performance in any supported workload be it a a read or write.
   
   > And another concern i always have in my mind is hudi needs a stable release tooo much ! We can not make huge changes to core reader/writers now at this moment before we do enough tests/practice, and we should not rush in the code for just the reason of code rebase effort.
   
   Totally agree with you there, and it's one of the reasons why we decided that it's a good idea to take a more measured approach here and avoid pushing really hard (and compromising on quality testing) to meet 0.12 deadline.


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
danny0405 commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1193649030

   > 
   
   Not very persuaded by the improvement number: read 33% and write 9%, if the number is real and can be re-productive, i would suggest to lower priority of the patch, for example, after release 1.0.0.
   
   I had expected about 5x ~ 10x performance improvement, BTW.


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1179120702

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758) 
   * 42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r917632197


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {

Review Comment:
   changed all prop to props



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r917646430


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    Option<SparkKeyGeneratorInterface> keyGeneratorOpt = Option.empty();
+    if (useKeygen && !Boolean.parseBoolean(prop.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        keyGeneratorOpt = Option.of((SparkKeyGeneratorInterface) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(prop)));
+      } catch (IOException e) {
+        throw new HoodieException("Only SparkKeyGeneratorInterface are supported when meta columns are disabled ", e);
+      }
+    }
+    String key = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(data, structType)
+        : data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString();
+    String partition = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getPartitionPath(data, structType)
+        : data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString();
+    this.key = new HoodieKey(key, partition);
+
+    return this;
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+
+  @Override
+  public boolean isPresent(Schema schema, Properties prop) throws IOException {
+    if (null == data) {
+      return false;
+    }
+    if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) == null) {
+      return true;
+    }
+    Object deleteMarker = data.get(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).pos(), BooleanType);
+    return !(deleteMarker instanceof Boolean && (boolean) deleteMarker);
+  }
+
+  @Override
+  public boolean shouldIgnore(Schema schema, Properties prop) throws IOException {
+    // TODO SENTINEL should refactor SENTINEL without Avro(GenericRecord)
+    if (null != data && data.equals(SENTINEL)) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Option<IndexedRecord> toIndexedRecord(Schema schema, Properties prop) throws IOException {

Review Comment:
   HoodieAvroRecord#toIndexedRecord could return option#empty



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1154624809

   @hudi-bot run azure


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r909622446


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java:
##########
@@ -132,7 +133,8 @@ public static List<String> filterKeysFromFile(Path filePath, List<String> candid
       // Load all rowKeys from the file, to double-confirm
       if (!candidateRecordKeys.isEmpty()) {
         HoodieTimer timer = new HoodieTimer().startTimer();
-        HoodieAvroFileReader fileReader = HoodieFileReaderFactory.getFileReader(configuration, filePath);
+        // Just get row keys

Review Comment:
   fixed



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
danny0405 commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1131641091

   A huge patch, may need some careful review :)


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r917619328


##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java:
##########
@@ -61,42 +57,43 @@ public static FlinkMergeHelper newInstance() {
   @Override
   public void runMerge(HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table,
                        HoodieMergeHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> mergeHandle) throws IOException {
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
     Schema readSchema;
+    Schema readerSchema;
+    Schema writerSchema;

Review Comment:
   will add some inline comments



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r920463727


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java:
##########
@@ -54,42 +63,64 @@
   private final SerializableSchema schema;
   private final HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy;
   private final HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType;
+  private final HoodieRecordType recordType;
 
   public RDDSpatialCurveSortPartitioner(HoodieSparkEngineContext sparkEngineContext,
-                                        String[] orderByColumns,
-                                        HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy,
-                                        HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType,
-                                        Schema schema) {
+      String[] orderByColumns,
+      LayoutOptimizationStrategy layoutOptStrategy,
+      SpatialCurveCompositionStrategyType curveCompositionStrategyType,
+      Schema schema, HoodieRecordType recordType) {
     this.sparkEngineContext = sparkEngineContext;
     this.orderByColumns = orderByColumns;
     this.layoutOptStrategy = layoutOptStrategy;
     this.curveCompositionStrategyType = curveCompositionStrategyType;
     this.schema = new SerializableSchema(schema);
+    this.recordType = recordType;
   }
 
   @Override
   public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions) {
-    JavaRDD<GenericRecord> genericRecordsRDD =
-        records.map(f -> (GenericRecord) f.toIndexedRecord(schema.get(), new Properties()).get());
-
-    Dataset<Row> sourceDataset =
-        AvroConversionUtils.createDataFrame(
-            genericRecordsRDD.rdd(),
-            schema.toString(),
-            sparkEngineContext.getSqlContext().sparkSession()
-        );
-
-    Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
-
-    return HoodieSparkUtils.createRdd(sortedDataset, schema.get().getName(), schema.get().getNamespace(), false, Option.empty())
-        .toJavaRDD()
-        .map(record -> {
-          String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
-          String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
-          HoodieKey hoodieKey = new HoodieKey(key, partition);
-          HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record));
-          return hoodieRecord;
-        });
+    if (recordType == HoodieRecordType.AVRO) {
+      JavaRDD<GenericRecord> genericRecordsRDD =
+          records.map(f -> (GenericRecord) f.toIndexedRecord(schema.get(), new Properties()).get());
+
+      Dataset<Row> sourceDataset =
+          AvroConversionUtils.createDataFrame(
+              genericRecordsRDD.rdd(),
+              schema.toString(),
+              sparkEngineContext.getSqlContext().sparkSession()
+          );
+
+      Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
+
+      return HoodieSparkUtils.createRdd(sortedDataset, schema.get().getName(), schema.get().getNamespace(), false, Option.empty())
+          .toJavaRDD()
+          .map(record -> {
+            String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+            String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
+            HoodieKey hoodieKey = new HoodieKey(key, partition);
+            HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record));
+            return hoodieRecord;
+          });
+    } else if (recordType == HoodieRecordType.SPARK) {
+      StructType structType = HoodieInternalRowUtils.getCachedSchema(schema.get());
+      Dataset<Row> sourceDataset = SparkConversionUtils.createDataFrame(records.rdd(), sparkEngineContext.getSqlContext().sparkSession(), structType);
+
+      Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
+
+      return sortedDataset.queryExecution().toRdd()
+          .toJavaRDD()
+          .map(row -> {
+            InternalRow internalRow = row.copy();

Review Comment:
   @xushiyan in short: you can't hold the reference to `InternalRow` outside the closure where you have access to it, since underlying Spark uses mutable buffer to reduce # of allocations to a minimum (in other words you always get the same IR object that gets reset every time iterator moves)



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r920465935


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java:
##########
@@ -54,42 +63,64 @@
   private final SerializableSchema schema;
   private final HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy;
   private final HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType;
+  private final HoodieRecordType recordType;
 
   public RDDSpatialCurveSortPartitioner(HoodieSparkEngineContext sparkEngineContext,
-                                        String[] orderByColumns,
-                                        HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy,
-                                        HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType,
-                                        Schema schema) {
+      String[] orderByColumns,
+      LayoutOptimizationStrategy layoutOptStrategy,
+      SpatialCurveCompositionStrategyType curveCompositionStrategyType,
+      Schema schema, HoodieRecordType recordType) {
     this.sparkEngineContext = sparkEngineContext;
     this.orderByColumns = orderByColumns;
     this.layoutOptStrategy = layoutOptStrategy;
     this.curveCompositionStrategyType = curveCompositionStrategyType;
     this.schema = new SerializableSchema(schema);
+    this.recordType = recordType;
   }
 
   @Override
   public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions) {
-    JavaRDD<GenericRecord> genericRecordsRDD =
-        records.map(f -> (GenericRecord) f.toIndexedRecord(schema.get(), new Properties()).get());
-
-    Dataset<Row> sourceDataset =
-        AvroConversionUtils.createDataFrame(
-            genericRecordsRDD.rdd(),
-            schema.toString(),
-            sparkEngineContext.getSqlContext().sparkSession()
-        );
-
-    Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
-
-    return HoodieSparkUtils.createRdd(sortedDataset, schema.get().getName(), schema.get().getNamespace(), false, Option.empty())
-        .toJavaRDD()
-        .map(record -> {
-          String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
-          String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
-          HoodieKey hoodieKey = new HoodieKey(key, partition);
-          HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record));
-          return hoodieRecord;
-        });
+    if (recordType == HoodieRecordType.AVRO) {
+      JavaRDD<GenericRecord> genericRecordsRDD =
+          records.map(f -> (GenericRecord) f.toIndexedRecord(schema.get(), new Properties()).get());
+
+      Dataset<Row> sourceDataset =
+          AvroConversionUtils.createDataFrame(
+              genericRecordsRDD.rdd(),
+              schema.toString(),
+              sparkEngineContext.getSqlContext().sparkSession()
+          );
+
+      Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
+
+      return HoodieSparkUtils.createRdd(sortedDataset, schema.get().getName(), schema.get().getNamespace(), false, Option.empty())
+          .toJavaRDD()
+          .map(record -> {
+            String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+            String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
+            HoodieKey hoodieKey = new HoodieKey(key, partition);
+            HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record));
+            return hoodieRecord;
+          });
+    } else if (recordType == HoodieRecordType.SPARK) {
+      StructType structType = HoodieInternalRowUtils.getCachedSchema(schema.get());
+      Dataset<Row> sourceDataset = SparkConversionUtils.createDataFrame(records.rdd(), sparkEngineContext.getSqlContext().sparkSession(), structType);
+
+      Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
+
+      return sortedDataset.queryExecution().toRdd()
+          .toJavaRDD()
+          .map(row -> {
+            InternalRow internalRow = row.copy();

Review Comment:
   @minihippo let's add a comment explaining why they copy is being made here



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1192259928

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133) 
   * c3f5e348484ed6f538113b04dc1e2d7166d43848 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1189869707

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284) 
   * f482ae2541c0de736c021466bace00ce85ccf625 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r903762871


##########
hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java:
##########
@@ -436,14 +444,18 @@ public static GenericRecord removeFields(GenericRecord record, List<String> fiel
 
   private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) {
     Schema oldSchema = oldRecord.getSchema();
-    Object fieldValue = oldSchema.getField(field.name()) == null ? null : oldRecord.get(field.name());
+    Field oldSchemaField = oldSchema.getField(field.name());
+    Object fieldValue = oldSchemaField == null ? null : oldRecord.get(field.name());
 
     if (fieldValue != null) {
       // In case field's value is a nested record, we have to rewrite it as well
       Object newFieldValue;
       if (fieldValue instanceof GenericRecord) {
         GenericRecord record = (GenericRecord) fieldValue;
-        newFieldValue = rewriteRecord(record, resolveUnionSchema(field.schema(), record.getSchema().getFullName()));
+        // May return null when use rewrite
+        String recordFullName = record.getSchema().getFullName();
+        String fullName = recordFullName != null ? recordFullName : oldSchemaField.name();
+        newFieldValue = rewriteRecord(record, resolveUnionSchema(field.schema(), fullName));

Review Comment:
   This unexpected situation is caused by changing org.apache.hudi.table.action.commit.AbstractMergeHelper#transformRecordBasedOnNewSchema to org.apache.hudi.common.model.HoodieRecord#rewriteRecord. Maybe org.apache.hudi.avro.HoodieAvroUtils#rewriteRecord cause fullName null.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1204978139

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * c4f8afd6d63100feba2a218ee4331fad5790ef69 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524) 
   * a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r938157656


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -124,12 +126,17 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. "
           + "This will render any value set for PRECOMBINE_FIELD_OPT_VAL in-effective");
 
-  public static final ConfigProperty<String> MERGE_CLASS_NAME = ConfigProperty
+  public static final ConfigProperty<String> MERGER_CLASS_NAME = ConfigProperty

Review Comment:
   We probably fine either way, but for simplicity i'd suggest we stick with our current approach of keeping it table-property:
   
   1. It's easier for users to understand (existing behavior)
   2. It stays part of the table's (immutable) configuration, avoiding the toll of needing to specify it explicitly with every write (could be error-prone)
   



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
danny0405 commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1202059975

   > I don't think we are aligned on this one: Hudi is and will be staying engine-neutral project. However for the top-of-the-line performance on _any_ engine (to stay competitive with other formats) we _have to_ use engine-specific representations (think `Row`, `RowData`, `ArrayWritable`, Arrow, etc). There's just no other way
   
   I do have some different thoughts, performance is on first priority if it is critical, say Hudi performs bad on some benchmark, but for the long run, i do think as a storage we should have our own data structures/reader writers/schema like every storage engine do, looks like how easy it is to a new engine to integrate with Iceberg and how hard it is for Hudi, the ease to integrate  is important for the ecosystem especially as a `format`.


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1202725902

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a7e980d34598ccee7262b93b439f92c98e78c0c1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498) 
   * 0ed3f481f20ae3420e2d3d71e320a2f0992b963a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522) 
   * c4f8afd6d63100feba2a218ee4331fad5790ef69 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1199619090

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 279857485f18875cab94f72b5bf61522bdaecd31 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1202871499

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 0ed3f481f20ae3420e2d3d71e320a2f0992b963a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522) 
   * c4f8afd6d63100feba2a218ee4331fad5790ef69 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231827852

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 4ee2969401d4db019780389b2898526f6c99fd86 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044) 
   * a7a1ff3ce32f1e7281eed9d42679cbff70a7d523 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050) 
   * 756a4a94fa739de42fd722a7e036563f3df60d27 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231982090

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 4ee2969401d4db019780389b2898526f6c99fd86 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044) 
   * a7a1ff3ce32f1e7281eed9d42679cbff70a7d523 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050) 
   * 756a4a94fa739de42fd722a7e036563f3df60d27 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055) 
   * f3330b66baa3e3abd2d2c5b238443359d707955b UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231329824

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a149effad0c1992bd717e65208fbcef601deeb97 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019) 
   * 4ab845a4c27a678876f33abf8196bf760df53408 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038) 
   * faf16a7e76db8efc6b23be7a8362c4fe35496556 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231649724

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 4ab845a4c27a678876f33abf8196bf760df53408 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038) 
   * faf16a7e76db8efc6b23be7a8362c4fe35496556 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042) 
   * 4ee2969401d4db019780389b2898526f6c99fd86 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044) 
   * a7a1ff3ce32f1e7281eed9d42679cbff70a7d523 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1234331607

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 1af6f84fe01384f08f05a8cc95fb711855c37eb6 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070) 
   * 17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089) 
   * 87eeee945845062be0b1d4d31b26032e7fc0eb5a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1234672908

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 87eeee945845062be0b1d4d31b26032e7fc0eb5a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092) 
   * 5002e131469f9587103c5ca39b322e9470eba79a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1234728323

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097",
       "triggerID" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 87eeee945845062be0b1d4d31b26032e7fc0eb5a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092) 
   * 5002e131469f9587103c5ca39b322e9470eba79a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1207633040

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 96701a705ec8c271438ff696c1451364b9b4398d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660) 
   * a80d4bdd93c349b09b6e640dd2229379f2173ff0 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r975552226


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.spark.sql.HoodieCatalystExpressionUtils$;
+import org.apache.spark.sql.HoodieUnsafeRowUtils;
+import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();

Review Comment:
   This func is the same logic as SpillableMapUtils#convertToHoodieRecordPayload. Should we change it?



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1251753429

   @alexeykudinkin I'm already rebased on master and add the config mergerStrategy with uuid. You can do final review.


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r975617746


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,111 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+
+import org.apache.spark.sql.HoodieCatalystExpressionUtils$;
+import org.apache.spark.sql.HoodieUnsafeRowUtils;
+import org.apache.spark.sql.HoodieUnsafeRowUtils.NestedFieldPath;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();

Review Comment:
   Interesting, this seems like a bug to me -- if `populateMetaFields` is false this won't work for non-SimpleKeyGen



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r976644654


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkStructTypeSerializer.scala:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.spark.sql.hudi
+
+import com.esotericsoftware.kryo.Kryo
+import com.esotericsoftware.kryo.io.{Input, Output}
+import com.twitter.chill.KSerializer
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+import java.nio.ByteBuffer
+import java.nio.charset.StandardCharsets
+import org.apache.avro.SchemaNormalization
+import org.apache.commons.io.IOUtils
+import org.apache.hudi.commmon.model.HoodieSparkRecord
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+import org.apache.spark.{SparkEnv, SparkException}
+import scala.collection.mutable
+
+/**
+ * Custom serializer used for generic spark records. If the user registers the schemas
+ * ahead of time, then the schema's fingerprint will be sent with each message instead of the actual
+ * schema, as to reduce network IO.
+ * Actions like parsing or compressing schemas are computationally expensive so the serializer
+ * caches all previously seen values as to reduce the amount of work needed to do.
+ * @param schemas a map where the keys are unique IDs for spark schemas and the values are the
+ *                string representation of the Avro schema, used to decrease the amount of data
+ *                that needs to be serialized.
+ */
+class SparkStructTypeSerializer(schemas: Map[Long, StructType]) extends KSerializer[HoodieSparkRecord] {

Review Comment:
   I found setting sparkconf spark.kryo.registrator -> mypackage.MyRegistrator not works. 
   ```scala
   // Make sure to set these properties *before* creating a SparkContext!
   System.setProperty("spark.serializer", "spark.KryoSerializer")
   System.setProperty("spark.kryo.registrator", "mypackage.MyRegistrator")
   val sc = new SparkContext(...)
   ``` 
   So we can not make this serializer to be registered w/ Spark in hudi project code. This Serializer is for ExternalSpillableMap to serialize HoodieSparkRecord.
   



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r977042462


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -291,59 +284,51 @@ public void checkState() {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
-
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;

Review Comment:
   Understood. Let's keep it for now, but just rename it to `joinWith` to avoid confusion



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] XuQianJin-Stars commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1184599116

   Is this PR going to be merged into 0.12?I feel this is too big. A refactoring of this magnitude would have an impact.


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1131163451

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c34256ac98d03c787d264e56e35a7058d4273442 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755) 
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1131212873

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] xushiyan commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
xushiyan commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r909038899


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -142,6 +146,11 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Easily configure one the built-in key generators, instead of specifying the key generator class."
           + "Currently supports SIMPLE, COMPLEX, TIMESTAMP, CUSTOM, NON_PARTITION, GLOBAL_DELETE");
 
+  public static final ConfigProperty<String> RECORD_TYPE = ConfigProperty
+      .key("hoodie.datasource.write.record.type")
+      .defaultValue(HoodieRecordType.AVRO.toString())
+      .withDocumentation("test");

Review Comment:
   reminder to fix the doc



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java:
##########
@@ -84,8 +87,19 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute(String instantTime,
         dedupedKeys = keys.repartition(parallelism);
       }
 
-      HoodieData<HoodieRecord<T>> dedupedRecords =
-          dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      HoodieData dedupedRecords;
+      if (config.getRecordType() == HoodieRecordType.AVRO) {
+        dedupedRecords =
+            dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      } else if (config.getRecordType() == HoodieRecordType.SPARK) {
+        dedupedRecords = dedupedKeys.map(key -> {
+          Class<?> recordClazz = ReflectionUtils.getClass("org.apache.hudi.commmon.model.HoodieSparkRecord");
+          Method method = recordClazz.getMethod("empty", HoodieKey.class);
+          return method.invoke(null, key);

Review Comment:
   this reflection API call is gonna impact perf to quite some extent



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java:
##########
@@ -132,7 +133,8 @@ public static List<String> filterKeysFromFile(Path filePath, List<String> candid
       // Load all rowKeys from the file, to double-confirm
       if (!candidateRecordKeys.isEmpty()) {
         HoodieTimer timer = new HoodieTimer().startTimer();
-        HoodieAvroFileReader fileReader = HoodieFileReaderFactory.getFileReader(configuration, filePath);
+        // Just get row keys

Review Comment:
   this comment looks redundant



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -142,6 +146,11 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Easily configure one the built-in key generators, instead of specifying the key generator class."
           + "Currently supports SIMPLE, COMPLEX, TIMESTAMP, CUSTOM, NON_PARTITION, GLOBAL_DELETE");
 
+  public static final ConfigProperty<String> RECORD_TYPE = ConfigProperty
+      .key("hoodie.datasource.write.record.type")
+      .defaultValue(HoodieRecordType.AVRO.toString())
+      .withDocumentation("test");

Review Comment:
   state valid values for this config?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -977,6 +988,16 @@ public String getKeyGeneratorClass() {
     return getString(KEYGENERATOR_CLASS_NAME);
   }
 
+  public HoodieRecord.HoodieRecordType getRecordType() {
+    HoodieRecordType recordType = HoodieRecord.HoodieRecordType.valueOf(getString(RECORD_TYPE));
+    String basePath = getString(BASE_PATH);
+    boolean metadataTable = HoodieTableMetadata.isMetadataTable(basePath);
+    if (metadataTable) {
+      recordType = HoodieRecordType.AVRO;
+    }
+    return recordType;

Review Comment:
   looks like this `getRecordType()` is invoked everywhere throughout the write path. we should optimize the perf here somehow. for e.g., the ENUM instantiation , metadata table check. since this info is not gonna change throughout a write operation, we should consider cache it.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] minihippo commented on pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
minihippo commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1155208557

   @hudi-bot run azure


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1155259103

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1189131347

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 02f8bdabc35820fa3765e77b154de6478440739d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579) 
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
danny0405 commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1191235928

   How much gains do we get after the patch, feel nervous about this change, let's pay more attention about stability first.


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1189215890

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 02f8bdabc35820fa3765e77b154de6478440739d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579) 
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 2130d320fe85d766d5b939eee2541fe7e1a00e96 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1191156859

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 10b42e4cc2af26b5eba6d62fe89a184ede548d72 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101) 
   * 7fbce260c6b883c33fd982a3aa600e53f54916be Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121) 
   * 5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1225649936

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 819824dcc83e97a7a36dab27cb2f877c113de4c6 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231820417

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 4ee2969401d4db019780389b2898526f6c99fd86 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044) 
   * a7a1ff3ce32f1e7281eed9d42679cbff70a7d523 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1234848786

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097",
       "triggerID" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 5002e131469f9587103c5ca39b322e9470eba79a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231731637

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * faf16a7e76db8efc6b23be7a8362c4fe35496556 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042) 
   * 4ee2969401d4db019780389b2898526f6c99fd86 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044) 
   * a7a1ff3ce32f1e7281eed9d42679cbff70a7d523 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1232452099

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * f3330b66baa3e3abd2d2c5b238443359d707955b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058) 
   * f5cbb2bc74556f86f8d86b79e359823645067bd8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068) 
   * 1af6f84fe01384f08f05a8cc95fb711855c37eb6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1232057297

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 4ee2969401d4db019780389b2898526f6c99fd86 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044) 
   * a7a1ff3ce32f1e7281eed9d42679cbff70a7d523 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050) 
   * 756a4a94fa739de42fd722a7e036563f3df60d27 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055) 
   * f3330b66baa3e3abd2d2c5b238443359d707955b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1189866676

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284) 
   * f482ae2541c0de736c021466bace00ce85ccf625 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1207597233

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 96701a705ec8c271438ff696c1451364b9b4398d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660) 
   * a80d4bdd93c349b09b6e640dd2229379f2173ff0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1205057038

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] minihippo commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
minihippo commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1191692939

   > How much gains do we get after the patch, feel nervous about this change, let's pay more attention about stability first.
   
   @danny0405  The current benchmark result of RFC-46:
   - no overhead introduced of avro log read/write with new merger
   - Compare with avro record in reading/writing parquet log
      - for read, spark record improves ~33% (after e2e duration-before e2e duration)*before
      - for write, spark record improves 7%~9%. In the stage that transforms input to hoodie record, it improves ~25%. But the shuffle data becomes bigger that before. It costs more in the following shuffle read stage.


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1179125217

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1179183542

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715) 
   * 028507e70c6ab8ea5682742495205c88f3c8c623 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r946077270


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -20,62 +20,33 @@
 package org.apache.hudi.common.table.log;
 
 import org.apache.hudi.common.model.HoodieRecord;
-import org.apache.hudi.common.model.HoodieRecordPayload;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
-import org.apache.hudi.exception.HoodieIOException;
 import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
-import java.util.stream.StreamSupport;
+import java.util.Properties;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
-public class HoodieFileSliceReader<T extends HoodieRecordPayload> implements Iterator<HoodieRecord<T>> {
+public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord<? extends HoodieRecordPayload> hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().getKeyWithParams(schema, props,

Review Comment:
   @wzx140 let's sync up on Slack to speed up this conversation



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1250747439

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097",
       "triggerID" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11342",
       "triggerID" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "33d151a1041104ecfbeeb7b749e82643e1abb11b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11493",
       "triggerID" : "33d151a1041104ecfbeeb7b749e82643e1abb11b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 6cd6761a77f288c15508f286a5294491026ffb81 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11342) 
   * 33d151a1041104ecfbeeb7b749e82643e1abb11b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11493) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r936362145


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -30,9 +34,19 @@
  * It can implement the merging logic of HoodieRecord of different engines
  * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
  */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
+public interface HoodieRecordMerger extends Serializable {
+
+  /**
+   * This method converges combineAndGetUpdateValue and precombine from HoodiePayload.
+   * It'd be associative operation: f(a, f(b, c)) = f(f(a, b), c) (which we can translate as having 3 versions A, B, C
+   * of the single record, both orders of operations applications have to yield the same result)
+   */
+  Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
 
-  Option<HoodieRecord> combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
+  /**
+   * The record type handled by the current merger.
+   * SPARK, AVRO, FLINK
+   */
+  HoodieRecordType getRecordType();

Review Comment:
   Yes, one RecordMerger is associated to one record type. 



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1180212763

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 028507e70c6ab8ea5682742495205c88f3c8c623 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135) 
   * 5f47e8e08a77e72c0916868e296e80d72fcd7d18 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1180341538

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 028507e70c6ab8ea5682742495205c88f3c8c623 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135) 
   * 5f47e8e08a77e72c0916868e296e80d72fcd7d18 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845) 
   * d12f5116645664470119b8c79787c2a7da5feb87 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r917628648


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {

Review Comment:
   We will remove it.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1201299871

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 6f6733fb084f2a1489527ccc3ff99caa92f52db7 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490) 
   * e7b6f7af743021b77f5c9245daed77de82577e39 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1201475794

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a7e980d34598ccee7262b93b439f92c98e78c0c1 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1203051603

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * c4f8afd6d63100feba2a218ee4331fad5790ef69 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1201077059

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 6f6733fb084f2a1489527ccc3ff99caa92f52db7 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1192371594

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * c3f5e348484ed6f538113b04dc1e2d7166d43848 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1193662703

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * c3f5e348484ed6f538113b04dc1e2d7166d43848 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190) 
   * 652a0d666fe29487d3ce2c2ce1cef70dc443dd61 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1194409817

   @danny0405 a few considerations we need to keep in mind here:
   
   1. RFC-46 is a stepping stone for transitioning from our current "modus operandi" with intermediate representation (Avro) to a state where we'd completely hybrid in relying on engine-specific containers (Dataset/RDD for Spark, for ex) as well as Data representation formats (`InternalRow` for Spark, for ex). This change is very critical first step in that direction of decoupling Hudi fro Avro.
   2. Given how dynamic our code-base is we can't park this change for long. Even now after 2 months of dev, it's going to be a humongous effort to rebase it again onto the latest changes given how much have landed in these 2 months.
   
   While i understand that we all expect radical improvements, we need to keep in mind that these will come when we reach the final state.
   
   P.S. Also, BTW, we won't see 5x improvements, it's gonna be more like up to 2x in the best case simply b/c Hudi is pretty tight in terms of performance across the board.


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r959222815


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -124,12 +126,17 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. "
           + "This will render any value set for PRECOMBINE_FIELD_OPT_VAL in-effective");
 
-  public static final ConfigProperty<String> MERGE_CLASS_NAME = ConfigProperty
+  public static final ConfigProperty<String> MERGER_CLASS_NAME = ConfigProperty

Review Comment:
   @vinothchandar You left comments here. "But high level should n't this be a table config like before."



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1232605871

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * f5cbb2bc74556f86f8d86b79e359823645067bd8 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068) 
   * 1af6f84fe01384f08f05a8cc95fb711855c37eb6 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r959063675


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -290,59 +286,59 @@ public void checkState() {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Rewrite record into new schema(add meta columns)
+   */
+  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException;
 
   /**
-   * Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields.
+   * Support schema evolution.
    */
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException;
+  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map<String, String> renameCols) throws IOException;

Review Comment:
   Let's create an override for this method to avoid providing empty-map in every call: 
   ```
   HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema) {
     rewriteRecordWithNewSchema(recordSchema. props, newSchema, Collections.emptyMap());
   }
   ```



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -290,59 +286,59 @@ public void checkState() {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Rewrite record into new schema(add meta columns)
+   */
+  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException;
 
   /**
-   * Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields.
+   * Support schema evolution.
    */
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException;
+  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map<String, String> renameCols) throws IOException;
 
-  public abstract HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException;
+  public abstract HoodieRecord updateValues(Schema recordSchema, Properties props, Map<String, String> metadataValues) throws IOException;

Review Comment:
   Where are we using this one (PR is already too large for GH, so can't search in the PR itself)? Seems quite dangerous method to have.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -126,11 +129,11 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. "
           + "This will render any value set for PRECOMBINE_FIELD_OPT_VAL in-effective");
 
-  public static final ConfigProperty<String> MERGE_CLASS_NAME = ConfigProperty
-      .key("hoodie.datasource.write.merge.class")
-      .defaultValue(HoodieAvroRecordMerge.class.getName())
-      .withDocumentation("Merge class provide stateless component interface for merging records, and support various HoodieRecord "
-          + "types, such as Spark records or Flink records.");
+  public static final ConfigProperty<String> MERGER_STRATEGY = ConfigProperty
+      .key("hoodie.datasource.write.merge.strategy")
+      .defaultValue(HoodieAvroRecordMerger.class.getName())
+      .withDocumentation("A list of merge class provide stateless component interface for merging records, and support various HoodieRecord "

Review Comment:
   nit: "List of HoodieMerger implementations constituting Hudi's merging strategy -- based on the engine used Hudi will pick most efficient implementation to perform merging/combining of the records (during update, reading MOR table, etc)"



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -126,11 +129,11 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. "
           + "This will render any value set for PRECOMBINE_FIELD_OPT_VAL in-effective");
 
-  public static final ConfigProperty<String> MERGE_CLASS_NAME = ConfigProperty
-      .key("hoodie.datasource.write.merge.class")
-      .defaultValue(HoodieAvroRecordMerge.class.getName())
-      .withDocumentation("Merge class provide stateless component interface for merging records, and support various HoodieRecord "
-          + "types, such as Spark records or Flink records.");
+  public static final ConfigProperty<String> MERGER_STRATEGY = ConfigProperty
+      .key("hoodie.datasource.write.merge.strategy")

Review Comment:
   I'd rather avoid introducing one more "strategy" term since we're not really leveraging it to full extent. We can simply name config "hoodie.datasource.write.merger.impls" to avoid confusion regarding what strategy really is in this context



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r936360772


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -19,6 +19,10 @@
 package org.apache.hudi.common.model;
 
 import org.apache.avro.Schema;

Review Comment:
   Do we need to complete schema unification in rfc46? The HoodieRecord API is used everywhere in Hudi code. If we want to use HoodieSchema, we need to almost remove avro schema in hudi code, it will be a big project.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r938247328


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.RowKeyGeneratorHelper;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+
+import scala.Tuple2;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(InternalRow data, StructType structType) {
+    return new HoodieSparkRecord(data, structType);
+  }
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();
+    final String partitionPath = (partitionName.isPresent() ? partitionName.get() :
+        getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString());
+
+    Object preCombineVal = getPreCombineVal(structType, data, preCombineField);

Review Comment:
   @wzx140 let's pull back here a little bit:
   
   > If every SparkRecord has a schema, it will bring huge performance loss (shuffle schema). The current solution is that those spark records that need to be shuffled do not save the schema like record payload.
   
   Yeah, we already had that discussion on Slack and it was a fair call-out that i previously had as a hunch and didn't get a chance to explore more thoroughly, which i'll try to rectify now: 
   
   See, i was coming from an angle that Hudi currently passes fully-deserialized payload (Avro) bearing native Java types across the board. Therefore transitioning to a state where we'd be able to instead pass a *non-serialized* payload will require non-trivial amount of effort, since assumptions that you can get any value from the record *w/o providing any schema* have been baked in pretty firmly in some areas (like the one you're referring to).
   
   In that sense, i was thinking that our v1 `HoodieRecord` impl should carry schema as well so that transition from "fully-deserialized" to "non-deserialized" records being passed around would be a smoother sail.
   
   We can approach it actually in a way that is similar to how Spark is currently handling it for... Avro actually: every Avro record bears the schema (exposed by `getSchema`) and Spark have solved the problem of needing to pump this superfluous I/O by implementing custom Kryo serializer (`GenericAvroSerializer`) that does NOT serialize the schema, but instead serializes its fingerprint w/ every record and when deserializing puts it back in. This would avoid this "schema-penalty" altogether.
   
   cc @vinothchandar 
   
   > Orderingval needs to be passed in SparkWriteHelper#deduicateRecords for deduplication. Passing schema from HoodieSparkSqlWriter to SparkWriteHelper will need to change many basic APIs
   
   We actually don't need to do that: Avro schema is passed w/in `HoodieWriteConfig` which is available in the caller of this method so you can easily extract it and parse it there and pass to `getOrderingVal` method



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1154625980

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] xushiyan commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
xushiyan commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r916351993


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java:
##########
@@ -84,8 +86,13 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute(String instantTime,
         dedupedKeys = keys.repartition(parallelism);
       }
 
-      HoodieData<HoodieRecord<T>> dedupedRecords =
-          dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      HoodieData dedupedRecords;
+      if (config.getRecordType() == HoodieRecordType.AVRO) {
+        dedupedRecords =
+            dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      } else {
+        dedupedRecords = dedupedKeys.map(key -> new HoodieEmptyRecord<>(key, config.getRecordType()));
+      }

Review Comment:
   so this is for BWC and avro type still using empty payload. it's helpful to add a note here to clarify the if else. in future we will remove avro record and empty payload completely, right?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -84,21 +80,21 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
     Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
     HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
 
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
     Schema readSchema;
+    Schema readerSchema;
+    Schema writerSchema;
     if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) {
-      readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
-      gWriter = new GenericDatumWriter<>(readSchema);
-      gReader = new GenericDatumReader<>(readSchema, mergeHandle.getWriterSchemaWithMetaFields());
+      readSchema = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordType()).getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
+      writerSchema = readSchema;
+      readerSchema = mergeHandle.getWriterSchemaWithMetaFields();
     } else {
-      gReader = null;
-      gWriter = null;
+      readerSchema = null;
+      writerSchema = null;
       readSchema = mergeHandle.getWriterSchemaWithMetaFields();
     }
 
     BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
-    HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());
+    HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordType()).getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());

Review Comment:
   this FileReader creation can be unified with the one in the if block above?



##########
hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaMergeHelper.java:
##########
@@ -66,38 +62,39 @@ public void runMerge(HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List
     HoodieMergeHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> mergeHandle = upsertHandle;
     HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
 
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
+    Schema readerSchema;
+    Schema writerSchema;

Review Comment:
   similar comments as made for HoodieMergeHelper



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);

Review Comment:
   i wonder why do we need getRecordColumnValues to be under the utils. it is the same signature as this instance method, and looks only specific to HoodieSparkRecord. Is it going to be used else where? if not, you can just keep it in the instance method here. Utils is supposed to have reusable and more generic APIs.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    Option<SparkKeyGeneratorInterface> keyGeneratorOpt = Option.empty();
+    if (useKeygen && !Boolean.parseBoolean(prop.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        keyGeneratorOpt = Option.of((SparkKeyGeneratorInterface) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(prop)));
+      } catch (IOException e) {
+        throw new HoodieException("Only SparkKeyGeneratorInterface are supported when meta columns are disabled ", e);
+      }
+    }
+    String key = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(data, structType)
+        : data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString();
+    String partition = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getPartitionPath(data, structType)
+        : data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString();
+    this.key = new HoodieKey(key, partition);
+
+    return this;
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+
+  @Override
+  public boolean isPresent(Schema schema, Properties prop) throws IOException {
+    if (null == data) {
+      return false;
+    }
+    if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) == null) {
+      return true;
+    }
+    Object deleteMarker = data.get(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).pos(), BooleanType);
+    return !(deleteMarker instanceof Boolean && (boolean) deleteMarker);
+  }
+
+  @Override
+  public boolean shouldIgnore(Schema schema, Properties prop) throws IOException {
+    // TODO SENTINEL should refactor SENTINEL without Avro(GenericRecord)
+    if (null != data && data.equals(SENTINEL)) {

Review Comment:
   let's stick with style of `data != null` instead of `null != data`  throughout the codebase



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    Option<SparkKeyGeneratorInterface> keyGeneratorOpt = Option.empty();

Review Comment:
   use of Option here is redundant and make it less readable. you can just create this.key in the if else block, can't you?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -135,21 +131,22 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
         readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
       } else {
         if (needToReWriteRecord) {
-          readerIterator = new RewriteIterator(reader.getRecordIterator(HoodieAvroIndexedRecord::new), readSchema, readSchema, table.getConfig().getProps(), renameCols);
+          readerIterator = new RewriteIterator(reader.getRecordIterator(), readSchema, readSchema, table.getConfig().getProps(), renameCols);
         } else {
-          readerIterator = reader.getRecordIterator(readSchema, HoodieAvroIndexedRecord::new);
+          readerIterator = reader.getRecordIterator(readSchema);
         }
       }
 
-      ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
-      ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
       wrapper = new BoundedInMemoryExecutor(table.getConfig().getWriteBufferLimitBytes(), readerIterator,
           new UpdateHandler(mergeHandle), record -> {
         if (!externalSchemaTransformation) {
           return record;
         }
-        // TODO Other type of record need to change
-        return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) ((HoodieRecord)record).getData());
+        try {
+          return ((HoodieRecord) record).rewriteRecord(writerSchema, readerSchema, new TypedProperties());
+        } catch (IOException e) {
+          throw new HoodieException(e);

Review Comment:
   can you add some message when wrap an exception to help troubleshoot?



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -84,21 +80,21 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
     Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
     HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
 
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
     Schema readSchema;
+    Schema readerSchema;
+    Schema writerSchema;

Review Comment:
   please use better names to distinguish these 3 vars. people will confuse: what is the difference btw readSchema and readerSchema? why readSchema assign to writerSchema? even with better names, you can add inline comments to explain the usage & scenarios further for these vars.



##########
hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkMergeHelper.java:
##########
@@ -61,42 +57,43 @@ public static FlinkMergeHelper newInstance() {
   @Override
   public void runMerge(HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table,
                        HoodieMergeHandle<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> mergeHandle) throws IOException {
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
     Schema readSchema;
+    Schema readerSchema;
+    Schema writerSchema;

Review Comment:
   similar comments as made for HoodieMergeHelper



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -135,21 +131,22 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
         readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
       } else {
         if (needToReWriteRecord) {
-          readerIterator = new RewriteIterator(reader.getRecordIterator(HoodieAvroIndexedRecord::new), readSchema, readSchema, table.getConfig().getProps(), renameCols);
+          readerIterator = new RewriteIterator(reader.getRecordIterator(), readSchema, readSchema, table.getConfig().getProps(), renameCols);
         } else {
-          readerIterator = reader.getRecordIterator(readSchema, HoodieAvroIndexedRecord::new);
+          readerIterator = reader.getRecordIterator(readSchema);
         }
       }
 
-      ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
-      ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
       wrapper = new BoundedInMemoryExecutor(table.getConfig().getWriteBufferLimitBytes(), readerIterator,
           new UpdateHandler(mergeHandle), record -> {
         if (!externalSchemaTransformation) {
           return record;
         }
-        // TODO Other type of record need to change
-        return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) ((HoodieRecord)record).getData());
+        try {
+          return ((HoodieRecord) record).rewriteRecord(writerSchema, readerSchema, new TypedProperties());

Review Comment:
   > new TypedProperties()
   
   no need to pass any properties from `table` ?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);

Review Comment:
   similar to getRecordColumnValues



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {

Review Comment:
   not making use of the props at all ?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {

Review Comment:
   why boolean as an option? also a general rule of practice is have <=5 args. A public API with >5 args is really hard to use.



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReader.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.io.storage;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.common.util.MappingIterator;
+
+import org.apache.avro.Schema;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+
+import static org.apache.hudi.TypeUtils.unsafeCast;
+
+public interface HoodieSparkFileReader extends HoodieFileReader<InternalRow> {
+
+  Logger LOG = LogManager.getLogger(HoodieSparkFileReader.class);

Review Comment:
   dont think Logger needs here at interface level. the implementation should create its own logger



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    Option<SparkKeyGeneratorInterface> keyGeneratorOpt = Option.empty();
+    if (useKeygen && !Boolean.parseBoolean(prop.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        keyGeneratorOpt = Option.of((SparkKeyGeneratorInterface) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(prop)));
+      } catch (IOException e) {
+        throw new HoodieException("Only SparkKeyGeneratorInterface are supported when meta columns are disabled ", e);
+      }
+    }
+    String key = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(data, structType)
+        : data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString();
+    String partition = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getPartitionPath(data, structType)
+        : data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString();
+    this.key = new HoodieKey(key, partition);
+
+    return this;
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+
+  @Override
+  public boolean isPresent(Schema schema, Properties prop) throws IOException {
+    if (null == data) {
+      return false;
+    }
+    if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) == null) {
+      return true;
+    }
+    Object deleteMarker = data.get(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).pos(), BooleanType);
+    return !(deleteMarker instanceof Boolean && (boolean) deleteMarker);
+  }
+
+  @Override
+  public boolean shouldIgnore(Schema schema, Properties prop) throws IOException {
+    // TODO SENTINEL should refactor SENTINEL without Avro(GenericRecord)
+    if (null != data && data.equals(SENTINEL)) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Option<IndexedRecord> toIndexedRecord(Schema schema, Properties prop) throws IOException {

Review Comment:
   Why return an option? An API like `toXYZ` should always return non-null XYZ. 



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java:
##########
@@ -54,42 +63,64 @@
   private final SerializableSchema schema;
   private final HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy;
   private final HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType;
+  private final HoodieRecordType recordType;
 
   public RDDSpatialCurveSortPartitioner(HoodieSparkEngineContext sparkEngineContext,
-                                        String[] orderByColumns,
-                                        HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy,
-                                        HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType,
-                                        Schema schema) {
+      String[] orderByColumns,
+      LayoutOptimizationStrategy layoutOptStrategy,
+      SpatialCurveCompositionStrategyType curveCompositionStrategyType,
+      Schema schema, HoodieRecordType recordType) {
     this.sparkEngineContext = sparkEngineContext;
     this.orderByColumns = orderByColumns;
     this.layoutOptStrategy = layoutOptStrategy;
     this.curveCompositionStrategyType = curveCompositionStrategyType;
     this.schema = new SerializableSchema(schema);
+    this.recordType = recordType;
   }
 
   @Override
   public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions) {
-    JavaRDD<GenericRecord> genericRecordsRDD =
-        records.map(f -> (GenericRecord) f.toIndexedRecord(schema.get(), new Properties()).get());
-
-    Dataset<Row> sourceDataset =
-        AvroConversionUtils.createDataFrame(
-            genericRecordsRDD.rdd(),
-            schema.toString(),
-            sparkEngineContext.getSqlContext().sparkSession()
-        );
-
-    Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
-
-    return HoodieSparkUtils.createRdd(sortedDataset, schema.get().getName(), schema.get().getNamespace(), false, Option.empty())
-        .toJavaRDD()
-        .map(record -> {
-          String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
-          String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
-          HoodieKey hoodieKey = new HoodieKey(key, partition);
-          HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record));
-          return hoodieRecord;
-        });
+    if (recordType == HoodieRecordType.AVRO) {
+      JavaRDD<GenericRecord> genericRecordsRDD =
+          records.map(f -> (GenericRecord) f.toIndexedRecord(schema.get(), new Properties()).get());
+
+      Dataset<Row> sourceDataset =
+          AvroConversionUtils.createDataFrame(
+              genericRecordsRDD.rdd(),
+              schema.toString(),
+              sparkEngineContext.getSqlContext().sparkSession()
+          );
+
+      Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
+
+      return HoodieSparkUtils.createRdd(sortedDataset, schema.get().getName(), schema.get().getNamespace(), false, Option.empty())
+          .toJavaRDD()
+          .map(record -> {
+            String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+            String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
+            HoodieKey hoodieKey = new HoodieKey(key, partition);
+            HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record));
+            return hoodieRecord;
+          });
+    } else if (recordType == HoodieRecordType.SPARK) {
+      StructType structType = HoodieInternalRowUtils.getCachedSchema(schema.get());
+      Dataset<Row> sourceDataset = SparkConversionUtils.createDataFrame(records.rdd(), sparkEngineContext.getSqlContext().sparkSession(), structType);
+
+      Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
+
+      return sortedDataset.queryExecution().toRdd()
+          .toJavaRDD()
+          .map(row -> {
+            InternalRow internalRow = row.copy();

Review Comment:
   can you help clarify why copy() here?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {

Review Comment:
   Properties props



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.io.storage;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+
+import java.io.IOException;
+import java.util.Properties;
+
+public interface HoodieSparkFileWriter extends HoodieFileWriter {
+  boolean canWrite();
+
+  void close() throws IOException;
+
+  void writeRowWithMetadata(HoodieKey recordKey, InternalRow row) throws IOException;
+
+  void writeRow(String recordKey, InternalRow row) throws IOException;
+
+  @Override
+  default void write(String recordKey, HoodieRecord record, Schema schema, Properties props) throws IOException {
+    writeRow(recordKey, (InternalRow) record.getData());
+  }
+
+  @Override
+  default void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema, Properties props) throws IOException {
+    writeRowWithMetadata(key, (InternalRow) record.getData());
+  }
+
+  default InternalRow prepRecordWithMetadata(HoodieKey key, InternalRow row, String instantTime, Integer partitionId, long recordIndex, String fileName)  {
+    String seqId = HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex);
+    row.update(HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(instantTime));
+    row.update(HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(seqId));
+    row.update(HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getRecordKey()));
+    row.update(HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getPartitionPath()));
+    row.update(HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(fileName));
+    return row;
+    // Object[] metadata = {instantTime, seqId, key.getRecordKey(), key.getPartitionPath(), fileName};
+    // InternalRow metadataRow = new GenericInternalRow(Arrays.stream(metadata)
+    //    .map(o -> CatalystTypeConverters.convertToCatalyst(o)).toArray());
+    // return new JoinedRow(metadataRow, row);

Review Comment:
   clean up?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.io.storage;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+
+import java.io.IOException;
+import java.util.Properties;
+
+public interface HoodieSparkFileWriter extends HoodieFileWriter {
+  boolean canWrite();
+
+  void close() throws IOException;
+
+  void writeRowWithMetadata(HoodieKey recordKey, InternalRow row) throws IOException;
+
+  void writeRow(String recordKey, InternalRow row) throws IOException;
+
+  @Override
+  default void write(String recordKey, HoodieRecord record, Schema schema, Properties props) throws IOException {
+    writeRow(recordKey, (InternalRow) record.getData());
+  }
+
+  @Override
+  default void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema, Properties props) throws IOException {
+    writeRowWithMetadata(key, (InternalRow) record.getData());
+  }
+
+  default InternalRow prepRecordWithMetadata(HoodieKey key, InternalRow row, String instantTime, Integer partitionId, long recordIndex, String fileName)  {
+    String seqId = HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex);
+    row.update(HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(instantTime));
+    row.update(HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(seqId));
+    row.update(HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getRecordKey()));
+    row.update(HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getPartitionPath()));
+    row.update(HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(fileName));

Review Comment:
   use static imports to help reduce verbosity



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    Option<SparkKeyGeneratorInterface> keyGeneratorOpt = Option.empty();
+    if (useKeygen && !Boolean.parseBoolean(prop.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        keyGeneratorOpt = Option.of((SparkKeyGeneratorInterface) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(prop)));
+      } catch (IOException e) {
+        throw new HoodieException("Only SparkKeyGeneratorInterface are supported when meta columns are disabled ", e);
+      }
+    }
+    String key = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(data, structType)
+        : data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString();
+    String partition = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getPartitionPath(data, structType)
+        : data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString();
+    this.key = new HoodieKey(key, partition);
+
+    return this;
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+
+  @Override
+  public boolean isPresent(Schema schema, Properties prop) throws IOException {

Review Comment:
   Properties props



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {

Review Comment:
   Properties props



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] minihippo commented on pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
minihippo commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1154959592

   @hudi-bot run azure


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1154988234

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r905687890


##########
hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java:
##########
@@ -436,14 +444,18 @@ public static GenericRecord removeFields(GenericRecord record, List<String> fiel
 
   private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) {
     Schema oldSchema = oldRecord.getSchema();
-    Object fieldValue = oldSchema.getField(field.name()) == null ? null : oldRecord.get(field.name());
+    Field oldSchemaField = oldSchema.getField(field.name());
+    Object fieldValue = oldSchemaField == null ? null : oldRecord.get(field.name());

Review Comment:
   When a Field is constructed, field name will be checked by org.apache.avro.Schema#validateName



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r924551196


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {

Review Comment:
   HoodieRecordPayload do not need to hold its schema. Otherwise, additional expenses will be brought from schema.



##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/HoodieInternalRowUtils.scala:
##########
@@ -188,29 +211,82 @@ object HoodieInternalRowUtils {
     schemaMap.get(schema)
   }
 
+  def getProjection(from: Schema, to: Schema): Projection = {
+    getCachedUnsafeProjection(getCachedSchema(from), getCachedSchema(to))
+  }
+
   private def getCachedProjection(from: StructType, to: StructType): Projection = {
     val schemaPair = (from, to)
     if (!projectionMap.contains(schemaPair)) {
       projectionMap.synchronized {
         if (!projectionMap.contains(schemaPair)) {
-          val projection = generateMutableProjection(from, to)
+          val utilsClazz = ReflectionUtils.getClass("org.apache.hudi.HoodieSparkProjectionUtils")

Review Comment:
   fixed



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r924546218


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {

Review Comment:
   This is strongly related to the data type. The implementation of each kind of data is different, so it is better to put it in HoodieRecord.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r903763481


##########
hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java:
##########
@@ -1037,4 +1049,25 @@ public GenericRecord next() {
       }
     };
   }
+
+  public static HoodieRecord createHoodieRecordFromAvro(IndexedRecord data, Map<String, Object> mapperConfig) {

Review Comment:
   will fix



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1207665227

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a80d4bdd93c349b09b6e640dd2229379f2173ff0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1190055664

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 11cc4bffb3c49d2f6580e72c2a599bb9a174824e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] vinothchandar commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
vinothchandar commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r909140564


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java:
##########
@@ -84,8 +87,19 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute(String instantTime,
         dedupedKeys = keys.repartition(parallelism);
       }
 
-      HoodieData<HoodieRecord<T>> dedupedRecords =
-          dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      HoodieData dedupedRecords;
+      if (config.getRecordType() == HoodieRecordType.AVRO) {
+        dedupedRecords =
+            dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      } else if (config.getRecordType() == HoodieRecordType.SPARK) {
+        dedupedRecords = dedupedKeys.map(key -> {
+          Class<?> recordClazz = ReflectionUtils.getClass("org.apache.hudi.commmon.model.HoodieSparkRecord");
+          Method method = recordClazz.getMethod("empty", HoodieKey.class);
+          return method.invoke(null, key);

Review Comment:
   +1 actually anything done per record like this is going to affect perf severly



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java:
##########
@@ -194,6 +194,8 @@ public List<WriteStatus> compact(HoodieCompactionHandler compactionHandler,
         .withBitCaskDiskMapCompressionEnabled(config.getCommonConfig().isBitCaskDiskMapCompressionEnabled())
         .withOperationField(config.allowOperationMetadataField())
         .withPartition(operation.getPartitionPath())
+        .withRecordType(config.getRecordType())
+        .withCombiningEngineClassFQN(config.getMergeClass())

Review Comment:
   rename builder name?



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##########
@@ -766,4 +764,59 @@ object HoodieSparkSqlWriter {
       Map.empty
     }
   }
+
+  private def createHoodieRecordRdd(df: DataFrame, config: HoodieConfig, parameters: Map[String, String], schema: Schema): JavaRDD[HoodieRecord[_]] = {
+    val reconcileSchema = parameters(DataSourceWriteOptions.RECONCILE_SCHEMA.key()).toBoolean
+    val tblName = config.getString(HoodieWriteConfig.TBL_NAME)
+    val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tblName)
+    val shouldCombine = parameters(INSERT_DROP_DUPS.key()).toBoolean ||
+      WriteOperationType.fromValue(config.getString(OPERATION)).equals(WriteOperationType.UPSERT) ||
+      parameters.getOrElse(HoodieWriteConfig.COMBINE_BEFORE_INSERT.key(),
+        HoodieWriteConfig.COMBINE_BEFORE_INSERT.defaultValue()).toBoolean
+    val precombineField = config.getString(PRECOMBINE_FIELD)
+    val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(config.getProps))
+    val partitionCols = HoodieSparkUtils.getPartitionColumns(keyGenerator, toProperties(parameters))
+    val dropPartitionColumns = config.getBoolean(DataSourceWriteOptions.DROP_PARTITION_COLUMNS)
+   HoodieRecord.HoodieRecordType.valueOf(config.getStringOrDefault(HoodieWriteConfig.RECORD_TYPE)) match {

Review Comment:
   ok understood how we are choosing both the paths now



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java:
##########
@@ -140,29 +136,31 @@ public abstract class AbstractHoodieLogRecordReader {
   private Option<String> partitionName;
   // Populate meta fields for the records
   private boolean populateMetaFields = true;
+  // Record type read from log block
+  protected final HoodieRecordType recordType;
 
   protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List<String> logFilePaths,
                                           Schema readerSchema,
                                           String latestInstantTime, boolean readBlocksLazily, boolean reverseReader,
                                           int bufferSize, Option<InstantRange> instantRange,
-                                          boolean withOperationField) {
+                                          boolean withOperationField, HoodieRecordType recordType, String combiningEngineClassFQN) {
     this(fs, basePath, logFilePaths, readerSchema, latestInstantTime, readBlocksLazily, reverseReader, bufferSize,
-        instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema());
+        instantRange, withOperationField, true, Option.empty(), InternalSchema.getEmptyInternalSchema(), recordType, combiningEngineClassFQN);
   }
 
   protected AbstractHoodieLogRecordReader(FileSystem fs, String basePath, List<String> logFilePaths,
                                           Schema readerSchema, String latestInstantTime, boolean readBlocksLazily,
                                           boolean reverseReader, int bufferSize, Option<InstantRange> instantRange,
                                           boolean withOperationField, boolean forceFullScan,
-                                          Option<String> partitionName, InternalSchema internalSchema) {
+                                          Option<String> partitionName, InternalSchema internalSchema, HoodieRecordType recordType, String combiningEngineClassFQN) {
     this.readerSchema = readerSchema;
     this.latestInstantTime = latestInstantTime;
     this.hoodieTableMetaClient = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(basePath).build();
     // load class from the payload fully qualified class name
     HoodieTableConfig tableConfig = this.hoodieTableMetaClient.getTableConfig();
     this.payloadClassFQN = tableConfig.getPayloadClass();
     this.preCombineField = tableConfig.getPreCombineField();
-    this.mergeClassFQN = tableConfig.getMergeClass();
+    this.mergeClassFQN = combiningEngineClassFQN;

Review Comment:
   rename variables across the board?



##########
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java:
##########
@@ -89,6 +91,8 @@ private HoodieMergedLogRecordScanner getMergedLogRecordScanner() throws IOExcept
         .withDiskMapType(jobConf.getEnum(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()))
         .withBitCaskDiskMapCompressionEnabled(jobConf.getBoolean(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(),
             HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()))
+        .withRecordType(HoodieRecordType.AVRO)
+        .withCombiningEngineClassFQN(HoodieAvroRecordMerge.class.getName())

Review Comment:
   so we override this from the reader, even though the table property has say `hoodie.merge.class` as say HoodieSparkRecordMerge?



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r909618867


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -142,6 +146,11 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Easily configure one the built-in key generators, instead of specifying the key generator class."
           + "Currently supports SIMPLE, COMPLEX, TIMESTAMP, CUSTOM, NON_PARTITION, GLOBAL_DELETE");
 
+  public static final ConfigProperty<String> RECORD_TYPE = ConfigProperty
+      .key("hoodie.datasource.write.record.type")
+      .defaultValue(HoodieRecordType.AVRO.toString())
+      .withDocumentation("test");

Review Comment:
   fixed



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r909632681


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java:
##########
@@ -84,8 +87,19 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute(String instantTime,
         dedupedKeys = keys.repartition(parallelism);
       }
 
-      HoodieData<HoodieRecord<T>> dedupedRecords =
-          dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      HoodieData dedupedRecords;
+      if (config.getRecordType() == HoodieRecordType.AVRO) {
+        dedupedRecords =
+            dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      } else if (config.getRecordType() == HoodieRecordType.SPARK) {
+        dedupedRecords = dedupedKeys.map(key -> {
+          Class<?> recordClazz = ReflectionUtils.getClass("org.apache.hudi.commmon.model.HoodieSparkRecord");
+          Method method = recordClazz.getMethod("empty", HoodieKey.class);
+          return method.invoke(null, key);

Review Comment:
   Fixed. Use HoodieEmptyRecord instead of ReflectionUtils.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r975569045


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkStructTypeSerializer.scala:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.spark.sql.hudi
+
+import com.esotericsoftware.kryo.Kryo
+import com.esotericsoftware.kryo.io.{Input, Output}
+import com.twitter.chill.KSerializer
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+import java.nio.ByteBuffer
+import java.nio.charset.StandardCharsets
+import org.apache.avro.SchemaNormalization
+import org.apache.commons.io.IOUtils
+import org.apache.hudi.commmon.model.HoodieSparkRecord
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+import org.apache.spark.{SparkEnv, SparkException}
+import scala.collection.mutable
+
+/**
+ * Custom serializer used for generic spark records. If the user registers the schemas
+ * ahead of time, then the schema's fingerprint will be sent with each message instead of the actual
+ * schema, as to reduce network IO.
+ * Actions like parsing or compressing schemas are computationally expensive so the serializer
+ * caches all previously seen values as to reduce the amount of work needed to do.
+ * @param schemas a map where the keys are unique IDs for spark schemas and the values are the
+ *                string representation of the Avro schema, used to decrease the amount of data
+ *                that needs to be serialized.
+ */
+class SparkStructTypeSerializer(schemas: Map[Long, StructType]) extends KSerializer[HoodieSparkRecord] {

Review Comment:
   How to pass schemas: Map[Long, StructType] into HoodieSparkRecordSerializer? There seems to be no entry through sparkconf.
   I persist them in HoodieInternalRowUtils with static map and init them in HoodieSparkSqlWriter#881



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1225816974

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 819824dcc83e97a7a36dab27cb2f877c113de4c6 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920) 
   * 5165092d2dca99c4e684d76811ff3d38ca0ee049 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r977041457


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkStructTypeSerializer.scala:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.spark.sql.hudi
+
+import com.esotericsoftware.kryo.Kryo
+import com.esotericsoftware.kryo.io.{Input, Output}
+import com.twitter.chill.KSerializer
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+import java.nio.ByteBuffer
+import java.nio.charset.StandardCharsets
+import org.apache.avro.SchemaNormalization
+import org.apache.commons.io.IOUtils
+import org.apache.hudi.commmon.model.HoodieSparkRecord
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+import org.apache.spark.{SparkEnv, SparkException}
+import scala.collection.mutable
+
+/**
+ * Custom serializer used for generic spark records. If the user registers the schemas
+ * ahead of time, then the schema's fingerprint will be sent with each message instead of the actual
+ * schema, as to reduce network IO.
+ * Actions like parsing or compressing schemas are computationally expensive so the serializer
+ * caches all previously seen values as to reduce the amount of work needed to do.
+ * @param schemas a map where the keys are unique IDs for spark schemas and the values are the
+ *                string representation of the Avro schema, used to decrease the amount of data
+ *                that needs to be serialized.
+ */
+class SparkStructTypeSerializer(schemas: Map[Long, StructType]) extends KSerializer[HoodieSparkRecord] {

Review Comment:
   https://hudi.apache.org/docs/quick-start-guide



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r924553356


##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java:
##########
@@ -91,13 +87,13 @@ public abstract class AbstractHoodieLogRecordReader {
   // Latest valid instant time
   // Log-Blocks belonging to inflight delta-instants are filtered-out using this high-watermark.
   private final String latestInstantTime;
-  private final HoodieTableMetaClient hoodieTableMetaClient;
+  protected final HoodieTableMetaClient hoodieTableMetaClient;
   // Merge strategy to use when combining records from log
   private final String payloadClassFQN;
   // preCombine field
   private final String preCombineField;
   // Stateless component for merging records
-  private final String mergeClassFQN;
+  private final String mergeClass;

Review Comment:
   fixed



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r924554706


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java:
##########
@@ -84,8 +86,14 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute(String instantTime,
         dedupedKeys = keys.repartition(parallelism);
       }
 
-      HoodieData<HoodieRecord<T>> dedupedRecords =
-          dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      HoodieData dedupedRecords;
+      if (config.getRecordType() == HoodieRecordType.AVRO) {

Review Comment:
   For BWC, will remove when HoodieRecordPayload removed



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1191025641

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 10b42e4cc2af26b5eba6d62fe89a184ede548d72 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101) 
   * 7fbce260c6b883c33fd982a3aa600e53f54916be Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1191023495

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 10b42e4cc2af26b5eba6d62fe89a184ede548d72 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101) 
   * 7fbce260c6b883c33fd982a3aa600e53f54916be UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1191232172

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 7fbce260c6b883c33fd982a3aa600e53f54916be Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121) 
   * 5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r909637222


##########
hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/RealtimeCompactedRecordReader.java:
##########
@@ -89,6 +91,8 @@ private HoodieMergedLogRecordScanner getMergedLogRecordScanner() throws IOExcept
         .withDiskMapType(jobConf.getEnum(HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.key(), HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE.defaultValue()))
         .withBitCaskDiskMapCompressionEnabled(jobConf.getBoolean(HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.key(),
             HoodieCommonConfig.DISK_MAP_BITCASK_COMPRESSION_ENABLED.defaultValue()))
+        .withRecordType(HoodieRecordType.AVRO)
+        .withCombiningEngineClassFQN(HoodieAvroRecordMerge.class.getName())

Review Comment:
   I just implemented spark related. MR should also follow the old logic.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1189451053

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1193790184

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 652a0d666fe29487d3ce2c2ce1cef70dc443dd61 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r927340028


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -21,64 +21,46 @@
 
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
 import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
+import java.util.Properties;
 import java.util.stream.StreamSupport;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
 public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieAvroFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().expansion(props, simpleKeyGenFieldsOpt,
+            scanner.isWithOperationField(), scanner.getPartitionName(), false));
       }
       return new HoodieFileSliceReader(scanner.iterator());
     } else {
       Iterable<HoodieRecord> iterable = () -> scanner.iterator();
-      HoodiePayloadConfig payloadConfig = HoodiePayloadConfig.newBuilder().withPayloadOrderingField(preCombineField).build();
       return new HoodieFileSliceReader(StreamSupport.stream(iterable.spliterator(), false)
           .map(e -> {
             try {
-              GenericRecord record = (GenericRecord) e.toIndexedRecord(schema, payloadConfig.getProps()).get();
-              return transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
+              return e.expansion(props, simpleKeyGenFieldsOpt, scanner.isWithOperationField(), scanner.getPartitionName(), false);

Review Comment:
   @alexeykudinkin I think the extraction of HoodieKey should be triggered when needed. Keeping these methods makes us be able to decide whether to expend. Do you think this is reasonable?



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1225638673

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a80d4bdd93c349b09b6e640dd2229379f2173ff0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661) 
   * 819824dcc83e97a7a36dab27cb2f877c113de4c6 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r959208761


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -126,11 +129,11 @@ public class HoodieWriteConfig extends HoodieConfig {
       .withDocumentation("Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. "
           + "This will render any value set for PRECOMBINE_FIELD_OPT_VAL in-effective");
 
-  public static final ConfigProperty<String> MERGE_CLASS_NAME = ConfigProperty
-      .key("hoodie.datasource.write.merge.class")
-      .defaultValue(HoodieAvroRecordMerge.class.getName())
-      .withDocumentation("Merge class provide stateless component interface for merging records, and support various HoodieRecord "
-          + "types, such as Spark records or Flink records.");
+  public static final ConfigProperty<String> MERGER_STRATEGY = ConfigProperty
+      .key("hoodie.datasource.write.merge.strategy")

Review Comment:
   fix



##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -290,59 +286,59 @@ public void checkState() {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Rewrite record into new schema(add meta columns)
+   */
+  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties props, Schema targetSchema) throws IOException;
 
   /**
-   * Rewrite the GenericRecord with the Schema containing the Hoodie Metadata fields.
+   * Support schema evolution.
    */
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException;
+  public abstract HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties props, Schema newSchema, Map<String, String> renameCols) throws IOException;

Review Comment:
   fix



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1232759493

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 1af6f84fe01384f08f05a8cc95fb711855c37eb6 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1200993156

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 279857485f18875cab94f72b5bf61522bdaecd31 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458) 
   * 6f6733fb084f2a1489527ccc3ff99caa92f52db7 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1202882975

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 0ed3f481f20ae3420e2d3d71e320a2f0992b963a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522) 
   * c4f8afd6d63100feba2a218ee4331fad5790ef69 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1200916676

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 279857485f18875cab94f72b5bf61522bdaecd31 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458) 
   * 6f6733fb084f2a1489527ccc3ff99caa92f52db7 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1180345849

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 028507e70c6ab8ea5682742495205c88f3c8c623 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135) 
   * 5f47e8e08a77e72c0916868e296e80d72fcd7d18 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845) 
   * d12f5116645664470119b8c79787c2a7da5feb87 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1201384576

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 6f6733fb084f2a1489527ccc3ff99caa92f52db7 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490) 
   * e7b6f7af743021b77f5c9245daed77de82577e39 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496) 
   * a7e980d34598ccee7262b93b439f92c98e78c0c1 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1201465010

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * e7b6f7af743021b77f5c9245daed77de82577e39 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496) 
   * a7e980d34598ccee7262b93b439f92c98e78c0c1 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r935547265


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.RowKeyGeneratorHelper;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+
+import scala.Tuple2;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(InternalRow data, StructType structType) {
+    return new HoodieSparkRecord(data, structType);
+  }
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();
+    final String partitionPath = (partitionName.isPresent() ? partitionName.get() :
+        getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString());
+
+    Object preCombineVal = getPreCombineVal(structType, data, preCombineField);

Review Comment:
   @alexeykudinkin  I found that the remove orderingVal in HoodieSparkRecord is very painful.  Because we need pass schema to getOrderingVal when call it.
   
   1. If every SparkRecord has a schema, it will bring huge performance loss (shuffle schema). The current solution is that those spark records that need to be shuffled do not save the schema like record payload. 
   
   2. Orderingval needs to be passed in SparkWriteHelper#deduicateRecords for deduplication. Passing schema from HoodieSparkSqlWriter to SparkWriteHelper will need to change many basic APIs
   
   Do you have any good idea?



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r935547265


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/util/HoodieSparkRecordUtils.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.util;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.StringUtils;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.keygen.RowKeyGeneratorHelper;
+
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.util.List;
+
+import scala.Tuple2;
+
+public class HoodieSparkRecordUtils {
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(InternalRow data, StructType structType) {
+    return new HoodieSparkRecord(data, structType);
+  }
+
+  /**
+   * Utility method to convert InternalRow to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, Option.empty());
+  }
+
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, boolean withOperationField,
+      Option<String> partitionName) {
+    return convertToHoodieSparkRecord(structType, data, preCombineField,
+        Pair.of(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.PARTITION_PATH_METADATA_FIELD),
+        withOperationField, partitionName);
+  }
+
+  /**
+   * Utility method to convert bytes to HoodieRecord using schema and payload class.
+   */
+  public static HoodieRecord<InternalRow> convertToHoodieSparkRecord(StructType structType, InternalRow data, String preCombineField, Pair<String, String> recordKeyPartitionPathFieldPair,
+      boolean withOperationField, Option<String> partitionName) {
+    final String recKey = getValue(structType, recordKeyPartitionPathFieldPair.getKey(), data).toString();
+    final String partitionPath = (partitionName.isPresent() ? partitionName.get() :
+        getValue(structType, recordKeyPartitionPathFieldPair.getRight(), data).toString());
+
+    Object preCombineVal = getPreCombineVal(structType, data, preCombineField);

Review Comment:
   I found that the remove orderingVal in HoodieSparkRecord is very painful.  Because we need pass schema to getOrderingVal when call it.
   
   1. If every SparkRecord has a schema, it will bring huge performance loss (shuffle schema). The current solution is that those spark records that need to be shuffled do not save the schema like record payload. 
   
   2. Orderingval needs to be passed in SparkWriteHelper#deduicateRecords for deduplication. Passing schema from HoodieSparkSqlWriter to SparkWriteHelper will need to change many basic APIs



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r938154747


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -30,9 +34,19 @@
  * It can implement the merging logic of HoodieRecord of different engines
  * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
  */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
+public interface HoodieRecordMerger extends Serializable {
+
+  /**
+   * This method converges combineAndGetUpdateValue and precombine from HoodiePayload.
+   * It'd be associative operation: f(a, f(b, c)) = f(f(a, b), c) (which we can translate as having 3 versions A, B, C
+   * of the single record, both orders of operations applications have to yield the same result)
+   */
+  Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
 
-  Option<HoodieRecord> combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
+  /**
+   * The record type handled by the current merger.
+   * SPARK, AVRO, FLINK
+   */
+  HoodieRecordType getRecordType();

Review Comment:
   @wzx140 we should actually do it in a similar fashion to how `KeyGenerator` interface is currently implemented:
   
   - You have generic API (`KeyGenerator`) accepting Avro (this is a bare minimum to be implemented, is used as a fallback)
   - You have engine specific API (`SparkKeyGeneratorInterface`) which provides for engine-specific APIs that you should implement natively for better performance.
   
   So when Acme implements their own `RecordMerger` they can choose to implement either 
   
   1. A bare-minimum API (Avro), that would allow it to work across the engines but will lack performance
   2. Or fully (Avro, Spark, etc) which will be performant across engines



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1207592117

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579) 
   * 96701a705ec8c271438ff696c1451364b9b4398d Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1246203644

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097",
       "triggerID" : "5002e131469f9587103c5ca39b322e9470eba79a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "6cd6761a77f288c15508f286a5294491026ffb81",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 5002e131469f9587103c5ca39b322e9470eba79a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11097) 
   * 6cd6761a77f288c15508f286a5294491026ffb81 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r974865936


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##########
@@ -505,7 +514,9 @@ public class HoodieWriteConfig extends HoodieConfig {
   private HoodieMetadataConfig metadataConfig;
   private HoodieMetastoreConfig metastoreConfig;
   private HoodieCommonConfig commonConfig;
+  private HoodieStorageConfig storageConfig;
   private EngineType engineType;
+  private HoodieRecordMerger recordMerger;

Review Comment:
   getRecordMerger will be called more than once for getting recordType(SPARK, AVRO). Holding recordMerger will be better? Or we can make it lazy loading.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r975619535


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -291,59 +284,51 @@ public void checkState() {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
-
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;

Review Comment:
   I see what you're saying. This method is confusing though, since it's also called merge (and i think some impls are actually using RecordMerger)
   
   Ideally, we shouldn't even have it in the interface (since it's not generic enough). Can we try to extract it out? 



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r975619535


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecord.java:
##########
@@ -291,59 +284,51 @@ public void checkState() {
     }
   }
 
-  //////////////////////////////////////////////////////////////////////////////
-
-  //
-  // NOTE: This method duplicates those ones of the HoodieRecordPayload and are placed here
-  //       for the duration of RFC-46 implementation, until migration off `HoodieRecordPayload`
-  //       is complete
-  //
-  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema readerSchema, Schema writerSchema) throws IOException;
+  /**
+   * Get column in record to support RDDCustomColumnsSortPartitioner
+   */
+  public abstract Object getRecordColumnValues(Schema recordSchema, String[] columns, boolean consistentLogicalTimestampEnabled);
 
-  public abstract HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException;
+  /**
+   * Support bootstrap.
+   */
+  public abstract HoodieRecord mergeWith(HoodieRecord other, Schema targetSchema) throws IOException;

Review Comment:
   I see what you're saying. This method is confusing though, since it's also called merge. 
   
   Ideally, we shouldn't even have it in the interface (since it's not generic enough). Can we try to extract it out? 



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231642997

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 4ab845a4c27a678876f33abf8196bf760df53408 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038) 
   * faf16a7e76db8efc6b23be7a8362c4fe35496556 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042) 
   * 4ee2969401d4db019780389b2898526f6c99fd86 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044) 
   * a7a1ff3ce32f1e7281eed9d42679cbff70a7d523 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231412548

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a149effad0c1992bd717e65208fbcef601deeb97 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019) 
   * 4ab845a4c27a678876f33abf8196bf760df53408 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038) 
   * faf16a7e76db8efc6b23be7a8362c4fe35496556 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042) 
   * 4ee2969401d4db019780389b2898526f6c99fd86 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1231419488

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a149effad0c1992bd717e65208fbcef601deeb97 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019) 
   * 4ab845a4c27a678876f33abf8196bf760df53408 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038) 
   * faf16a7e76db8efc6b23be7a8362c4fe35496556 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042) 
   * 4ee2969401d4db019780389b2898526f6c99fd86 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1232419009

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * f3330b66baa3e3abd2d2c5b238443359d707955b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058) 
   * f5cbb2bc74556f86f8d86b79e359823645067bd8 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1160107618

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1180754243

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * d12f5116645664470119b8c79787c2a7da5feb87 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850) 
   * 02f8bdabc35820fa3765e77b154de6478440739d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1234236675

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 1af6f84fe01384f08f05a8cc95fb711855c37eb6 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070) 
   * 17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1234585315

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 87eeee945845062be0b1d4d31b26032e7fc0eb5a Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] minihippo closed pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
minihippo closed pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.
URL: https://github.com/apache/hudi/pull/5629


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] xushiyan commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
xushiyan commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r919572926


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {

Review Comment:
   @wzx140 i meant why not just make it boolean instead of Option<Boolean>



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1191285061

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4 Azure: [CANCELED](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r917645537


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {

Review Comment:
   This method is to replace multiple old functions e.g. HoodieFileSliceReader#transform, AbstractHoodieLogRecordReader#createHoodieRecord. The boolean populateMetaFields is the gap between these two functions.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1180208461

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 028507e70c6ab8ea5682742495205c88f3c8c623 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135) 
   * 5f47e8e08a77e72c0916868e296e80d72fcd7d18 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1201377919

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 6f6733fb084f2a1489527ccc3ff99caa92f52db7 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490) 
   * e7b6f7af743021b77f5c9245daed77de82577e39 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496) 
   * a7e980d34598ccee7262b93b439f92c98e78c0c1 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1189994493

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * f482ae2541c0de736c021466bace00ce85ccf625 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r924555404


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(data, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties props, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    String key;
+    String partition;
+    if (useKeygen && !Boolean.parseBoolean(prop.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        SparkKeyGeneratorInterface keyGeneratorOpt = (SparkKeyGeneratorInterface) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(prop));
+        key = keyGeneratorOpt.getRecordKey(data, structType);
+        partition = keyGeneratorOpt.getPartitionPath(data, structType);
+      } catch (IOException e) {
+        throw new HoodieException("Only SparkKeyGeneratorInterface are supported when meta columns are disabled ", e);
+      }
+    } else {
+      key = data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString();
+      partition = data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString();
+    }
+    this.key = new HoodieKey(key, partition);
+
+    return this;
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+
+  @Override
+  public boolean isPresent(Schema schema, Properties prop) throws IOException {
+    if (null == data) {
+      return false;
+    }
+    if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) == null) {
+      return true;
+    }
+    Object deleteMarker = data.get(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).pos(), BooleanType);
+    return !(deleteMarker instanceof Boolean && (boolean) deleteMarker);
+  }
+
+  @Override
+  public boolean shouldIgnore(Schema schema, Properties prop) throws IOException {
+    // TODO SENTINEL should refactor SENTINEL without Avro(GenericRecord)
+    if (data != null && data.equals(SENTINEL)) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override
+  public Option<IndexedRecord> toIndexedRecord(Schema schema, Properties prop) throws IOException {

Review Comment:
   Its only used for HoodieAvroRecord and HoodieAvroIndexRecord.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1189210435

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 02f8bdabc35820fa3765e77b154de6478440739d Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579) 
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 2130d320fe85d766d5b939eee2541fe7e1a00e96 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r938150723


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -19,6 +19,10 @@
 package org.apache.hudi.common.model;
 
 import org.apache.avro.Schema;

Review Comment:
   We've discussed this prior that it would be gigantic rename across the board, and agreed that it's probably better to be carried out separately



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1201216585

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 6f6733fb084f2a1489527ccc3ff99caa92f52db7 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490) 
   * e7b6f7af743021b77f5c9245daed77de82577e39 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r938517253


##########
hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRecordMerger.java:
##########
@@ -30,9 +34,19 @@
  * It can implement the merging logic of HoodieRecord of different engines
  * and avoid the performance consumption caused by the serialization/deserialization of Avro payload.
  */
-public interface HoodieMerge extends Serializable {
-  
-  HoodieRecord preCombine(HoodieRecord older, HoodieRecord newer);
+@PublicAPIClass(maturity = ApiMaturityLevel.EVOLVING)
+public interface HoodieRecordMerger extends Serializable {
+
+  /**
+   * This method converges combineAndGetUpdateValue and precombine from HoodiePayload.
+   * It'd be associative operation: f(a, f(b, c)) = f(f(a, b), c) (which we can translate as having 3 versions A, B, C
+   * of the single record, both orders of operations applications have to yield the same result)
+   */
+  Option<HoodieRecord> merge(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
 
-  Option<HoodieRecord> combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException;
+  /**
+   * The record type handled by the current merger.
+   * SPARK, AVRO, FLINK
+   */
+  HoodieRecordType getRecordType();

Review Comment:
   Thanks for the suggestion. I think it's great. I've started a discussion in slack to confirm it further.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1227658927

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 64ddff55a9f3083be754e0951bf5f082fecca9e5 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1234423559

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11068",
       "triggerID" : "f5cbb2bc74556f86f8d86b79e359823645067bd8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11070",
       "triggerID" : "1af6f84fe01384f08f05a8cc95fb711855c37eb6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089",
       "triggerID" : "17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092",
       "triggerID" : "87eeee945845062be0b1d4d31b26032e7fc0eb5a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 17acfe57f4c2e9a95878e0b4f1f1d9d7bffd70e0 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11089) 
   * 87eeee945845062be0b1d4d31b26032e7fc0eb5a Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11092) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1232182975

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * a7a1ff3ce32f1e7281eed9d42679cbff70a7d523 Azure: [SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050) 
   * 756a4a94fa739de42fd722a7e036563f3df60d27 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055) 
   * f3330b66baa3e3abd2d2c5b238443359d707955b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1232287203

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * 756a4a94fa739de42fd722a7e036563f3df60d27 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055) 
   * f3330b66baa3e3abd2d2c5b238443359d707955b Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1232365428

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10302",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     }, {
       "hash" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10458",
       "triggerID" : "279857485f18875cab94f72b5bf61522bdaecd31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10490",
       "triggerID" : "6f6733fb084f2a1489527ccc3ff99caa92f52db7",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10496",
       "triggerID" : "e7b6f7af743021b77f5c9245daed77de82577e39",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10498",
       "triggerID" : "a7e980d34598ccee7262b93b439f92c98e78c0c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10522",
       "triggerID" : "0ed3f481f20ae3420e2d3d71e320a2f0992b963a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10524",
       "triggerID" : "c4f8afd6d63100feba2a218ee4331fad5790ef69",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10579",
       "triggerID" : "a4c8dda18e7718ec7abb1c60b7e6ca4c48b2618e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10660",
       "triggerID" : "96701a705ec8c271438ff696c1451364b9b4398d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10661",
       "triggerID" : "a80d4bdd93c349b09b6e640dd2229379f2173ff0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10920",
       "triggerID" : "819824dcc83e97a7a36dab27cb2f877c113de4c6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10922",
       "triggerID" : "5165092d2dca99c4e684d76811ff3d38ca0ee049",
       "triggerType" : "PUSH"
     }, {
       "hash" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10953",
       "triggerID" : "64ddff55a9f3083be754e0951bf5f082fecca9e5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10966",
       "triggerID" : "858a47a5b106462a5089ecf77278196bc7c7a0a8",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11017",
       "triggerID" : "a3b38b3f1927b4487224c47f1d9ad1e4481fa5c1",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11019",
       "triggerID" : "a149effad0c1992bd717e65208fbcef601deeb97",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11038",
       "triggerID" : "4ab845a4c27a678876f33abf8196bf760df53408",
       "triggerType" : "PUSH"
     }, {
       "hash" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11042",
       "triggerID" : "faf16a7e76db8efc6b23be7a8362c4fe35496556",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11044",
       "triggerID" : "4ee2969401d4db019780389b2898526f6c99fd86",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11050",
       "triggerID" : "a7a1ff3ce32f1e7281eed9d42679cbff70a7d523",
       "triggerType" : "PUSH"
     }, {
       "hash" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11055",
       "triggerID" : "756a4a94fa739de42fd722a7e036563f3df60d27",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058",
       "triggerID" : "f3330b66baa3e3abd2d2c5b238443359d707955b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * f3330b66baa3e3abd2d2c5b238443359d707955b Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=11058) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
alexeykudinkin commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r975734758


##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkStructTypeSerializer.scala:
##########
@@ -0,0 +1,157 @@
+/*
+ * 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.spark.sql.hudi
+
+import com.esotericsoftware.kryo.Kryo
+import com.esotericsoftware.kryo.io.{Input, Output}
+import com.twitter.chill.KSerializer
+import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, ObjectOutputStream}
+import java.nio.ByteBuffer
+import java.nio.charset.StandardCharsets
+import org.apache.avro.SchemaNormalization
+import org.apache.commons.io.IOUtils
+import org.apache.hudi.commmon.model.HoodieSparkRecord
+import org.apache.spark.io.CompressionCodec
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.Utils
+import org.apache.spark.{SparkEnv, SparkException}
+import scala.collection.mutable
+
+/**
+ * Custom serializer used for generic spark records. If the user registers the schemas
+ * ahead of time, then the schema's fingerprint will be sent with each message instead of the actual
+ * schema, as to reduce network IO.
+ * Actions like parsing or compressing schemas are computationally expensive so the serializer
+ * caches all previously seen values as to reduce the amount of work needed to do.
+ * @param schemas a map where the keys are unique IDs for spark schemas and the values are the
+ *                string representation of the Avro schema, used to decrease the amount of data
+ *                that needs to be serialized.
+ */
+class SparkStructTypeSerializer(schemas: Map[Long, StructType]) extends KSerializer[HoodieSparkRecord] {

Review Comment:
   We can't since we don't control Kryo lifecycle, and i don't think that we should -- Kryo injected into Spark's `SerializerManager` could be its own thing and we cache the schemas for our needs separately



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r975557523


##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {

Review Comment:
   I do agree we should move the base implementation into HoodieBaseRecord. But I think we should make HoodieRecord a generic interface. Because the user of HoodieRecord(WriteHandle and others) need to know the type of the HoodieRecord.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r928518937


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/common/table/log/HoodieFileSliceReader.java:
##########
@@ -21,64 +21,46 @@
 
 import org.apache.hudi.common.model.HoodieRecord;
 import org.apache.hudi.common.util.Option;
-import org.apache.hudi.common.util.SpillableMapUtils;
 import org.apache.hudi.common.util.collection.Pair;
-import org.apache.hudi.config.HoodiePayloadConfig;
 import org.apache.hudi.exception.HoodieIOException;
-import org.apache.hudi.io.storage.HoodieAvroFileReader;
+import org.apache.hudi.io.storage.HoodieFileReader;
 
 import org.apache.avro.Schema;
-import org.apache.avro.generic.GenericRecord;
 
 import java.io.IOException;
 import java.util.Iterator;
+import java.util.Properties;
 import java.util.stream.StreamSupport;
 
 /**
  * Reads records from base file and merges any updates from log files and provides iterable over all records in the file slice.
  */
 public class HoodieFileSliceReader<T> implements Iterator<HoodieRecord<T>> {
+
   private final Iterator<HoodieRecord<T>> recordsIterator;
 
   public static HoodieFileSliceReader getFileSliceReader(
-      Option<HoodieAvroFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, String payloadClass,
-      String preCombineField, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
+      Option<HoodieFileReader> baseFileReader, HoodieMergedLogRecordScanner scanner, Schema schema, Properties props, Option<Pair<String, String>> simpleKeyGenFieldsOpt) throws IOException {
     if (baseFileReader.isPresent()) {
-      Iterator baseIterator = baseFileReader.get().getRecordIterator(schema);
+      Iterator<HoodieRecord> baseIterator = baseFileReader.get().getRecordIterator(schema);
       while (baseIterator.hasNext()) {
-        GenericRecord record = (GenericRecord) baseIterator.next();
-        HoodieRecord hoodieRecord = transform(
-            record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
-        scanner.processNextRecord(hoodieRecord);
+        scanner.processNextRecord(baseIterator.next().expansion(props, simpleKeyGenFieldsOpt,
+            scanner.isWithOperationField(), scanner.getPartitionName(), false));
       }
       return new HoodieFileSliceReader(scanner.iterator());
     } else {
       Iterable<HoodieRecord> iterable = () -> scanner.iterator();
-      HoodiePayloadConfig payloadConfig = HoodiePayloadConfig.newBuilder().withPayloadOrderingField(preCombineField).build();
       return new HoodieFileSliceReader(StreamSupport.stream(iterable.spliterator(), false)
           .map(e -> {
             try {
-              GenericRecord record = (GenericRecord) e.toIndexedRecord(schema, payloadConfig.getProps()).get();
-              return transform(record, scanner, payloadClass, preCombineField, simpleKeyGenFieldsOpt);
+              return e.expansion(props, simpleKeyGenFieldsOpt, scanner.isWithOperationField(), scanner.getPartitionName(), false);

Review Comment:
   I looked at this carefully and found that expansion func is not unnecessary here. I also change the func names. expansion -> getKeyWithParams and transform -> getKeyWithKeyGen.



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1193658388

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9845",
       "triggerID" : "5f47e8e08a77e72c0916868e296e80d72fcd7d18",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9850",
       "triggerID" : "d12f5116645664470119b8c79787c2a7da5feb87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9579",
       "triggerID" : "02f8bdabc35820fa3765e77b154de6478440739d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "8bd34a6bee3084bdc6029f3c0740cf06906acfd5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10062",
       "triggerID" : "2130d320fe85d766d5b939eee2541fe7e1a00e96",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9284",
       "triggerID" : "0ebb0d7509128ca9c9f710dd37b0de93cbde2d2a",
       "triggerType" : "PUSH"
     }, {
       "hash" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10083",
       "triggerID" : "f482ae2541c0de736c021466bace00ce85ccf625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9290",
       "triggerID" : "11cc4bffb3c49d2f6580e72c2a599bb9a174824e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10101",
       "triggerID" : "10b42e4cc2af26b5eba6d62fe89a184ede548d72",
       "triggerType" : "PUSH"
     }, {
       "hash" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10121",
       "triggerID" : "7fbce260c6b883c33fd982a3aa600e53f54916be",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10133",
       "triggerID" : "5b6c32edec1f81183d94a0de274fc5a1ee5dd4a4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190",
       "triggerID" : "c3f5e348484ed6f538113b04dc1e2d7166d43848",
       "triggerType" : "PUSH"
     }, {
       "hash" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "652a0d666fe29487d3ce2c2ce1cef70dc443dd61",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 8bd34a6bee3084bdc6029f3c0740cf06906acfd5 UNKNOWN
   * c3f5e348484ed6f538113b04dc1e2d7166d43848 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10190) 
   * 652a0d666fe29487d3ce2c2ce1cef70dc443dd61 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
danny0405 commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1195084568

   > This change is very critical first step in that direction of decoupling Hudi fro Avro
   
   I agree we should decouple Hudi from Avro, but that does not mean we should lean back to  engine-specific data structures which is very hard to maintain as a engine neutral project, see how hard it is for hudi to integrate with a new engine now :),
   i kind of expect hudi's own reader/writer/data structures, which is the right direction we should elaborate with.
   
   >Given how dynamic our code-base is
   
   I know the code dev is hard because it is a core change which is very influential and piecemeal changes. But that should not be a critical prove we should fight forward with this direction.
   
   And another concern i always have in my mind is hudi needs a stable release tooo much ! We can not make huge changes to core reader/writers now at this moment before we do enough tests/practice, and we should not rush in the code for just the reason of code rebase effort.
   


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r916795887


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieDeleteHelper.java:
##########
@@ -84,8 +86,13 @@ public HoodieWriteMetadata<HoodieData<WriteStatus>> execute(String instantTime,
         dedupedKeys = keys.repartition(parallelism);
       }
 
-      HoodieData<HoodieRecord<T>> dedupedRecords =
-          dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      HoodieData dedupedRecords;
+      if (config.getRecordType() == HoodieRecordType.AVRO) {
+        dedupedRecords =
+            dedupedKeys.map(key -> new HoodieAvroRecord(key, new EmptyHoodieRecordPayload()));
+      } else {
+        dedupedRecords = dedupedKeys.map(key -> new HoodieEmptyRecord<>(key, config.getRecordType()));
+      }

Review Comment:
   Yes, I will add some comment



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] minihippo commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
minihippo commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r914608155


##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieParquetDataBlock.java:
##########
@@ -93,53 +89,41 @@ protected byte[] serializeRecords(List<HoodieRecord> records) throws IOException
     }
 
     Schema writerSchema = new Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
-
-    HoodieAvroWriteSupport writeSupport = new HoodieAvroWriteSupport(
-        new AvroSchemaConverter().convert(writerSchema), writerSchema, Option.empty());
-
-    HoodieParquetConfig<HoodieAvroWriteSupport> avroParquetConfig =
-        new HoodieParquetConfig<>(
-            writeSupport,
-            compressionCodecName.get(),
-            ParquetWriter.DEFAULT_BLOCK_SIZE,
-            ParquetWriter.DEFAULT_PAGE_SIZE,
-            1024 * 1024 * 1024,
-            new Configuration(),
-            Double.parseDouble(String.valueOf(0.1)));//HoodieStorageConfig.PARQUET_COMPRESSION_RATIO.defaultValue()));
-
     ByteArrayOutputStream baos = new ByteArrayOutputStream();
-
     try (FSDataOutputStream outputStream = new FSDataOutputStream(baos)) {
-      try (HoodieParquetStreamWriter parquetWriter = new HoodieParquetStreamWriter(outputStream, avroParquetConfig)) {
-        for (HoodieRecord record : records) {
+      HoodieFileWriter parquetWriter = null;
+      HoodieStorageConfig storageConfig =  HoodieStorageConfig.newBuilder().build();

Review Comment:
   A better way to do it



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkParquetReader.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.io.storage;
+
+import org.apache.avro.Schema;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.bloom.BloomFilter;
+import org.apache.hudi.common.model.HoodieFileFormat;
+import org.apache.hudi.common.util.BaseFileUtils;
+import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.common.util.ParquetReaderIterator;
+import org.apache.parquet.hadoop.ParquetReader;
+import org.apache.parquet.hadoop.api.ReadSupport;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.io.InputFile;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.execution.datasources.parquet.ParquetReadSupport;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+
+public class HoodieSparkParquetReader implements HoodieSparkFileReader {
+
+  private final Path path;
+  private final Configuration conf;
+  private final BaseFileUtils parquetUtils;
+  private List<ParquetReaderIterator> readerIterators = new ArrayList<>();
+
+  public HoodieSparkParquetReader(Configuration conf, Path path) {
+    this.path = path;
+    this.conf = conf;
+    this.parquetUtils = BaseFileUtils.getInstance(HoodieFileFormat.PARQUET);
+  }
+
+  @Override
+  public String[] readMinMaxRecordKeys() {
+    return parquetUtils.readMinMaxRecordKeys(conf, path);
+  }
+
+  @Override
+  public BloomFilter readBloomFilter() {
+    return parquetUtils.readBloomFilterFromMetadata(conf, path);
+  }
+
+  @Override
+  public Set<String> filterRowKeys(Set<String> candidateRowKeys) {
+    return parquetUtils.filterRowKeys(conf, path, candidateRowKeys);
+  }
+
+  @Override
+  public ClosableIterator<InternalRow> getInternalRowIterator(Schema schema) throws IOException {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    conf.set(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA(), structType.json());
+    // todo: get it from spark context
+    conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING().key(),false);
+    conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), true);
+    InputFile inputFile = HadoopInputFile.fromPath(path, conf);
+    ParquetReader reader = new ParquetReader.Builder<InternalRow>(inputFile) {
+      @Override
+      protected ReadSupport getReadSupport() {
+        return new ParquetReadSupport();
+      }
+    }.withConf(conf).build();
+    ParquetReaderIterator<InternalRow> parquetReaderIterator = new ParquetReaderIterator<>(reader, InternalRow::copy);

Review Comment:
   `InternalRow::copy` any better way to solve the problem



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    Option<SparkKeyGeneratorInterface> keyGeneratorOpt = Option.empty();
+    if (useKeygen && !Boolean.parseBoolean(prop.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        keyGeneratorOpt = Option.of((SparkKeyGeneratorInterface) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(prop)));
+      } catch (IOException e) {
+        throw new HoodieException("Only SparkKeyGeneratorInterface are supported when meta columns are disabled ", e);
+      }
+    }
+    String key = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(data, structType)
+        : data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString();
+    String partition = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getPartitionPath(data, structType)
+        : data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString();
+    this.key = new HoodieKey(key, partition);
+
+    return this;
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+
+  @Override
+  public boolean isPresent(Schema schema, Properties prop) throws IOException {

Review Comment:
   will fix



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    Option<SparkKeyGeneratorInterface> keyGeneratorOpt = Option.empty();
+    if (useKeygen && !Boolean.parseBoolean(prop.getOrDefault(POPULATE_META_FIELDS.key(), POPULATE_META_FIELDS.defaultValue().toString()).toString())) {
+      try {
+        keyGeneratorOpt = Option.of((SparkKeyGeneratorInterface) HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(prop)));
+      } catch (IOException e) {
+        throw new HoodieException("Only SparkKeyGeneratorInterface are supported when meta columns are disabled ", e);
+      }
+    }
+    String key = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getRecordKey(data, structType)
+        : data.get(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), StringType).toString();
+    String partition = keyGeneratorOpt.isPresent() ? keyGeneratorOpt.get().getPartitionPath(data, structType)
+        : data.get(HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), StringType).toString();
+    this.key = new HoodieKey(key, partition);
+
+    return this;
+  }
+
+  @Override
+  public Option<Map<String, String>> getMetadata() {
+    return Option.empty();
+  }
+
+  @Override
+  public boolean isPresent(Schema schema, Properties prop) throws IOException {
+    if (null == data) {
+      return false;
+    }
+    if (schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD) == null) {
+      return true;
+    }
+    Object deleteMarker = data.get(schema.getField(HoodieRecord.HOODIE_IS_DELETED_FIELD).pos(), BooleanType);
+    return !(deleteMarker instanceof Boolean && (boolean) deleteMarker);
+  }
+
+  @Override
+  public boolean shouldIgnore(Schema schema, Properties prop) throws IOException {
+    // TODO SENTINEL should refactor SENTINEL without Avro(GenericRecord)
+    if (null != data && data.equals(SENTINEL)) {

Review Comment:
   will fix



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java:
##########
@@ -380,27 +379,22 @@ private boolean isNewInstantBlock(HoodieLogBlock logBlock) {
    * handle it.
    */
   private void processDataBlock(HoodieDataBlock dataBlock, Option<KeySpec> keySpecOpt) throws Exception {
-    HoodieRecord.Mapper mapper = (rec) -> createHoodieRecord(rec, this.hoodieTableMetaClient.getTableConfig(),
-        this.payloadClassFQN, this.preCombineField, this.withOperationField, this.simpleKeyGenFields, this.partitionName);
-
-    try (ClosableIterator<HoodieRecord> recordIterator = getRecordsIterator(dataBlock, keySpecOpt, mapper)) {
+    try (ClosableIterator<HoodieRecord> recordIterator = getRecordsIterator(dataBlock, keySpecOpt, recordType)) {
       Option<Schema> schemaOption = getMergedSchema(dataBlock);
-      Schema finalReadSchema;
-      if (recordIterator instanceof RecordIterator) {
-        finalReadSchema = ((RecordIterator) recordIterator).getFinalReadSchema();
-      } else {
-        finalReadSchema = dataBlock.getSchema();
-      }
       while (recordIterator.hasNext()) {
         HoodieRecord currentRecord = recordIterator.next();
-        HoodieRecord record = schemaOption.isPresent()
-            ? currentRecord.rewriteRecordWithNewSchema(finalReadSchema, new Properties(), schemaOption.get(), new HashMap<>(), mapper) : currentRecord;
-        processNextRecord(record);
+        HoodieRecord record = schemaOption.isPresent() ? currentRecord.rewriteRecordWithNewSchema(dataBlock.getSchema(), new Properties(), schemaOption.get(), new HashMap<>()) : currentRecord;

Review Comment:
   unused schema here. Introduced by step1&2, should be deleted



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java:
##########
@@ -317,16 +325,36 @@ public Builder withPartition(String partitionName) {
       return this;
     }
 
+    @Override
+    public Builder withRecordType(HoodieRecordType type) {
+      this.recordType = type;
+      return this;
+    }
+
+    @Override
+    public Builder withMergeClass(String mergeClass) {
+      this.mergeClass = mergeClass;
+      return this;
+    }
+
     @Override
     public HoodieMergedLogRecordScanner build() {
       if (this.partitionName == null && CollectionUtils.nonEmpty(this.logFilePaths)) {
         this.partitionName = getRelativePartitionPath(new Path(basePath), new Path(this.logFilePaths.get(0)).getParent());
       }
+      assert recordType != null;
+      assert mergeClass != null;
+
+      if (HoodieTableMetadata.isMetadataTable(basePath)) {
+        recordType = HoodieRecordType.AVRO;
+        mergeClass = HoodieAvroRecordMerge.class.getName();
+      }

Review Comment:
   Duplicate code with `HoodieUnMergedLogRecordScanner`



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -135,21 +131,22 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
         readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
       } else {
         if (needToReWriteRecord) {
-          readerIterator = new RewriteIterator(reader.getRecordIterator(HoodieAvroIndexedRecord::new), readSchema, readSchema, table.getConfig().getProps(), renameCols);
+          readerIterator = new RewriteIterator(reader.getRecordIterator(), readSchema, readSchema, table.getConfig().getProps(), renameCols);
         } else {
-          readerIterator = reader.getRecordIterator(readSchema, HoodieAvroIndexedRecord::new);
+          readerIterator = reader.getRecordIterator(readSchema);
         }
       }
 
-      ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
-      ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
       wrapper = new BoundedInMemoryExecutor(table.getConfig().getWriteBufferLimitBytes(), readerIterator,
           new UpdateHandler(mergeHandle), record -> {
         if (!externalSchemaTransformation) {
           return record;
         }
-        // TODO Other type of record need to change
-        return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) ((HoodieRecord)record).getData());
+        try {
+          return ((HoodieRecord) record).rewriteRecord(writerSchema, readerSchema, new TypedProperties());
+        } catch (IOException e) {
+          throw new HoodieException(e);

Review Comment:
   will fix



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/io/HoodieMergeHandle.java:
##########
@@ -329,19 +329,16 @@ protected boolean writeRecord(HoodieRecord<T> hoodieRecord, Option<HoodieRecord>
    * Go through an old record. Here if we detect a newer version shows up, we write the new one to the file.
    */
   public void write(HoodieRecord<T> oldRecord) {
-    String key = oldRecord.getRecordKey(keyGeneratorOpt);
-    boolean copyOldRecord = true;
     Schema schema = useWriterSchemaForCompaction ? tableSchemaWithMetaFields : tableSchema;
+    boolean copyOldRecord = true;
+    String key = oldRecord.getRecordKey(keyGeneratorOpt);

Review Comment:
   Revert the changes



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.io.storage;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+
+import java.io.IOException;
+import java.util.Properties;
+
+public interface HoodieSparkFileWriter extends HoodieFileWriter {
+  boolean canWrite();
+
+  void close() throws IOException;
+
+  void writeRowWithMetadata(HoodieKey recordKey, InternalRow row) throws IOException;
+
+  void writeRow(String recordKey, InternalRow row) throws IOException;
+
+  @Override
+  default void write(String recordKey, HoodieRecord record, Schema schema, Properties props) throws IOException {
+    writeRow(recordKey, (InternalRow) record.getData());
+  }
+
+  @Override
+  default void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema, Properties props) throws IOException {
+    writeRowWithMetadata(key, (InternalRow) record.getData());
+  }
+
+  default InternalRow prepRecordWithMetadata(HoodieKey key, InternalRow row, String instantTime, Integer partitionId, long recordIndex, String fileName)  {
+    String seqId = HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex);
+    row.update(HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(instantTime));
+    row.update(HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(seqId));
+    row.update(HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getRecordKey()));
+    row.update(HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getPartitionPath()));
+    row.update(HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(fileName));
+    return row;
+    // Object[] metadata = {instantTime, seqId, key.getRecordKey(), key.getPartitionPath(), fileName};
+    // InternalRow metadataRow = new GenericInternalRow(Arrays.stream(metadata)
+    //    .map(o -> CatalystTypeConverters.convertToCatalyst(o)).toArray());
+    // return new JoinedRow(metadataRow, row);

Review Comment:
   will fix



##########
hudi-common/src/main/java/org/apache/hudi/common/table/log/HoodieMergedLogRecordScanner.java:
##########
@@ -189,8 +188,13 @@ protected void processNextDeletedRecord(DeleteRecord deleteRecord) {
       }
     }
     // Put the DELETE record
-    records.put(key, SpillableMapUtils.generateEmptyPayload(key,
-        deleteRecord.getPartitionPath(), deleteRecord.getOrderingValue(), getPayloadClassFQN()));
+    if (recordType == HoodieRecordType.AVRO) {
+      records.put(key, SpillableMapUtils.generateEmptyPayload(key,
+          deleteRecord.getPartitionPath(), deleteRecord.getOrderingValue(), getPayloadClassFQN()));
+    } else {
+      HoodieEmptyRecord record = new HoodieEmptyRecord<>(new HoodieKey(key, deleteRecord.getPartitionPath()), deleteRecord.getOrderingValue(), recordType);

Review Comment:
   moving into `SpillableMapUtils.generateEmptyPayload` is restraint



##########
hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/HoodieSparkDefaultRecordMerge.java:
##########
@@ -0,0 +1,68 @@
+/*
+ * 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;
+
+import org.apache.hudi.common.model.HoodieEmptyRecord;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType;
+import org.apache.hudi.common.util.Option;
+
+import org.apache.avro.Schema;
+
+import java.io.IOException;
+import java.util.Properties;
+
+public class HoodieSparkDefaultRecordMerge extends HoodieSparkRecordMerge {
+
+  @Override
+  public Option<HoodieRecord> combineAndGetUpdateValue(HoodieRecord older, HoodieRecord newer, Schema schema, Properties props) throws IOException {
+    assert older.getRecordType() == HoodieRecordType.SPARK;
+    assert newer.getRecordType() == HoodieRecordType.SPARK;
+
+    // Null check is needed here to support schema evolution. The record in storage may be from old schema where
+    // the new ordering column might not be present and hence returns null.
+    if (!needUpdatingPersistedRecord(older, newer, props)) {

Review Comment:
   Cloud precombine field be changed?



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/execution/bulkinsert/RDDSpatialCurveSortPartitioner.java:
##########
@@ -54,42 +63,64 @@
   private final SerializableSchema schema;
   private final HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy;
   private final HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType;
+  private final HoodieRecordType recordType;
 
   public RDDSpatialCurveSortPartitioner(HoodieSparkEngineContext sparkEngineContext,
-                                        String[] orderByColumns,
-                                        HoodieClusteringConfig.LayoutOptimizationStrategy layoutOptStrategy,
-                                        HoodieClusteringConfig.SpatialCurveCompositionStrategyType curveCompositionStrategyType,
-                                        Schema schema) {
+      String[] orderByColumns,
+      LayoutOptimizationStrategy layoutOptStrategy,
+      SpatialCurveCompositionStrategyType curveCompositionStrategyType,
+      Schema schema, HoodieRecordType recordType) {
     this.sparkEngineContext = sparkEngineContext;
     this.orderByColumns = orderByColumns;
     this.layoutOptStrategy = layoutOptStrategy;
     this.curveCompositionStrategyType = curveCompositionStrategyType;
     this.schema = new SerializableSchema(schema);
+    this.recordType = recordType;
   }
 
   @Override
   public JavaRDD<HoodieRecord<T>> repartitionRecords(JavaRDD<HoodieRecord<T>> records, int outputSparkPartitions) {
-    JavaRDD<GenericRecord> genericRecordsRDD =
-        records.map(f -> (GenericRecord) f.toIndexedRecord(schema.get(), new Properties()).get());
-
-    Dataset<Row> sourceDataset =
-        AvroConversionUtils.createDataFrame(
-            genericRecordsRDD.rdd(),
-            schema.toString(),
-            sparkEngineContext.getSqlContext().sparkSession()
-        );
-
-    Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
-
-    return HoodieSparkUtils.createRdd(sortedDataset, schema.get().getName(), schema.get().getNamespace(), false, Option.empty())
-        .toJavaRDD()
-        .map(record -> {
-          String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
-          String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
-          HoodieKey hoodieKey = new HoodieKey(key, partition);
-          HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record));
-          return hoodieRecord;
-        });
+    if (recordType == HoodieRecordType.AVRO) {
+      JavaRDD<GenericRecord> genericRecordsRDD =
+          records.map(f -> (GenericRecord) f.toIndexedRecord(schema.get(), new Properties()).get());
+
+      Dataset<Row> sourceDataset =
+          AvroConversionUtils.createDataFrame(
+              genericRecordsRDD.rdd(),
+              schema.toString(),
+              sparkEngineContext.getSqlContext().sparkSession()
+          );
+
+      Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
+
+      return HoodieSparkUtils.createRdd(sortedDataset, schema.get().getName(), schema.get().getNamespace(), false, Option.empty())
+          .toJavaRDD()
+          .map(record -> {
+            String key = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString();
+            String partition = record.get(HoodieRecord.PARTITION_PATH_METADATA_FIELD).toString();
+            HoodieKey hoodieKey = new HoodieKey(key, partition);
+            HoodieRecord hoodieRecord = new HoodieAvroRecord(hoodieKey, new RewriteAvroPayload(record));
+            return hoodieRecord;
+          });
+    } else if (recordType == HoodieRecordType.SPARK) {
+      StructType structType = HoodieInternalRowUtils.getCachedSchema(schema.get());
+      Dataset<Row> sourceDataset = SparkConversionUtils.createDataFrame(records.rdd(), sparkEngineContext.getSqlContext().sparkSession(), structType);
+
+      Dataset<Row> sortedDataset = reorder(sourceDataset, outputSparkPartitions);
+
+      return sortedDataset.queryExecution().toRdd()
+          .toJavaRDD()
+          .map(row -> {
+            InternalRow internalRow = row.copy();

Review Comment:
   The root cause is in the `ParquetReaderIterator`.
   ```java
    public T next() {
       try {
         // To handle case when next() is called before hasNext()
         if (this.next == null) {
           if (!hasNext()) {
             throw new HoodieException("No more records left to read from parquet file");
           }
         }
         T retVal = this.next;
         this.next = read();
         return retVal;
       } catch (Exception e) {
         FileIOUtils.closeQuietly(parquetReader);
         throw new HoodieException("unable to read next record from parquet file ", e);
       }
     }
   
     private T read() throws IOException {
       T record = parquetReader.read();
       if (mapper == null || record == null) {
         return record;
       } else {
         return mapper.apply(record);
       }
     }
   
   ```
   Caused by inner implementations of the internalRow, when go next and read without copying, the value of `retVal` will be changed meanwhile.



##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -84,21 +80,21 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
     Configuration cfgForHoodieFile = new Configuration(table.getHadoopConf());
     HoodieBaseFile baseFile = mergeHandle.baseFileForMerge();
 
-    final GenericDatumWriter<GenericRecord> gWriter;
-    final GenericDatumReader<GenericRecord> gReader;
     Schema readSchema;
+    Schema readerSchema;
+    Schema writerSchema;
     if (externalSchemaTransformation || baseFile.getBootstrapBaseFile().isPresent()) {
-      readSchema = HoodieFileReaderFactory.getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
-      gWriter = new GenericDatumWriter<>(readSchema);
-      gReader = new GenericDatumReader<>(readSchema, mergeHandle.getWriterSchemaWithMetaFields());
+      readSchema = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordType()).getFileReader(table.getHadoopConf(), mergeHandle.getOldFilePath()).getSchema();
+      writerSchema = readSchema;
+      readerSchema = mergeHandle.getWriterSchemaWithMetaFields();
     } else {
-      gReader = null;
-      gWriter = null;
+      readerSchema = null;
+      writerSchema = null;
       readSchema = mergeHandle.getWriterSchemaWithMetaFields();
     }
 
     BoundedInMemoryExecutor<GenericRecord, GenericRecord, Void> wrapper = null;
-    HoodieFileReader reader = HoodieFileReaderFactory.getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());
+    HoodieFileReader reader = HoodieFileReaderFactory.getReaderFactory(table.getConfig().getRecordType()).getFileReader(cfgForHoodieFile, mergeHandle.getOldFilePath());

Review Comment:
   will fix



##########
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieMergeOnReadRDD.scala:
##########
@@ -302,14 +306,29 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
     private def serialize(curRowRecord: InternalRow): GenericRecord =
       serializer.serialize(curRowRecord).asInstanceOf[GenericRecord]
 
-    private def merge(curAvroRecord: GenericRecord, newRecord: HoodieRecord[_ <: HoodieRecordPayload[_]]): Option[IndexedRecord] = {
+    private def merge(curRow: InternalRow, newRecord: HoodieRecord[_]): Option[InternalRow] = {
       // NOTE: We have to pass in Avro Schema used to read from Delta Log file since we invoke combining API
       //       on the record from the Delta Log
-      val combinedRecord = merge.combineAndGetUpdateValue(new HoodieAvroIndexedRecord(curAvroRecord), newRecord, logFileReaderAvroSchema, payloadProps)
-      if (combinedRecord.isPresent) {
-        toScalaOption(combinedRecord.get.asInstanceOf[HoodieAvroIndexedRecord].toIndexedRecord)
-      } else {
-        Option.empty
+      newRecord.getRecordType match {
+        case HoodieRecordType.SPARK =>
+          // Get ordering value in curAvroRecord
+          var curRecord = new HoodieSparkRecord(curRow, baseFileReaderSchema.structTypeSchema)
+          val orderField = payloadProps.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY)

Review Comment:
   move to sparkMerger



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileReader.java:
##########
@@ -0,0 +1,54 @@
+/*
+ * 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.io.storage;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.commmon.model.HoodieSparkRecord;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.ClosableIterator;
+import org.apache.hudi.common.util.MappingIterator;
+
+import org.apache.avro.Schema;
+import org.apache.log4j.LogManager;
+import org.apache.log4j.Logger;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+
+import static org.apache.hudi.TypeUtils.unsafeCast;
+
+public interface HoodieSparkFileReader extends HoodieFileReader<InternalRow> {
+
+  Logger LOG = LogManager.getLogger(HoodieSparkFileReader.class);

Review Comment:
   will fix



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/HoodieSparkFileWriter.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.io.storage;
+
+import org.apache.avro.Schema;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+
+import java.io.IOException;
+import java.util.Properties;
+
+public interface HoodieSparkFileWriter extends HoodieFileWriter {
+  boolean canWrite();
+
+  void close() throws IOException;
+
+  void writeRowWithMetadata(HoodieKey recordKey, InternalRow row) throws IOException;
+
+  void writeRow(String recordKey, InternalRow row) throws IOException;
+
+  @Override
+  default void write(String recordKey, HoodieRecord record, Schema schema, Properties props) throws IOException {
+    writeRow(recordKey, (InternalRow) record.getData());
+  }
+
+  @Override
+  default void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema, Properties props) throws IOException {
+    writeRowWithMetadata(key, (InternalRow) record.getData());
+  }
+
+  default InternalRow prepRecordWithMetadata(HoodieKey key, InternalRow row, String instantTime, Integer partitionId, long recordIndex, String fileName)  {
+    String seqId = HoodieRecord.generateSequenceId(instantTime, partitionId, recordIndex);
+    row.update(HoodieRecord.HoodieMetadataField.COMMIT_TIME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(instantTime));
+    row.update(HoodieRecord.HoodieMetadataField.COMMIT_SEQNO_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(seqId));
+    row.update(HoodieRecord.HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getRecordKey()));
+    row.update(HoodieRecord.HoodieMetadataField.PARTITION_PATH_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(key.getPartitionPath()));
+    row.update(HoodieRecord.HoodieMetadataField.FILENAME_METADATA_FIELD.ordinal(), CatalystTypeConverters.convertToCatalyst(fileName));

Review Comment:
   will fix



##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/commmon/model/HoodieSparkRecord.java:
##########
@@ -0,0 +1,270 @@
+/*
+ * 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.commmon.model;
+
+import org.apache.hudi.HoodieInternalRowUtils;
+import org.apache.hudi.common.config.TypedProperties;
+import org.apache.hudi.common.model.HoodieKey;
+import org.apache.hudi.common.model.HoodieOperation;
+import org.apache.hudi.common.model.HoodieRecord;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.exception.HoodieException;
+import org.apache.hudi.keygen.BaseKeyGenerator;
+import org.apache.hudi.keygen.SparkKeyGeneratorInterface;
+import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
+import org.apache.hudi.util.HoodieSparkRecordUtils;
+
+import org.apache.avro.Schema;
+import org.apache.avro.generic.IndexedRecord;
+import org.apache.spark.sql.catalyst.CatalystTypeConverters;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import scala.Tuple2;
+
+import static org.apache.hudi.common.table.HoodieTableConfig.POPULATE_META_FIELDS;
+import static org.apache.spark.sql.types.DataTypes.BooleanType;
+import static org.apache.spark.sql.types.DataTypes.StringType;
+
+/**
+ * Spark Engine-specific Implementations of `HoodieRecord`.
+ */
+public class HoodieSparkRecord extends HoodieRecord<InternalRow> {
+
+  // IndexedRecord hold its schema, InternalRow should also hold its schema
+  private final StructType structType;
+
+  public HoodieSparkRecord(InternalRow data, StructType schema) {
+    super(null, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(InternalRow data, StructType schema, Comparable orderingVal) {
+    super(null, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema) {
+    super(key, data);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, Comparable orderingVal) {
+    super(key, data, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieKey key, InternalRow data, StructType schema, HoodieOperation operation, Comparable orderingVal) {
+    super(key, data, operation, orderingVal);
+    this.structType = schema;
+  }
+
+  public HoodieSparkRecord(HoodieSparkRecord record) {
+    super(record);
+    this.structType = record.structType;
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance() {
+    return new HoodieSparkRecord(this);
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key, HoodieOperation op) {
+    return new HoodieSparkRecord(key, data, structType, op, getOrderingValue());
+  }
+
+  @Override
+  public HoodieRecord<InternalRow> newInstance(HoodieKey key) {
+    return new HoodieSparkRecord(key, data, structType, getOrderingValue());
+  }
+
+  @Override
+  public void deflate() {
+  }
+
+  @Override
+  public String getRecordKey(Option<BaseKeyGenerator> keyGeneratorOpt) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    return keyGeneratorOpt.isPresent() ? ((SparkKeyGeneratorInterface) keyGeneratorOpt.get()).getRecordKey(data, structType) : data.getString(HoodieMetadataField.RECORD_KEY_METADATA_FIELD.ordinal());
+  }
+
+  @Override
+  public String getRecordKey(String keyFieldName) {
+    if (key != null) {
+      return getRecordKey();
+    }
+    Tuple2<StructField, Object> tuple2 = HoodieInternalRowUtils.getCachedSchemaPosMap(structType).get(keyFieldName).get();
+    DataType dataType = tuple2._1.dataType();
+    int pos = (Integer) tuple2._2;
+    return data.get(pos, dataType).toString();
+  }
+
+  @Override
+  public HoodieRecordType getRecordType() {
+    return HoodieRecordType.SPARK;
+  }
+
+  @Override
+  public Object getRecordColumnValues(String[] columns, Schema schema, boolean consistentLogicalTimestampEnabled) {
+    return HoodieSparkRecordUtils.getRecordColumnValues(this, columns, structType, consistentLogicalTimestampEnabled);
+  }
+
+  @Override
+  public HoodieRecord mergeWith(Schema schema, HoodieRecord other, Schema otherSchema, Schema writerSchema) throws IOException {
+    StructType otherStructType = HoodieInternalRowUtils.getCachedSchema(otherSchema);
+    StructType writerStructType = HoodieInternalRowUtils.getCachedSchema(writerSchema);
+    InternalRow mergeRow = HoodieInternalRowUtils.stitchRecords(data, structType, (InternalRow) other.getData(), otherStructType, writerStructType);
+    return new HoodieSparkRecord(getKey(), mergeRow, writerStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Schema targetSchema, TypedProperties props) throws IOException {
+    StructType targetStructType = HoodieInternalRowUtils.getCachedSchema(targetSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, targetStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, targetStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecord(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, writeSchemaWithMetaFieldsStructType, new HashMap<>())
+        : HoodieInternalRowUtils.rewriteRecord(data, structType, writeSchemaWithMetaFieldsStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithMetadata(Schema recordSchema, Properties prop, boolean schemaOnReadEnabled, Schema writeSchemaWithMetaFields, String fileName) throws IOException {
+    StructType writeSchemaWithMetaFieldsStructType = HoodieInternalRowUtils.getCachedSchema(writeSchemaWithMetaFields);
+    InternalRow rewriteRow = schemaOnReadEnabled ? HoodieInternalRowUtils.rewriteEvolutionRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName)
+        : HoodieInternalRowUtils.rewriteRecordWithMetadata(data, structType, writeSchemaWithMetaFieldsStructType, fileName);
+    return new HoodieSparkRecord(getKey(), rewriteRow, writeSchemaWithMetaFieldsStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema, Map<String, String> renameCols) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecordWithNewSchema(data, structType, newStructType, renameCols);
+    return new HoodieSparkRecord(getKey(), rewriteRow, newStructType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord rewriteRecordWithNewSchema(Schema recordSchema, Properties prop, Schema newSchema) throws IOException {
+    StructType newStructType = HoodieInternalRowUtils.getCachedSchema(newSchema);
+    InternalRow rewriteRow = HoodieInternalRowUtils.rewriteRecord(data, structType, newStructType);
+    return new HoodieSparkRecord(getKey(), rewriteRow, structType, getOperation());
+  }
+
+  @Override
+  public HoodieRecord overrideMetadataFieldValue(Schema recordSchema, Properties prop, int pos, String newValue) throws IOException {
+    data.update(pos, CatalystTypeConverters.convertToCatalyst(newValue));
+    return this;
+  }
+
+  @Override
+  public HoodieRecord addMetadataValues(Schema recordSchema, Properties prop, Map<HoodieMetadataField, String> metadataValues) throws IOException {
+    Arrays.stream(HoodieMetadataField.values()).forEach(metadataField -> {
+      String value = metadataValues.get(metadataField);
+      if (value != null) {
+        data.update(recordSchema.getField(metadataField.getFieldName()).pos(), CatalystTypeConverters.convertToCatalyst(value));
+      }
+    });
+    return this;
+  }
+
+  @Override
+  public HoodieRecord expansion(Schema schema, Properties prop, String payloadClass,
+      String preCombineField,
+      Option<Pair<String, String>> simpleKeyGenFieldsOpt,
+      Boolean withOperation,
+      Option<String> partitionNameOp,
+      Option<Boolean> populateMetaFieldsOp) {
+    boolean populateMetaFields = populateMetaFieldsOp.orElse(false);
+    if (populateMetaFields) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation);
+    } else if (simpleKeyGenFieldsOpt.isPresent()) {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, simpleKeyGenFieldsOpt.get(), withOperation, Option.empty());
+    } else {
+      return HoodieSparkRecordUtils.convertToHoodieSparkRecord(structType, data, preCombineField, withOperation, partitionNameOp);
+    }
+  }
+
+  @Override
+  public HoodieRecord transform(Schema schema, Properties prop, boolean useKeygen) {
+    StructType structType = HoodieInternalRowUtils.getCachedSchema(schema);
+    Option<SparkKeyGeneratorInterface> keyGeneratorOpt = Option.empty();

Review Comment:
   will fix
   



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r916849149


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -135,21 +131,22 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
         readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
       } else {
         if (needToReWriteRecord) {
-          readerIterator = new RewriteIterator(reader.getRecordIterator(HoodieAvroIndexedRecord::new), readSchema, readSchema, table.getConfig().getProps(), renameCols);
+          readerIterator = new RewriteIterator(reader.getRecordIterator(), readSchema, readSchema, table.getConfig().getProps(), renameCols);
         } else {
-          readerIterator = reader.getRecordIterator(readSchema, HoodieAvroIndexedRecord::new);
+          readerIterator = reader.getRecordIterator(readSchema);
         }
       }
 
-      ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
-      ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
       wrapper = new BoundedInMemoryExecutor(table.getConfig().getWriteBufferLimitBytes(), readerIterator,
           new UpdateHandler(mergeHandle), record -> {
         if (!externalSchemaTransformation) {
           return record;
         }
-        // TODO Other type of record need to change
-        return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) ((HoodieRecord)record).getData());
+        try {
+          return ((HoodieRecord) record).rewriteRecord(writerSchema, readerSchema, new TypedProperties());

Review Comment:
   Properties is not need. will remove



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r916849149


##########
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/HoodieMergeHelper.java:
##########
@@ -135,21 +131,22 @@ public void runMerge(HoodieTable<T, HoodieData<HoodieRecord<T>>, HoodieData<Hood
         readerIterator = getMergingIterator(table, mergeHandle, baseFile, reader, readSchema, externalSchemaTransformation);
       } else {
         if (needToReWriteRecord) {
-          readerIterator = new RewriteIterator(reader.getRecordIterator(HoodieAvroIndexedRecord::new), readSchema, readSchema, table.getConfig().getProps(), renameCols);
+          readerIterator = new RewriteIterator(reader.getRecordIterator(), readSchema, readSchema, table.getConfig().getProps(), renameCols);
         } else {
-          readerIterator = reader.getRecordIterator(readSchema, HoodieAvroIndexedRecord::new);
+          readerIterator = reader.getRecordIterator(readSchema);
         }
       }
 
-      ThreadLocal<BinaryEncoder> encoderCache = new ThreadLocal<>();
-      ThreadLocal<BinaryDecoder> decoderCache = new ThreadLocal<>();
       wrapper = new BoundedInMemoryExecutor(table.getConfig().getWriteBufferLimitBytes(), readerIterator,
           new UpdateHandler(mergeHandle), record -> {
         if (!externalSchemaTransformation) {
           return record;
         }
-        // TODO Other type of record need to change
-        return transformRecordBasedOnNewSchema(gReader, gWriter, encoderCache, decoderCache, (GenericRecord) ((HoodieRecord)record).getData());
+        try {
+          return ((HoodieRecord) record).rewriteRecord(writerSchema, readerSchema, new TypedProperties());

Review Comment:
   will remove



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1179186723

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9715",
       "triggerID" : "42ba86dce3ec0b072cc6ec727a27ff7fd8a8a51f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135",
       "triggerID" : "028507e70c6ab8ea5682742495205c88f3c8c623",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 028507e70c6ab8ea5682742495205c88f3c8c623 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=9135) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1131053451

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * c34256ac98d03c787d264e56e35a7058d4273442 Azure: [PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755) 
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] minihippo closed pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
minihippo closed pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.
URL: https://github.com/apache/hudi/pull/5629


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] hudi-bot commented on pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
hudi-bot commented on PR #5629:
URL: https://github.com/apache/hudi/pull/5629#issuecomment-1160155096

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8755",
       "triggerID" : "c34256ac98d03c787d264e56e35a7058d4273442",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d0f078159313f8b35a41b1d1e016583204811383",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d0f078159313f8b35a41b1d1e016583204811383",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "00d5fed1954348b749859f8f81fec593422df774",
       "triggerType" : "PUSH"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154624809",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1154959592",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1155208557",
       "triggerType" : "MANUAL"
     }, {
       "hash" : "00d5fed1954348b749859f8f81fec593422df774",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758",
       "triggerID" : "1160106390",
       "triggerType" : "MANUAL"
     } ]
   }-->
   ## CI report:
   
   * d0f078159313f8b35a41b1d1e016583204811383 UNKNOWN
   * 00d5fed1954348b749859f8f81fec593422df774 Azure: [FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=8758) 
   
   <details>
   <summary>Bot commands</summary>
     @hudi-bot supports the following commands:
   
    - `@hudi-bot run azure` re-run the last Azure build
   </details>


-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wzx140 commented on a diff in pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
wzx140 commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r903766219


##########
hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java:
##########
@@ -218,4 +240,15 @@ public <T> String getStringOrThrow(ConfigProperty<T> configProperty, String erro
       throw new HoodieException(errorMessage);
     }
   }
+
+  public HoodieRecordType getRecordType() {

Review Comment:
   will fix



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] minihippo commented on a diff in pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
minihippo commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r903861647


##########
hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java:
##########
@@ -436,14 +444,18 @@ public static GenericRecord removeFields(GenericRecord record, List<String> fiel
 
   private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) {
     Schema oldSchema = oldRecord.getSchema();
-    Object fieldValue = oldSchema.getField(field.name()) == null ? null : oldRecord.get(field.name());
+    Field oldSchemaField = oldSchema.getField(field.name());
+    Object fieldValue = oldSchemaField == null ? null : oldRecord.get(field.name());

Review Comment:
   final name = null?



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] minihippo commented on a diff in pull request #5629: [WIP][HUDI-3384][HUDI-3385] Spark specific file reader/writer.

Posted by GitBox <gi...@apache.org>.
minihippo commented on code in PR #5629:
URL: https://github.com/apache/hudi/pull/5629#discussion_r903317660


##########
hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java:
##########
@@ -436,14 +444,18 @@ public static GenericRecord removeFields(GenericRecord record, List<String> fiel
 
   private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) {
     Schema oldSchema = oldRecord.getSchema();
-    Object fieldValue = oldSchema.getField(field.name()) == null ? null : oldRecord.get(field.name());
+    Field oldSchemaField = oldSchema.getField(field.name());
+    Object fieldValue = oldSchemaField == null ? null : oldRecord.get(field.name());
 
     if (fieldValue != null) {
       // In case field's value is a nested record, we have to rewrite it as well
       Object newFieldValue;
       if (fieldValue instanceof GenericRecord) {
         GenericRecord record = (GenericRecord) fieldValue;
-        newFieldValue = rewriteRecord(record, resolveUnionSchema(field.schema(), record.getSchema().getFullName()));
+        // May return null when use rewrite
+        String recordFullName = record.getSchema().getFullName();
+        String fullName = recordFullName != null ? recordFullName : oldSchemaField.name();
+        newFieldValue = rewriteRecord(record, resolveUnionSchema(field.schema(), fullName));

Review Comment:
   `fullName ` shouldn't be null. I suppose the changes is caused by `EmptyRecord`, right?



##########
hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java:
##########
@@ -1037,4 +1049,25 @@ public GenericRecord next() {
       }
     };
   }
+
+  public static HoodieRecord createHoodieRecordFromAvro(IndexedRecord data, Map<String, Object> mapperConfig) {

Review Comment:
   mapperconfig -> props may be better? Align with the current config passing way



##########
hudi-common/src/main/java/org/apache/hudi/avro/HoodieAvroUtils.java:
##########
@@ -436,14 +444,18 @@ public static GenericRecord removeFields(GenericRecord record, List<String> fiel
 
   private static void copyOldValueOrSetDefault(GenericRecord oldRecord, GenericRecord newRecord, Schema.Field field) {
     Schema oldSchema = oldRecord.getSchema();
-    Object fieldValue = oldSchema.getField(field.name()) == null ? null : oldRecord.get(field.name());
+    Field oldSchemaField = oldSchema.getField(field.name());
+    Object fieldValue = oldSchemaField == null ? null : oldRecord.get(field.name());

Review Comment:
   Can field.name be null? 



##########
hudi-common/src/main/java/org/apache/hudi/common/config/HoodieConfig.java:
##########
@@ -218,4 +240,15 @@ public <T> String getStringOrThrow(ConfigProperty<T> configProperty, String erro
       throw new HoodieException(errorMessage);
     }
   }
+
+  public HoodieRecordType getRecordType() {

Review Comment:
   It's not a basic function, so it's inappropriate to put it here



-- 
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: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org