[GitHub] [hudi] nsivabalan commented on a diff in pull request #6213: [HUDI-4081][HUDI-4472] Addressing Spark SQL vs Spark DS performance gap

2022-07-27 Thread GitBox


nsivabalan commented on code in PR #6213:
URL: https://github.com/apache/hudi/pull/6213#discussion_r931794171


##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##
@@ -241,39 +240,49 @@ object HoodieSparkSqlWriter {
 sparkContext.getConf.registerKryoClasses(
   Array(classOf[org.apache.avro.generic.GenericData],
 classOf[org.apache.avro.Schema]))
-var schema = 
AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, 
nameSpace)
-val lastestSchema = getLatestTableSchema(fs, basePath, 
sparkContext, schema)
+
+// TODO(HUDI-4472) revisit and simplify schema handling
+val sourceSchema = 
AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, 
nameSpace)
+val latestTableSchema = getLatestTableSchema(fs, basePath, 
sparkContext).getOrElse(sourceSchema)
+
+val enabledSchemaEvolution = 
parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), 
"false").toBoolean
 var internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, 
sparkContext)
-if (reconcileSchema && 
parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), 
"false").toBoolean
-  && internalSchemaOpt.isEmpty) {
-  // force apply full schema evolution.
-  internalSchemaOpt = 
Some(AvroInternalSchemaConverter.convert(schema))
-}
-if (reconcileSchema) {
-  schema = lastestSchema
-}
-if (internalSchemaOpt.isDefined) {
-  // Apply schema evolution.
-  val mergedSparkSchema = if (!reconcileSchema) {
-
AvroConversionUtils.convertAvroSchemaToStructType(AvroSchemaEvolutionUtils.canonicalizeColumnNullability(schema,
 lastestSchema))
+
+val writerSchema: Schema =
+  if (reconcileSchema) {
+// In case we need to reconcile the schema and schema 
evolution is enabled,
+// we will force-apply schema evolution to the writer's schema
+if (enabledSchemaEvolution && internalSchemaOpt.isEmpty) {
+  internalSchemaOpt = 
Some(AvroInternalSchemaConverter.convert(sourceSchema))
+}
+
+if (internalSchemaOpt.isDefined) {
+  // Apply schema evolution, by auto-merging write schema and 
read schema
+  val mergedInternalSchema = 
AvroSchemaEvolutionUtils.reconcileSchema(sourceSchema, internalSchemaOpt.get)
+  AvroInternalSchemaConverter.convert(mergedInternalSchema, 
latestTableSchema.getName)
+} else if 
(TableSchemaResolver.isSchemaCompatible(sourceSchema, latestTableSchema)) {
+  // In case schema reconciliation is enabled and source and 
latest table schemas
+  // are compatible (as defined by 
[[TableSchemaResolver#isSchemaCompatible]], then we will
+  // pick latest table's schema as the writer's schema
+  latestTableSchema
+} else {
+  // Otherwise fallback to original source's schema
+  sourceSchema
+}
   } else {
-// Auto merge write schema and read schema.
-val mergedInternalSchema = 
AvroSchemaEvolutionUtils.reconcileSchema(schema, internalSchemaOpt.get)
-
AvroConversionUtils.convertAvroSchemaToStructType(AvroInternalSchemaConverter.convert(mergedInternalSchema,
 lastestSchema.getName))
+// In case reconciliation is disabled, we still have to do 
nullability attributes
+// (minor) reconciliation, making sure schema of the incoming 
batch is in-line with
+// the data already committed in the table
+
AvroSchemaEvolutionUtils.canonicalizeColumnNullability(sourceSchema, 
latestTableSchema)

Review Comment:
   @alexeykudinkin : prior to this patch, when `reconcileSchema` is disabled 
and when no schema evolution is enabled, where do we do 
canonicalizecolumnnullability?  would you mind pointing me to the code path.



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] nsivabalan commented on a diff in pull request #6213: [HUDI-4081][HUDI-4472] Addressing Spark SQL vs Spark DS performance gap

2022-07-27 Thread GitBox


nsivabalan commented on code in PR #6213:
URL: https://github.com/apache/hudi/pull/6213#discussion_r931737037


##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##
@@ -241,39 +240,49 @@ object HoodieSparkSqlWriter {
 sparkContext.getConf.registerKryoClasses(
   Array(classOf[org.apache.avro.generic.GenericData],
 classOf[org.apache.avro.Schema]))
-var schema = 
AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, 
nameSpace)
-val lastestSchema = getLatestTableSchema(fs, basePath, 
sparkContext, schema)
+
+// TODO(HUDI-4472) revisit and simplify schema handling
+val sourceSchema = 
AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, 
nameSpace)
+val latestTableSchema = getLatestTableSchema(fs, basePath, 
sparkContext).getOrElse(sourceSchema)
+
+val enabledSchemaEvolution = 
parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), 
"false").toBoolean

Review Comment:
   should we use DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.defaultValue 
instead of explicit "false" ? 



##
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/InsertIntoHoodieTableCommand.scala:
##
@@ -66,100 +79,139 @@ object InsertIntoHoodieTableCommand extends Logging with 
ProvidesHoodieConfig {
* @param extraOptions Extra options for insert.
*/
   def run(sparkSession: SparkSession,
-  table: CatalogTable,
-  query: LogicalPlan,
-  insertPartitions: Map[String, Option[String]],
-  overwrite: Boolean,
-  refreshTable: Boolean = true,
-  extraOptions: Map[String, String] = Map.empty): Boolean = {
-
-val hoodieCatalogTable = new HoodieCatalogTable(sparkSession, table)
-val config = buildHoodieInsertConfig(hoodieCatalogTable, sparkSession, 
overwrite, insertPartitions, extraOptions)
-
-val mode = if (overwrite && hoodieCatalogTable.partitionFields.isEmpty) {
-  // insert overwrite non-partition table
+  table: CatalogTable,
+  query: LogicalPlan,
+  partitionSpec: Map[String, Option[String]],
+  overwrite: Boolean,
+  refreshTable: Boolean = true,
+  extraOptions: Map[String, String] = Map.empty): Boolean = {
+val catalogTable = new HoodieCatalogTable(sparkSession, table)
+val config = buildHoodieInsertConfig(catalogTable, sparkSession, 
overwrite, partitionSpec, extraOptions)
+
+// NOTE: In case of partitioned table we override specified "overwrite" 
parameter
+//   to instead append to the dataset
+val mode = if (overwrite && catalogTable.partitionFields.isEmpty) {
   SaveMode.Overwrite
 } else {
-  // for insert into or insert overwrite partition we use append mode.
   SaveMode.Append
 }
-val conf = sparkSession.sessionState.conf
-val alignedQuery = alignOutputFields(query, hoodieCatalogTable, 
insertPartitions, conf)
-// If we create dataframe using the Dataset.ofRows(sparkSession, 
alignedQuery),
-// The nullable attribute of fields will lost.
-// In order to pass the nullable attribute to the inputDF, we specify the 
schema
-// of the rdd.
-val inputDF = sparkSession.createDataFrame(
-  Dataset.ofRows(sparkSession, alignedQuery).rdd, alignedQuery.schema)
-val success =
-  HoodieSparkSqlWriter.write(sparkSession.sqlContext, mode, config, 
inputDF)._1
-if (success) {
-  if (refreshTable) {
-sparkSession.catalog.refreshTable(table.identifier.unquotedString)
-  }
-  true
-} else {
-  false
+
+val alignedQuery = alignQueryOutput(query, catalogTable, partitionSpec, 
sparkSession.sessionState.conf)
+
+val (success, _, _, _, _, _) = 
HoodieSparkSqlWriter.write(sparkSession.sqlContext, mode, config, 
Dataset.ofRows(sparkSession, alignedQuery))
+
+if (success && refreshTable) {
+  sparkSession.catalog.refreshTable(table.identifier.unquotedString)
 }
+
+success
   }
 
   /**
-   * Aligned the type and name of query's output fields with the result 
table's fields.
-   * @param query The insert query which to aligned.
-   * @param hoodieCatalogTable The result hoodie catalog table.
-   * @param insertPartitions The insert partition map.
-   * @param conf The SQLConf.
-   * @return
+   * Align provided [[query]]'s output with the expected [[catalogTable]] 
schema by
+   *
+   * 
+   *   Performing type coercion (casting corresponding outputs, where 
needed)
+   *   Adding aliases (matching column names) to corresponding outputs 

+   * 
+   *
+   * @param query target query whose output is to be inserted
+   * @param catalogTable catalog table
+   * @param partitionsSpec partition spec specifying static/dynamic partition 
values
+   * @param conf Spark's [[SQLConf]]
*/
-  private def alignOutputFields(
-query: 

[GitHub] [hudi] hudi-bot commented on pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


hudi-bot commented on PR #6016:
URL: https://github.com/apache/hudi/pull/6016#issuecomment-1197663152

   
   ## CI report:
   
   * 9e57627ffddf0eedcd613b54e0f5be5d54462b0b Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10416)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] vamshigv commented on a diff in pull request #6228: [HUDI-4488] Improve S3EventsHoodieIncrSource efficiency

2022-07-27 Thread GitBox


vamshigv commented on code in PR #6228:
URL: https://github.com/apache/hudi/pull/6228#discussion_r931764557


##
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java:
##
@@ -172,37 +177,47 @@ public Pair>, String> 
fetchNextBatch(Option lastCkpt
 String s3FS = props.getString(Config.S3_FS_PREFIX, "s3").toLowerCase();
 String s3Prefix = s3FS + "://";
 
-// Extract distinct file keys from s3 meta hoodie table
-final List cloudMetaDf = source
+// Create S3 paths
+final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
+SerializableConfiguration serializableConfiguration = new 
SerializableConfiguration(sparkContext.hadoopConfiguration());
+List cloudFiles = source
 .filter(filter)
 .select("s3.bucket.name", "s3.object.key")
 .distinct()
-.collectAsList();
-// Create S3 paths
-final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
-List cloudFiles = new ArrayList<>();
-for (Row row : cloudMetaDf) {
-  // construct file path, row index 0 refers to bucket and 1 refers to key
-  String bucket = row.getString(0);
-  String filePath = s3Prefix + bucket + "/" + row.getString(1);
-  if (checkExists) {
-FileSystem fs = FSUtils.getFs(s3Prefix + bucket, 
sparkSession.sparkContext().hadoopConfiguration());
-try {
-  if (fs.exists(new Path(filePath))) {
-cloudFiles.add(filePath);
-  }
-} catch (IOException e) {
-  LOG.error(String.format("Error while checking path exists for %s ", 
filePath), e);
-}
-  } else {
-cloudFiles.add(filePath);
-  }
-}
+.rdd().toJavaRDD().mapPartitions(fileListIterator -> {
+  List cloudFilesPerPartition = new ArrayList<>();
+  fileListIterator.forEachRemaining(row -> {
+final Configuration configuration = 
serializableConfiguration.newCopy();
+String bucket = row.getString(0);
+String filePath = s3Prefix + bucket + "/" + row.getString(1);
+try {
+  String decodeUrl = URLDecoder.decode(filePath, 
StandardCharsets.UTF_8.name());
+  if (checkExists) {
+FileSystem fs = FSUtils.getFs(s3Prefix + bucket, 
configuration);
+try {

Review Comment:
   Can you please point me to some places elsewhere in hudi where this is done ?



##
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java:
##
@@ -172,37 +177,47 @@ public Pair>, String> 
fetchNextBatch(Option lastCkpt
 String s3FS = props.getString(Config.S3_FS_PREFIX, "s3").toLowerCase();
 String s3Prefix = s3FS + "://";
 
-// Extract distinct file keys from s3 meta hoodie table
-final List cloudMetaDf = source
+// Create S3 paths
+final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
+SerializableConfiguration serializableConfiguration = new 
SerializableConfiguration(sparkContext.hadoopConfiguration());
+List cloudFiles = source
 .filter(filter)
 .select("s3.bucket.name", "s3.object.key")
 .distinct()
-.collectAsList();
-// Create S3 paths
-final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
-List cloudFiles = new ArrayList<>();
-for (Row row : cloudMetaDf) {
-  // construct file path, row index 0 refers to bucket and 1 refers to key
-  String bucket = row.getString(0);
-  String filePath = s3Prefix + bucket + "/" + row.getString(1);
-  if (checkExists) {
-FileSystem fs = FSUtils.getFs(s3Prefix + bucket, 
sparkSession.sparkContext().hadoopConfiguration());
-try {
-  if (fs.exists(new Path(filePath))) {
-cloudFiles.add(filePath);
-  }
-} catch (IOException e) {
-  LOG.error(String.format("Error while checking path exists for %s ", 
filePath), e);
-}
-  } else {
-cloudFiles.add(filePath);
-  }
-}
+.rdd().toJavaRDD().mapPartitions(fileListIterator -> {
+  List cloudFilesPerPartition = new ArrayList<>();
+  fileListIterator.forEachRemaining(row -> {
+final Configuration configuration = 
serializableConfiguration.newCopy();

Review Comment:
   Good catch! IIUC that should be okay to do.



##
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java:
##
@@ -172,37 +177,47 @@ public Pair>, String> 
fetchNextBatch(Option lastCkpt
 String s3FS = props.getString(Config.S3_FS_PREFIX, "s3").toLowerCase();
 String s3Prefix = s3FS + "://";
 
-// Extract distinct file keys from s3 meta hoodie table
-final List cloudMetaDf = source
+// Create 

[GitHub] [hudi] neerajpadarthi opened a new issue, #6232: [SUPPORT] Hudi V0.9 truncating second precision for timestamp columns

2022-07-27 Thread GitBox


neerajpadarthi opened a new issue, #6232:
URL: https://github.com/apache/hudi/issues/6232

   Hi Team,
   
   Using the configs below, I see Hudi is truncating the second precisions 
while ingesting the data. We are currently on 0.9V and I have observed this 
issue with this version, but it worked with 0.11V. 
   
   Do I need to add any other configurations to make it work with 0.9V without 
migrating to 0.11V? Any help on how to avoid this issue would be greatly 
appreciated.
   
   Configs
   
   db_name = tst_db
   tableName =tst_tb
   pk = ‘id’
   de_dup = ‘last_updated’
   commonConfig = {
   “hoodie.datasource.hive_sync.database”: db_name,
   ‘hoodie.table.name’: tableName,
 ‘hoodie.datasource.hive_sync.support_timestamp’: ‘true’,
   ‘hoodie.datasource.write.recordkey.field’: pk,
   ‘hoodie.datasource.write.precombine.field’: de_dup,
   ‘hoodie.datasource.hive_sync.enable’: ‘true’,
   ‘hoodie.datasource.hive_sync.table’:  tableName
   }
   nonPartitionConfig = {
   ‘hoodie.datasource.hive_sync.partition_extractor_class’:
   ‘org.apache.hudi.hive.NonPartitionedExtractor’,
   ‘hoodie.datasource.write.keygenerator.class’:
   ‘org.apache.hudi.keygen.NonpartitionedKeyGenerator’
   }
   config = {
‘hoodie.bulkinsert.shuffle.parallelism’: 10,
   ‘hoodie.datasource.write.operation’: ‘bulk_insert’
   }
   S3Location = ‘s3://<>/hudi/tst_tb’
   combinedConf = {**commonConfig, **nonPartitionConfig, **config}
   df.write.format(‘org.apache.hudi’).options(
   **combinedConf).mode(‘overwrite’).save(S3Location)
   
   
   
Environment Description
   
   EMR: emr-6.5.0
   Hudi version : 0.9
   Spark version : Spark 3.1.2
   Hive version : Hive 3.1.2
   Hadoop version :Storage (HDFS/S3/GCS..) : S3
   Running on Docker? (yes/no) : no
   
   
   
   Source Data
   
   +--+--+--+
   |id|creation_date |last_updated  |
   +--+--+--+
   |7cb15b859e|2021-11-07 08:48:25.000232|2021-11-08 08:50:35.000359|
   |60ab5da73a|2022-07-02 19:48:27.000891|2022-07-03 20:05:19.000364|
   |abb663a826|2015-07-12 15:35:14   |2015-08-01 15:38:07   |
   |c92aaeedc1|2021-05-10 16:47:10.000455|2021-05-30 16:49:29.00063 |
   +--+--+--+
   
   Source Schema
   
   root
|-- id: string (nullable = true)
|-- creation_date: timestamp (nullable = true)
|-- last_updated: timestamp (nullable = true)
   
   
   Hudi 0.9V Output
   
   
+---++--+--+-+--+---+---+
   
|_hoodie_commit_time|_hoodie_commit_seqno|_hoodie_record_key|_hoodie_partition_path|_hoodie_file_name
|id|creation_date   
   |last_updated   |
   
+---++--+--+-+--+---+---+
   |20220728035114 |20220728035114_3_2  |c92aaeedc1|
  
|1736fb90-f6b2-4282-9c77-da2ace4bf0bd-0_3-10-80_20220728035114.parquet|c92aaeedc1|2021-05-10
 16:47:10|2021-05-30 16:49:29|
   |20220728035114 |20220728035114_1_3  |7cb15b859e|
  
|d650a502-386e-47b9-81f3-e72cf64b0c0e-0_1-10-78_20220728035114.parquet|7cb15b859e|2021-11-07
 08:48:25|2021-11-08 08:50:35|
   |20220728035114 |20220728035114_2_1  |abb663a826|
  
|941ca621-111e-47d9-8ca1-bdc943490371-0_2-10-79_20220728035114.parquet|abb663a826|2015-07-12
 15:35:14|2015-08-01 15:38:07|
   |20220728035114 |20220728035114_0_1  |60ab5da73a|
  |2d2fb872-7775-4b2d-bd28-93c289ae12c8-0_0-8-77_20220728035114.parquet 
|60ab5da73a|2022-07-02 19:48:27|2022-07-03 20:05:19|
   
+---++--+--+-+--+---+---+
   
   Hudi 0.11V Output
   
   
+---+-+--+--+-+--+--+--+
   |_hoodie_commit_time|_hoodie_commit_seqno 
|_hoodie_record_key|_hoodie_partition_path|_hoodie_file_name
|id|creation_date 
|last_updated  |
   

[GitHub] [hudi] hudi-bot commented on pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


hudi-bot commented on PR #6016:
URL: https://github.com/apache/hudi/pull/6016#issuecomment-1197622320

   
   ## CI report:
   
   * a04e54da5a30fad61de9db4350ed4c9b18f148dd Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10415)
 
   * 9e57627ffddf0eedcd613b54e0f5be5d54462b0b Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10416)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


hudi-bot commented on PR #6016:
URL: https://github.com/apache/hudi/pull/6016#issuecomment-1197595779

   
   ## CI report:
   
   * 178c9f74162b9a420bf42e08a7609cedf1db7104 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10412)
 
   * a04e54da5a30fad61de9db4350ed4c9b18f148dd Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10415)
 
   * 9e57627ffddf0eedcd613b54e0f5be5d54462b0b Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10416)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[hudi] branch master updated: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible (#6229)

2022-07-27 Thread xushiyan
This is an automated email from the ASF dual-hosted git repository.

xushiyan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
 new 0a5ce000bf [HUDI-4490] Make AWSDmsAvroPayload class backwards 
compatible (#6229)
0a5ce000bf is described below

commit 0a5ce000bfbd8a9696f08c763b1863939465627a
Author: Rahil C <32500120+rahi...@users.noreply.github.com>
AuthorDate: Wed Jul 27 19:55:06 2022 -0700

[HUDI-4490] Make AWSDmsAvroPayload class backwards compatible (#6229)

Co-authored-by: Rahil Chertara 
---
 .../hudi/common/model/AWSDmsAvroPayload.java   |  3 +-
 .../hudi/common/model}/TestAWSDmsAvroPayload.java  |  7 +---
 .../org/apache/hudi/payload/AWSDmsAvroPayload.java | 48 ++
 3 files changed, 51 insertions(+), 7 deletions(-)

diff --git 
a/hudi-common/src/main/java/org/apache/hudi/common/model/AWSDmsAvroPayload.java 
b/hudi-common/src/main/java/org/apache/hudi/common/model/AWSDmsAvroPayload.java
index c515338ee6..20a20fb629 100644
--- 
a/hudi-common/src/main/java/org/apache/hudi/common/model/AWSDmsAvroPayload.java
+++ 
b/hudi-common/src/main/java/org/apache/hudi/common/model/AWSDmsAvroPayload.java
@@ -18,11 +18,10 @@
 
 package org.apache.hudi.common.model;
 
-import org.apache.hudi.common.util.Option;
-
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.IndexedRecord;
+import org.apache.hudi.common.util.Option;
 
 import java.io.IOException;
 import java.util.Properties;
diff --git 
a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/payload/TestAWSDmsAvroPayload.java
 
b/hudi-common/src/test/java/org/apache/hudi/common/model/TestAWSDmsAvroPayload.java
similarity index 96%
rename from 
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/payload/TestAWSDmsAvroPayload.java
rename to 
hudi-common/src/test/java/org/apache/hudi/common/model/TestAWSDmsAvroPayload.java
index 7373553dfd..5ba537269e 100644
--- 
a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/payload/TestAWSDmsAvroPayload.java
+++ 
b/hudi-common/src/test/java/org/apache/hudi/common/model/TestAWSDmsAvroPayload.java
@@ -16,16 +16,13 @@
  * limitations under the License.
  */
 
-package org.apache.hudi.payload;
-
-import org.apache.hudi.common.model.AWSDmsAvroPayload;
-import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
-import org.apache.hudi.common.util.Option;
+package org.apache.hudi.common.model;
 
 import org.apache.avro.Schema;
 import org.apache.avro.generic.GenericData;
 import org.apache.avro.generic.GenericRecord;
 import org.apache.avro.generic.IndexedRecord;
+import org.apache.hudi.common.util.Option;
 import org.junit.jupiter.api.Test;
 
 import static org.junit.jupiter.api.Assertions.assertFalse;
diff --git 
a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/payload/AWSDmsAvroPayload.java
 
b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/payload/AWSDmsAvroPayload.java
new file mode 100644
index 00..1411d4f479
--- /dev/null
+++ 
b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/payload/AWSDmsAvroPayload.java
@@ -0,0 +1,48 @@
+/*
+ * 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.payload;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.hudi.common.util.Option;
+
+/**
+ * Provides support for seamlessly applying changes captured via Amazon 
Database Migration Service onto S3.
+ *
+ * Typically, we get the following pattern of full change records 
corresponding to DML against the
+ * source database
+ *
+ * - Full load records with no `Op` field
+ * - For inserts against the source table, records contain full after image 
with `Op=I`
+ * - For updates against the source table, records contain full after image 
with `Op=U`
+ * - For deletes against the source table, records contain full before image 
with `Op=D`
+ *
+ * This payload implementation will issue matching insert, delete, updates 
against the hudi table
+ *
+ */
+@Deprecated
+public class AWSDmsAvroPayload extends 
org.apache.hudi.common.model.AWSDmsAvroPayload 

[GitHub] [hudi] xushiyan merged pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


xushiyan merged PR #6229:
URL: https://github.com/apache/hudi/pull/6229


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


hudi-bot commented on PR #6016:
URL: https://github.com/apache/hudi/pull/6016#issuecomment-1197588724

   
   ## CI report:
   
   * 178c9f74162b9a420bf42e08a7609cedf1db7104 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10412)
 
   * a04e54da5a30fad61de9db4350ed4c9b18f148dd Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10415)
 
   * 9e57627ffddf0eedcd613b54e0f5be5d54462b0b UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


hudi-bot commented on PR #6229:
URL: https://github.com/apache/hudi/pull/6229#issuecomment-1197586525

   
   ## CI report:
   
   * 0c435a9e68878fd2a8239fd6f2efc281e27e82b4 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10414)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


hudi-bot commented on PR #6016:
URL: https://github.com/apache/hudi/pull/6016#issuecomment-1197586258

   
   ## CI report:
   
   * 178c9f74162b9a420bf42e08a7609cedf1db7104 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10412)
 
   * a04e54da5a30fad61de9db4350ed4c9b18f148dd Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10415)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a diff in pull request #5716: [HUDI-4167] Remove the timeline refresh with initializing hoodie table

2022-07-27 Thread GitBox


danny0405 commented on code in PR #5716:
URL: https://github.com/apache/hudi/pull/5716#discussion_r931725039


##
hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java:
##
@@ -199,6 +199,7 @@ protected void resetViewState() {
 LOG.info("Deleting all rocksdb data associated with table filesystem 
view");
 rocksDB.close();
 rocksDB = new RocksDBDAO(metaClient.getBasePath(), 
config.getRocksdbBasePath());
+schemaHelper.getAllColumnFamilies().forEach(rocksDB::addColumnFamily);

Review Comment:
   It is a necessary fix actually, to fix the refresh of the fs view.



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a diff in pull request #5716: [HUDI-4167] Remove the timeline refresh with initializing hoodie table

2022-07-27 Thread GitBox


danny0405 commented on code in PR #5716:
URL: https://github.com/apache/hudi/pull/5716#discussion_r931721050


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##
@@ -364,8 +364,8 @@ public class HoodieWriteConfig extends HoodieConfig {
 
   public static final ConfigProperty 
REFRESH_TIMELINE_SERVER_BASED_ON_LATEST_COMMIT = ConfigProperty
   .key("hoodie.refresh.timeline.server.based.on.latest.commit")
-  .defaultValue(false)
-  .withDocumentation("Refresh timeline in timeline server based on latest 
commit apart from timeline hash difference. By default (false), ");
+  .defaultValue(true)

Review Comment:
   > hy can't we keep it as false
   
   It is actually hard code to be true before this patch and this config option 
is fixed to be effective only after this PR.
   
   Take a look at `TimelineService#refreshTimelineBasedOnLatestCommit`, it's 
weird that the member is hard code to be true and there is no way to config it 
before this patch, config option 
`REFRESH_TIMELINE_SERVER_BASED_ON_LATEST_COMMIT` is totally ignored.



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a diff in pull request #5716: [HUDI-4167] Remove the timeline refresh with initializing hoodie table

2022-07-27 Thread GitBox


danny0405 commented on code in PR #5716:
URL: https://github.com/apache/hudi/pull/5716#discussion_r931721050


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##
@@ -364,8 +364,8 @@ public class HoodieWriteConfig extends HoodieConfig {
 
   public static final ConfigProperty 
REFRESH_TIMELINE_SERVER_BASED_ON_LATEST_COMMIT = ConfigProperty
   .key("hoodie.refresh.timeline.server.based.on.latest.commit")
-  .defaultValue(false)
-  .withDocumentation("Refresh timeline in timeline server based on latest 
commit apart from timeline hash difference. By default (false), ");
+  .defaultValue(true)

Review Comment:
   > hy can't we keep it as false
   
   It is actually hard code to be true before this patch and this config option 
is fixed to be effective only after this PR.
   
   Take a look at
   
[TimelineService.java](https://github.com/apache/hudi/pull/5716/files#diff-f6c3ec8d427d1c5586445abd044ac8c94131bc1a8d40482b85a2815ca32b03f7)
   
   And it's weird that this config options is totally ignored and hard code to 
be true.



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a diff in pull request #5716: [HUDI-4167] Remove the timeline refresh with initializing hoodie table

2022-07-27 Thread GitBox


danny0405 commented on code in PR #5716:
URL: https://github.com/apache/hudi/pull/5716#discussion_r931721763


##
hudi-common/src/main/java/org/apache/hudi/metadata/HoodieBackedTableMetadata.java:
##
@@ -567,6 +564,16 @@ private synchronized void close(Pair 
partitionFileSlicePair) {
 closeReader(readers);
   }
 
+  /**
+   * Close and clear all the partitions readers.
+   */
+  private void closePartitionReaders() {

Review Comment:
   No, it is related, because in this patch, we refresh the metadata table when 
the fs view is refreshed, and there are some states in the readers we need to 
clear about.



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a diff in pull request #5716: [HUDI-4167] Remove the timeline refresh with initializing hoodie table

2022-07-27 Thread GitBox


danny0405 commented on code in PR #5716:
URL: https://github.com/apache/hudi/pull/5716#discussion_r931721050


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##
@@ -364,8 +364,8 @@ public class HoodieWriteConfig extends HoodieConfig {
 
   public static final ConfigProperty 
REFRESH_TIMELINE_SERVER_BASED_ON_LATEST_COMMIT = ConfigProperty
   .key("hoodie.refresh.timeline.server.based.on.latest.commit")
-  .defaultValue(false)
-  .withDocumentation("Refresh timeline in timeline server based on latest 
commit apart from timeline hash difference. By default (false), ");
+  .defaultValue(true)

Review Comment:
   > hy can't we keep it as false
   
   It is actually hard code to be true before this patch and this config option 
is fixed to be effective only after this PR.



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] danny0405 commented on a diff in pull request #5716: [HUDI-4167] Remove the timeline refresh with initializing hoodie table

2022-07-27 Thread GitBox


danny0405 commented on code in PR #5716:
URL: https://github.com/apache/hudi/pull/5716#discussion_r931721050


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##
@@ -364,8 +364,8 @@ public class HoodieWriteConfig extends HoodieConfig {
 
   public static final ConfigProperty 
REFRESH_TIMELINE_SERVER_BASED_ON_LATEST_COMMIT = ConfigProperty
   .key("hoodie.refresh.timeline.server.based.on.latest.commit")
-  .defaultValue(false)
-  .withDocumentation("Refresh timeline in timeline server based on latest 
commit apart from timeline hash difference. By default (false), ");
+  .defaultValue(true)

Review Comment:
   > hy can't we keep it as false
   
   It is actually hard code to be true before this patch and this config option 
is fixed to be effective only in this PR.



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


alexeykudinkin commented on code in PR #6016:
URL: https://github.com/apache/hudi/pull/6016#discussion_r931715112


##
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##
@@ -244,12 +255,7 @@ private Map 
loadPartitionPathFiles() {
   );
 
   fetchedPartitionToFiles =
-  FSUtils.getFilesInPartitions(
-  engineContext,
-  metadataConfig,
-  basePath,
-  fullPartitionPathsMapToFetch.keySet().toArray(new String[0]),
-  fileSystemStorageConfig.getSpillableDir())

Review Comment:
   Good call



##
hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java:
##
@@ -124,28 +125,28 @@ public static HoodieTableMetaClient init(Configuration 
hadoopConf, String basePa
   }
 
   public static HoodieTableMetaClient init(Configuration hadoopConf, String 
basePath, HoodieTableType tableType,
-   Properties properties)
-  throws IOException {
-properties = HoodieTableMetaClient.withPropertyBuilder()
-  .setTableName(RAW_TRIPS_TEST_NAME)
-  .setTableType(tableType)
-  .setPayloadClass(HoodieAvroPayload.class)
-  .fromProperties(properties)
-  .build();
-return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, 
basePath, properties);
+   Properties properties) throws 
IOException {
+return init(hadoopConf, basePath, tableType, properties, null);
   }
 
   public static HoodieTableMetaClient init(Configuration hadoopConf, String 
basePath, HoodieTableType tableType,
Properties properties, String 
databaseName)
   throws IOException {
-properties = HoodieTableMetaClient.withPropertyBuilder()
-  .setDatabaseName(databaseName)
-  .setTableName(RAW_TRIPS_TEST_NAME)
-  .setTableType(tableType)
-  .setPayloadClass(HoodieAvroPayload.class)
-  .fromProperties(properties)
-  .build();
-return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, 
basePath, properties);
+HoodieTableMetaClient.PropertyBuilder builder =
+HoodieTableMetaClient.withPropertyBuilder()
+.setDatabaseName(databaseName)
+.setTableName(RAW_TRIPS_TEST_NAME)
+.setTableType(tableType)
+.setPayloadClass(HoodieAvroPayload.class);
+
+String keyGen = 
properties.getProperty("hoodie.datasource.write.keygenerator.class");
+if (!Objects.equals(keyGen, 
"org.apache.hudi.keygen.NonpartitionedKeyGenerator")) {
+  builder.setPartitionFields("some_nonexistent_field");

Review Comment:
   I don't think we should actually standardize on this one, it's just to stop 
the bleeding in misconfigured tests



##
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##
@@ -324,6 +335,26 @@ private void doRefresh() {
 LOG.info(String.format("Refresh table %s, spent: %d ms", 
metaClient.getTableConfig().getTableName(), duration));
   }
 
+  private Map 
getAllFilesInPartitionsUnchecked(Collection 
fullPartitionPathsMapToFetch) {
+try {
+  return tableMetadata.getAllFilesInPartitions(new 
ArrayList<>(fullPartitionPathsMapToFetch));
+} catch (IOException e) {
+  throw new HoodieIOException("Failed to list partition paths for a 
table", e);
+}
+  }
+
+  private List getAllPartitionPathsUnchecked() {
+try {
+  if (partitionColumns.length == 0) {
+return Collections.singletonList("");

Review Comment:
   Non-partitioned table has exactly one partition, which we designate w/ ""



##
hudi-common/src/main/java/org/apache/hudi/hadoop/CachingPath.java:
##
@@ -32,11 +33,12 @@
  * NOTE: This class is thread-safe
  */
 @ThreadSafe
-public class CachingPath extends Path implements Serializable {

Review Comment:
   Yep



##
hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java:
##
@@ -138,13 +144,17 @@ public FileStatus[] getAllFilesInPartition(Path 
partitionPath)
   }
 }
 
-return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, 
dataBasePath, metadataConfig.shouldAssumeDatePartitioning())
+return new FileSystemBackedTableMetadata(getEngineContext(), hadoopConf, 
dataBasePath.toString(), metadataConfig.shouldAssumeDatePartitioning())
 .getAllFilesInPartition(partitionPath);
   }
 
   @Override
   public Map getAllFilesInPartitions(List 
partitions)
   throws IOException {
+if (partitions.isEmpty()) {
+  return Collections.emptyMap();

Review Comment:
   Agree, this is somewhat dissonant, but that's just the way things are -- for 
non-partitioned tables it's assumed that the only partition that is there has 
to be identified by ""



##
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java:
##
@@ -398,7 +398,7 @@ public 

[GitHub] [hudi] xiarixiaoyao commented on a diff in pull request #6213: [HUDI-4081][HUDI-4472] Addressing Spark SQL vs Spark DS performance gap

2022-07-27 Thread GitBox


xiarixiaoyao commented on code in PR #6213:
URL: https://github.com/apache/hudi/pull/6213#discussion_r931716050


##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##
@@ -241,39 +240,49 @@ object HoodieSparkSqlWriter {
 sparkContext.getConf.registerKryoClasses(
   Array(classOf[org.apache.avro.generic.GenericData],
 classOf[org.apache.avro.Schema]))
-var schema = 
AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, 
nameSpace)
-val lastestSchema = getLatestTableSchema(fs, basePath, 
sparkContext, schema)
+
+// TODO(HUDI-4472) revisit and simplify schema handling
+val sourceSchema = 
AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, 
nameSpace)
+val latestTableSchema = getLatestTableSchema(fs, basePath, 
sparkContext).getOrElse(sourceSchema)
+
+val enabledSchemaEvolution = 
parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), 
"false").toBoolean
 var internalSchemaOpt = getLatestTableInternalSchema(fs, basePath, 
sparkContext)
-if (reconcileSchema && 
parameters.getOrDefault(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key(), 
"false").toBoolean
-  && internalSchemaOpt.isEmpty) {
-  // force apply full schema evolution.
-  internalSchemaOpt = 
Some(AvroInternalSchemaConverter.convert(schema))
-}
-if (reconcileSchema) {
-  schema = lastestSchema
-}
-if (internalSchemaOpt.isDefined) {
-  // Apply schema evolution.
-  val mergedSparkSchema = if (!reconcileSchema) {
-
AvroConversionUtils.convertAvroSchemaToStructType(AvroSchemaEvolutionUtils.canonicalizeColumnNullability(schema,
 lastestSchema))
+
+val writerSchema: Schema =
+  if (reconcileSchema) {
+// In case we need to reconcile the schema and schema 
evolution is enabled,
+// we will force-apply schema evolution to the writer's schema
+if (enabledSchemaEvolution && internalSchemaOpt.isEmpty) {
+  internalSchemaOpt = 
Some(AvroInternalSchemaConverter.convert(sourceSchema))
+}
+
+if (internalSchemaOpt.isDefined) {
+  // Apply schema evolution, by auto-merging write schema and 
read schema
+  val mergedInternalSchema = 
AvroSchemaEvolutionUtils.reconcileSchema(sourceSchema, internalSchemaOpt.get)
+  AvroInternalSchemaConverter.convert(mergedInternalSchema, 
latestTableSchema.getName)
+} else if 
(TableSchemaResolver.isSchemaCompatible(sourceSchema, latestTableSchema)) {
+  // In case schema reconciliation is enabled and source and 
latest table schemas
+  // are compatible (as defined by 
[[TableSchemaResolver#isSchemaCompatible]], then we will
+  // pick latest table's schema as the writer's schema
+  latestTableSchema
+} else {
+  // Otherwise fallback to original source's schema
+  sourceSchema
+}
   } else {
-// Auto merge write schema and read schema.
-val mergedInternalSchema = 
AvroSchemaEvolutionUtils.reconcileSchema(schema, internalSchemaOpt.get)
-
AvroConversionUtils.convertAvroSchemaToStructType(AvroInternalSchemaConverter.convert(mergedInternalSchema,
 lastestSchema.getName))
+// In case reconciliation is disabled, we still have to do 
nullability attributes
+// (minor) reconciliation, making sure schema of the incoming 
batch is in-line with
+// the data already committed in the table
+
AvroSchemaEvolutionUtils.canonicalizeColumnNullability(sourceSchema, 
latestTableSchema)

Review Comment:
   Notice:
   The avro Schema created by 
AvroSchemaEvolutionUtils.canonicalizecolumnnullability  has a different 
namespce with the schema created by 
AvroConversionUtils.convertStructTypeToAvroSchema.
   Avro 1.8.2 is Namespace sensitive.
   In the original logic, we will eventually call 
AvroConversionUtils.convertStructTypeToAvroSchema to maintain namespace 
consistency
   



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


hudi-bot commented on PR #6016:
URL: https://github.com/apache/hudi/pull/6016#issuecomment-1197559880

   
   ## CI report:
   
   * 178c9f74162b9a420bf42e08a7609cedf1db7104 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10412)
 
   * a04e54da5a30fad61de9db4350ed4c9b18f148dd UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] codope commented on a diff in pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


codope commented on code in PR #6016:
URL: https://github.com/apache/hudi/pull/6016#discussion_r931683314


##
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##
@@ -244,12 +255,7 @@ private Map 
loadPartitionPathFiles() {
   );
 
   fetchedPartitionToFiles =
-  FSUtils.getFilesInPartitions(
-  engineContext,
-  metadataConfig,
-  basePath,
-  fullPartitionPathsMapToFetch.keySet().toArray(new String[0]),
-  fileSystemStorageConfig.getSpillableDir())

Review Comment:
   So if this is not needed then let's remove and avoid instantiating 
`fileSystemStorageConfig` in the constructor.



##
hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableCompaction.java:
##
@@ -98,8 +99,13 @@ public void testWriteDuringCompaction() throws IOException {
 .withLayoutConfig(HoodieLayoutConfig.newBuilder()
 .withLayoutType(HoodieStorageLayout.LayoutType.BUCKET.name())
 
.withLayoutPartitioner(SparkBucketIndexPartitioner.class.getName()).build())
-
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BUCKET).withBucketNum("1").build()).build();
-metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, 
config.getProps());
+
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BUCKET).withBucketNum("1").build())
+.build();
+
+Properties props = getPropertiesForKeyGen(true);

Review Comment:
   pass `HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()` instead of 
hard-coding true?



##
hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java:
##
@@ -353,12 +395,14 @@ public String getPath() {
   return path;
 }
 
-Path fullPartitionPath(String basePath) {
+Path fullPartitionPath(Path basePath) {
   if (!path.isEmpty()) {
-return new CachingPath(basePath, path);
+// NOTE: Since we now that the path is a proper relative path that 
doesn't require

Review Comment:
   ```suggestion
   // NOTE: Since we know that the path is a proper relative path that 
doesn't require
   ```



##
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java:
##
@@ -398,7 +398,7 @@ public HoodieArchivedTimeline getArchivedTimeline(String 
startTs) {
   public void validateTableProperties(Properties properties) {
 // Once meta fields are disabled, it cant be re-enabled for a given table.
 if (!getTableConfig().populateMetaFields()
-&& Boolean.parseBoolean((String) 
properties.getOrDefault(HoodieTableConfig.POPULATE_META_FIELDS.key(), 
HoodieTableConfig.POPULATE_META_FIELDS.defaultValue( {
+&& Boolean.parseBoolean((String) 
properties.getOrDefault(HoodieTableConfig.POPULATE_META_FIELDS.key(), 
HoodieTableConfig.POPULATE_META_FIELDS.defaultValue().toString( {

Review Comment:
   is it necessary? it's already being type cast to String



##
hudi-common/src/main/java/org/apache/hudi/common/table/log/block/HoodieHFileDataBlock.java:
##
@@ -167,9 +164,8 @@ protected ClosableIterator 
deserializeRecords(byte[] content) thr
 // Get schema from the header
 Schema writerSchema = new 
Schema.Parser().parse(super.getLogBlockHeader().get(HeaderMetadataType.SCHEMA));
 
-FileSystem fs = FSUtils.getFs(pathForReader.toString(), new 
Configuration());
 // Read the content
-HoodieHFileReader reader = new HoodieHFileReader<>(fs, 
pathForReader, content, Option.of(writerSchema));
+HoodieHFileReader reader = new HoodieHFileReader<>(null, 
pathForReader, content, Option.of(writerSchema));

Review Comment:
   This could affect HFile reading. I believe there is some validation in HFile 
system or HFile's reader context for fs to be non-null. I think we should still 
pass `fs` and still keep this line in `HoodieHFileUtils#createHFileReader`: 
   ```
   Configuration conf = new Configuration(false);
   ```



##
hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java:
##
@@ -124,28 +125,28 @@ public static HoodieTableMetaClient init(Configuration 
hadoopConf, String basePa
   }
 
   public static HoodieTableMetaClient init(Configuration hadoopConf, String 
basePath, HoodieTableType tableType,
-   Properties properties)
-  throws IOException {
-properties = HoodieTableMetaClient.withPropertyBuilder()
-  .setTableName(RAW_TRIPS_TEST_NAME)
-  .setTableType(tableType)
-  .setPayloadClass(HoodieAvroPayload.class)
-  .fromProperties(properties)
-  .build();
-return HoodieTableMetaClient.initTableAndGetMetaClient(hadoopConf, 
basePath, properties);
+   Properties properties) throws 
IOException {
+return 

[GitHub] [hudi] hudi-bot commented on pull request #6231: [MINOR] minor changes around Spark 3.3 support

2022-07-27 Thread GitBox


hudi-bot commented on PR #6231:
URL: https://github.com/apache/hudi/pull/6231#issuecomment-1197555407

   
   ## CI report:
   
   * 57fcca6295342110e69d4e48e55e21c59ae0514b Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10411)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5771: [HUDI-4071] Relax record key requirement and write with minimal options

2022-07-27 Thread GitBox


alexeykudinkin commented on code in PR #5771:
URL: https://github.com/apache/hudi/pull/5771#discussion_r931694390


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##
@@ -115,7 +115,7 @@ public class HoodieWriteConfig extends HoodieConfig {
 
   public static final ConfigProperty PRECOMBINE_FIELD_NAME = 
ConfigProperty
   .key("hoodie.datasource.write.precombine.field")
-  .defaultValue("ts")
+  .noDefaultValue()

Review Comment:
   Need to call this out as non-BWC
   



##
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java:
##
@@ -64,6 +63,9 @@ public String getPartitionPath(GenericRecord record) {
 
   @Override
   public String getRecordKey(Row row) {

Review Comment:
   You also need to handle this in `InternalRow` one



##
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java:
##
@@ -393,18 +396,28 @@ public HoodieArchivedTimeline getArchivedTimeline(String 
startTs) {
 
   /**
* Validate table properties.
-   * @param properties Properties from writeConfig.
+   * @param writeConfigProps Properties from writeConfig.
*/
-  public void validateTableProperties(Properties properties) {
+  public void validateTableProperties(Properties writeConfigProps) {
+// Once table is configured to be append-only, it cannot be mutable or 
allow setting record key or precombine fields for updates
+if (getTableConfig().isAppendOnlyTable()) {
+  boolean appendOnlyTable = Boolean.parseBoolean(
+  
writeConfigProps.getProperty(HoodieTableConfig.APPEND_ONLY_TABLE.key(), 
String.valueOf(HoodieTableConfig.APPEND_ONLY_TABLE.defaultValue(;

Review Comment:
   Why do we need to specify on the write-config as well?



##
hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java:
##
@@ -235,6 +235,13 @@ public class HoodieTableConfig extends HoodieConfig {
   .withDocumentation("Comma-separated list of metadata partitions that 
have been completely built and in-sync with data table. "
   + "These partitions are ready for use by the readers");
 
+  public static final ConfigProperty APPEND_ONLY_TABLE = 
ConfigProperty
+  .key("hoodie.table.append.only")

Review Comment:
   Boolean flags are really hard to evolve and better reserved for toggle-like 
configs (switching on/off).
   
   In that case if we'd consider adding "table-type" we'd be exposed to can of 
warms of banning various permutations of this configs.
   
   I'd suggest to generalize this to be a "table-type" config which we can 
declare either "mutable" or "immutable"



##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##
@@ -791,6 +809,13 @@ object HoodieSparkSqlWriter {
 if (mergedParams.contains(PRECOMBINE_FIELD.key())) {
   mergedParams.put(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, 
mergedParams(PRECOMBINE_FIELD.key()))
 }
+// for the first ingest batch, if applicable, disable populate meta fields 
and enable append-only table config
+if (tableConfig == null
+  && !mergedParams.contains(RECORDKEY_FIELD.key)
+  && !mergedParams.contains(PRECOMBINE_FIELD.key)) {
+  mergedParams.put(HoodieTableConfig.POPULATE_META_FIELDS.key, 
(!HoodieTableConfig.POPULATE_META_FIELDS.defaultValue).toString)

Review Comment:
   We should just go w/ `false` here (inverse default makes it cryptic)



##
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/keygen/ComplexKeyGenerator.java:
##
@@ -64,6 +63,9 @@ public String getPartitionPath(GenericRecord record) {
 
   @Override
   public String getRecordKey(Row row) {
+if (recordKeyFields.isEmpty()) {
+  return EMPTY_STRING;

Review Comment:
   I'd suggest create a method `emptyKey` and redirect there so that if we 
decided to change the sentinel value we can switch it easily



##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala:
##
@@ -788,9 +790,10 @@ object DataSourceOptionsHelper {
 val partitionFields = 
props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), null)
 if (partitionFields != null) {
   val numPartFields = partitionFields.split(",").length
-  val recordsKeyFields = 
props.getString(DataSourceWriteOptions.RECORDKEY_FIELD.key(), 
DataSourceWriteOptions.RECORDKEY_FIELD.defaultValue())
-  val numRecordKeyFields = recordsKeyFields.split(",").length
-  if (numPartFields == 1 && numRecordKeyFields == 1) {
+  val numRecordKeyFields =
+if(props.contains(DataSourceWriteOptions.RECORDKEY_FIELD.key)) 
props.getString(DataSourceWriteOptions.RECORDKEY_FIELD.key).split(",").length

Review Comment:
   Spacing



##
hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala:
##

[GitHub] [hudi] hudi-bot commented on pull request #5771: [HUDI-4071] Relax record key requirement and write with minimal options

2022-07-27 Thread GitBox


hudi-bot commented on PR #5771:
URL: https://github.com/apache/hudi/pull/5771#issuecomment-1197550309

   
   ## CI report:
   
   * 0c014b0e842b20d4f4253b8dc480a4e38ac5e8d0 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10413)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6230: [HUDI-4478] Rename existing spark/flink modules with a concise name

2022-07-27 Thread GitBox


hudi-bot commented on PR #6230:
URL: https://github.com/apache/hudi/pull/6230#issuecomment-1197548195

   
   ## CI report:
   
   * 0262c3a012e6912ceffb8186dae4d50d9e5ebeeb Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10410)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


hudi-bot commented on PR #6229:
URL: https://github.com/apache/hudi/pull/6229#issuecomment-1197548181

   
   ## CI report:
   
   * d246c75a8724e5d3d4d046c0237d6a9c96fb8c93 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10409)
 
   * 0c435a9e68878fd2a8239fd6f2efc281e27e82b4 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10414)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


hudi-bot commented on PR #6016:
URL: https://github.com/apache/hudi/pull/6016#issuecomment-1197547958

   
   ## CI report:
   
   * 178c9f74162b9a420bf42e08a7609cedf1db7104 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10412)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


hudi-bot commented on PR #6229:
URL: https://github.com/apache/hudi/pull/6229#issuecomment-1197521091

   
   ## CI report:
   
   * 0cef10f25e931b40319bc1a9d0d0b33985533b34 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10408)
 
   * d246c75a8724e5d3d4d046c0237d6a9c96fb8c93 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10409)
 
   * 0c435a9e68878fd2a8239fd6f2efc281e27e82b4 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10414)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


hudi-bot commented on PR #6229:
URL: https://github.com/apache/hudi/pull/6229#issuecomment-1197518857

   
   ## CI report:
   
   * 0cef10f25e931b40319bc1a9d0d0b33985533b34 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10408)
 
   * d246c75a8724e5d3d4d046c0237d6a9c96fb8c93 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10409)
 
   * 0c435a9e68878fd2a8239fd6f2efc281e27e82b4 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #5771: [HUDI-4071] Relax record key requirement and write with minimal options

2022-07-27 Thread GitBox


hudi-bot commented on PR #5771:
URL: https://github.com/apache/hudi/pull/5771#issuecomment-1197518471

   
   ## CI report:
   
   * 675a63a88336db52078e3826e2589018d0b06826 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10398)
 
   * 0c014b0e842b20d4f4253b8dc480a4e38ac5e8d0 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10413)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #5771: [HUDI-4071] Relax record key requirement and write with minimal options

2022-07-27 Thread GitBox


hudi-bot commented on PR #5771:
URL: https://github.com/apache/hudi/pull/5771#issuecomment-1197516351

   
   ## CI report:
   
   * 675a63a88336db52078e3826e2589018d0b06826 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10398)
 
   * 0c014b0e842b20d4f4253b8dc480a4e38ac5e8d0 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] rahil-c commented on a diff in pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


rahil-c commented on code in PR #6229:
URL: https://github.com/apache/hudi/pull/6229#discussion_r931681772


##
hudi-common/src/test/java/org/apache/hudi/payload/TestAWSDmsAvroPayload.java:
##
@@ -18,14 +18,13 @@
 
 package org.apache.hudi.payload;

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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


hudi-bot commented on PR #6016:
URL: https://github.com/apache/hudi/pull/6016#issuecomment-1197513714

   
   ## CI report:
   
   * 40e76396c15d5934c90f8163b670c95ef79fa3fa Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10402)
 
   * 178c9f74162b9a420bf42e08a7609cedf1db7104 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10412)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] CTTY commented on pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

2022-07-27 Thread GitBox


CTTY commented on PR #5943:
URL: https://github.com/apache/hudi/pull/5943#issuecomment-1197510513

   Created this umbrella jira and linked existing follow-up jiras to it: 
https://issues.apache.org/jira/browse/HUDI-4492


-- 
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-unsubscr...@hudi.apache.org

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



[jira] [Created] (HUDI-4492) Spark 3.3 support follow up

2022-07-27 Thread Shawn Chang (Jira)
Shawn Chang created HUDI-4492:
-

 Summary: Spark 3.3 support follow up
 Key: HUDI-4492
 URL: https://issues.apache.org/jira/browse/HUDI-4492
 Project: Apache Hudi
  Issue Type: Improvement
Reporter: Shawn Chang


Umbrella Jira for spark 3.3 support follow up jiras



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [hudi] xushiyan commented on a diff in pull request #5716: [HUDI-4167] Remove the timeline refresh with initializing hoodie table

2022-07-27 Thread GitBox


xushiyan commented on code in PR #5716:
URL: https://github.com/apache/hudi/pull/5716#discussion_r931670720


##
hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java:
##
@@ -364,8 +364,8 @@ public class HoodieWriteConfig extends HoodieConfig {
 
   public static final ConfigProperty 
REFRESH_TIMELINE_SERVER_BASED_ON_LATEST_COMMIT = ConfigProperty
   .key("hoodie.refresh.timeline.server.based.on.latest.commit")
-  .defaultValue(false)
-  .withDocumentation("Refresh timeline in timeline server based on latest 
commit apart from timeline hash difference. By default (false), ");
+  .defaultValue(true)

Review Comment:
   This removed in https://github.com/apache/hudi/pull/6179
   but @danny0405 why can't we keep it as false? removing it completely is a 
breaking change. and normally we would at least deprecate it 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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


hudi-bot commented on PR #6016:
URL: https://github.com/apache/hudi/pull/6016#issuecomment-1197487511

   
   ## CI report:
   
   * 40e76396c15d5934c90f8163b670c95ef79fa3fa Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10402)
 
   * 178c9f74162b9a420bf42e08a7609cedf1db7104 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6231: [MINOR] minor changes around Spark 3.3 support

2022-07-27 Thread GitBox


hudi-bot commented on PR #6231:
URL: https://github.com/apache/hudi/pull/6231#issuecomment-1197482863

   
   ## CI report:
   
   * 57fcca6295342110e69d4e48e55e21c59ae0514b Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10411)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


hudi-bot commented on PR #6229:
URL: https://github.com/apache/hudi/pull/6229#issuecomment-1197482812

   
   ## CI report:
   
   * 0cef10f25e931b40319bc1a9d0d0b33985533b34 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10408)
 
   * d246c75a8724e5d3d4d046c0237d6a9c96fb8c93 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10409)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6230: [HUDI-4478] Rename existing spark/flink modules with a concise name

2022-07-27 Thread GitBox


hudi-bot commented on PR #6230:
URL: https://github.com/apache/hudi/pull/6230#issuecomment-1197482833

   
   ## CI report:
   
   * 0262c3a012e6912ceffb8186dae4d50d9e5ebeeb Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10410)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6231: [MINOR] minor changes around Spark 3.3 support

2022-07-27 Thread GitBox


hudi-bot commented on PR #6231:
URL: https://github.com/apache/hudi/pull/6231#issuecomment-1197480026

   
   ## CI report:
   
   * 57fcca6295342110e69d4e48e55e21c59ae0514b UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6230: [HUDI-4478] Rename existing spark/flink modules with a concise name

2022-07-27 Thread GitBox


hudi-bot commented on PR #6230:
URL: https://github.com/apache/hudi/pull/6230#issuecomment-1197480003

   
   ## CI report:
   
   * 0262c3a012e6912ceffb8186dae4d50d9e5ebeeb UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


hudi-bot commented on PR #6229:
URL: https://github.com/apache/hudi/pull/6229#issuecomment-1197479973

   
   ## CI report:
   
   * 0cef10f25e931b40319bc1a9d0d0b33985533b34 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10408)
 
   * d246c75a8724e5d3d4d046c0237d6a9c96fb8c93 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


hudi-bot commented on PR #6229:
URL: https://github.com/apache/hudi/pull/6229#issuecomment-1197477382

   
   ## CI report:
   
   * 0cef10f25e931b40319bc1a9d0d0b33985533b34 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10408)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6228: [HUDI-4488] Improve S3EventsHoodieIncrSource efficiency

2022-07-27 Thread GitBox


hudi-bot commented on PR #6228:
URL: https://github.com/apache/hudi/pull/6228#issuecomment-1197477353

   
   ## CI report:
   
   * 0cc2dbb39e432baf741bb3dd94c6d627cb250297 UNKNOWN
   * a70c45452f5ff6184c668c4209c29b9d2cd598d9 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10404)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6210: [HUDI-4464] Clear warnings in Azure CI

2022-07-27 Thread GitBox


hudi-bot commented on PR #6210:
URL: https://github.com/apache/hudi/pull/6210#issuecomment-1197477300

   
   ## CI report:
   
   * fd505a3e40b653d9c3d12350c37d5b203f25b9b6 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10403)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] xushiyan commented on a diff in pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


xushiyan commented on code in PR #6229:
URL: https://github.com/apache/hudi/pull/6229#discussion_r931653264


##
hudi-common/src/test/java/org/apache/hudi/payload/TestAWSDmsAvroPayload.java:
##
@@ -18,14 +18,13 @@
 
 package org.apache.hudi.payload;

Review Comment:
   the testing class package should be the same as the tested class



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] xushiyan commented on a diff in pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


xushiyan commented on code in PR #6229:
URL: https://github.com/apache/hudi/pull/6229#discussion_r931653264


##
hudi-common/src/test/java/org/apache/hudi/payload/TestAWSDmsAvroPayload.java:
##
@@ -18,14 +18,13 @@
 
 package org.apache.hudi.payload;

Review Comment:
   by right the testing class package should be the same as the tested class



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] rahil-c commented on a diff in pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


rahil-c commented on code in PR #6229:
URL: https://github.com/apache/hudi/pull/6229#discussion_r931650280


##
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/payload/TestAWSDmsAvroPayload.java:
##
@@ -18,7 +18,6 @@
 
 package org.apache.hudi.payload;
 
-import org.apache.hudi.common.model.AWSDmsAvroPayload;

Review Comment:
   fixed



##
hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestAWSDatabaseMigrationServiceSource.java:
##
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.utilities.functional;
 
-import org.apache.hudi.common.model.AWSDmsAvroPayload;
+import org.apache.hudi.payload.AWSDmsAvroPayload;

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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] rahil-c commented on a diff in pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


rahil-c commented on code in PR #6229:
URL: https://github.com/apache/hudi/pull/6229#discussion_r931646618


##
hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/AWSDmsTransformer.java:
##
@@ -19,8 +19,8 @@
 package org.apache.hudi.utilities.transform;
 
 import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.AWSDmsAvroPayload;
 import org.apache.hudi.common.util.Option;
+import org.apache.hudi.payload.AWSDmsAvroPayload;
 
 import org.apache.spark.api.java.JavaSparkContext;

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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] CTTY commented on pull request #6230: [HUDI-4478] Rename existing spark/flink modules with a concise name

2022-07-27 Thread GitBox


CTTY commented on PR #6230:
URL: https://github.com/apache/hudi/pull/6230#issuecomment-1197442457

   As discussed in https://github.com/apache/hudi/pull/5943 , we should have a 
more concise naming pattern for Spark/Flink modules


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] CTTY commented on pull request #6231: [MINOR] minor changes around Spark 3.3 support

2022-07-27 Thread GitBox


CTTY commented on PR #6231:
URL: https://github.com/apache/hudi/pull/6231#issuecomment-1197441763

   Addressing minor comments in https://github.com/apache/hudi/pull/5943


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] CTTY opened a new pull request, #6231: [MINOR] minor changes around Spark 3.3 support

2022-07-27 Thread GitBox


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

   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contribute/how-to-contribute before 
opening a pull request.*
   
   ## What is the purpose of the pull request
   
   *(For example: This pull request adds quick-start document.)*
   
   ## Brief change log
   
   *(for example:)*
 - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## 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-unsubscr...@hudi.apache.org

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



[jira] [Updated] (HUDI-4478) Rename hudi-spark/flink modules

2022-07-27 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-4478?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated HUDI-4478:
-
Labels: pull-request-available  (was: )

> Rename hudi-spark/flink modules
> ---
>
> Key: HUDI-4478
> URL: https://issues.apache.org/jira/browse/HUDI-4478
> Project: Apache Hudi
>  Issue Type: Improvement
>Reporter: Shawn Chang
>Priority: Major
>  Labels: pull-request-available
>
> Existing spark/flink modules are named like hudi-spark-3.1.x/hudi-flink-1.14.x
>  
> We should have a more concise module name like hudi-spark-3.1/hudi-flink-1.14



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [hudi] CTTY opened a new pull request, #6230: [HUDI-4478] Rename existing spark/flink modules with a concise name

2022-07-27 Thread GitBox


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

   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contribute/how-to-contribute before 
opening a pull request.*
   
   ## What is the purpose of the pull request
   
   *(For example: This pull request adds quick-start document.)*
   
   ## Brief change log
   
   *(for example:)*
 - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## 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-unsubscr...@hudi.apache.org

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



[jira] [Updated] (HUDI-4467) Port borrowed code from Spark 3.3

2022-07-27 Thread Shawn Chang (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-4467?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Shawn Chang updated HUDI-4467:
--
Description: 
Currently some classes are copied from Spark32 module w/o/w only necessary 
changes. we should port them from Spark 3.3 to use the latest implementation in 
Spark

 

Classes copied:

Spark33NestedSchemaPruning

  was:
Currently some classes are copied from Spark32 module w/o/w only necessary 
changes. we should port them from Spark 3.3 to use the latest implementation in 
Spark

 

Classes copied:

Spark33NestedSchemaPruning

hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala

hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala

hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/avro/AvroDeserializer.scala


> Port borrowed code from Spark 3.3
> -
>
> Key: HUDI-4467
> URL: https://issues.apache.org/jira/browse/HUDI-4467
> Project: Apache Hudi
>  Issue Type: Improvement
>Reporter: Shawn Chang
>Priority: Major
>
> Currently some classes are copied from Spark32 module w/o/w only necessary 
> changes. we should port them from Spark 3.3 to use the latest implementation 
> in Spark
>  
> Classes copied:
> Spark33NestedSchemaPruning



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Created] (HUDI-4491) Re-enable TestHoodieFlinkQuickstart

2022-07-27 Thread Shawn Chang (Jira)
Shawn Chang created HUDI-4491:
-

 Summary: Re-enable TestHoodieFlinkQuickstart 
 Key: HUDI-4491
 URL: https://issues.apache.org/jira/browse/HUDI-4491
 Project: Apache Hudi
  Issue Type: Bug
Reporter: Shawn Chang


This test was disabled before due to its flakiness. We need to re-enable it 
again



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [hudi] CTTY commented on pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

2022-07-27 Thread GitBox


CTTY commented on PR #5943:
URL: https://github.com/apache/hudi/pull/5943#issuecomment-1197438398

   > @yihua @CTTY the last commit disabled `testHoodieFlinkQuickstart`. I don't 
know why this affects flink tests. Please make a follow up to re-enable it back.
   
   Created this jira to track it: 
https://issues.apache.org/jira/browse/HUDI-4491


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] xushiyan commented on pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

2022-07-27 Thread GitBox


xushiyan commented on PR #5943:
URL: https://github.com/apache/hudi/pull/5943#issuecomment-1197435742

   @yihua @CTTY the last commit disabled `testHoodieFlinkQuickstart`. I don't 
know why this affects flink tests. Please make a follow up to re-enable it back.


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] xushiyan commented on a diff in pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


xushiyan commented on code in PR #6229:
URL: https://github.com/apache/hudi/pull/6229#discussion_r931625286


##
hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/AWSDmsTransformer.java:
##
@@ -19,8 +19,8 @@
 package org.apache.hudi.utilities.transform;
 
 import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.AWSDmsAvroPayload;
 import org.apache.hudi.common.util.Option;
+import org.apache.hudi.payload.AWSDmsAvroPayload;
 
 import org.apache.spark.api.java.JavaSparkContext;

Review Comment:
   why this change? the old class is only meant for bwc where users are 
reflection-loading the class via FQCN. internal codebase should always use the 
new class.



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] xushiyan commented on a diff in pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


xushiyan commented on code in PR #6229:
URL: https://github.com/apache/hudi/pull/6229#discussion_r931625350


##
hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestAWSDatabaseMigrationServiceSource.java:
##
@@ -18,7 +18,7 @@
 
 package org.apache.hudi.utilities.functional;
 
-import org.apache.hudi.common.model.AWSDmsAvroPayload;
+import org.apache.hudi.payload.AWSDmsAvroPayload;

Review Comment:
   ditto



##
hudi-utilities/src/main/java/org/apache/hudi/utilities/transform/AWSDmsTransformer.java:
##
@@ -19,8 +19,8 @@
 package org.apache.hudi.utilities.transform;
 
 import org.apache.hudi.common.config.TypedProperties;
-import org.apache.hudi.common.model.AWSDmsAvroPayload;
 import org.apache.hudi.common.util.Option;
+import org.apache.hudi.payload.AWSDmsAvroPayload;
 
 import org.apache.spark.api.java.JavaSparkContext;

Review Comment:
   why this change? the old class is not meant for bwc where users are loading 
the class via FQCN. internal codebase should always use the new class.



##
hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/payload/TestAWSDmsAvroPayload.java:
##
@@ -18,7 +18,6 @@
 
 package org.apache.hudi.payload;
 
-import org.apache.hudi.common.model.AWSDmsAvroPayload;

Review Comment:
   can this test be moved to hudi-common? where the tested class belongs



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


hudi-bot commented on PR #6229:
URL: https://github.com/apache/hudi/pull/6229#issuecomment-1197430870

   
   ## CI report:
   
   * 0cef10f25e931b40319bc1a9d0d0b33985533b34 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10408)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] xushiyan commented on pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


xushiyan commented on PR #6229:
URL: https://github.com/apache/hudi/pull/6229#issuecomment-1197429140

   follow-up for #6211


-- 
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-unsubscr...@hudi.apache.org

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



[jira] [Updated] (HUDI-4490) Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-4490?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated HUDI-4490:
-
Labels: pull-request-available  (was: )

> Make AWSDmsAvroPayload class backwards compatible
> -
>
> Key: HUDI-4490
> URL: https://issues.apache.org/jira/browse/HUDI-4490
> Project: Apache Hudi
>  Issue Type: Task
>Reporter: Rahil Chertara
>Priority: Major
>  Labels: pull-request-available
> Fix For: 0.12.0
>
>




--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [hudi] hudi-bot commented on pull request #6229: [HUDI-4490] Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


hudi-bot commented on PR #6229:
URL: https://github.com/apache/hudi/pull/6229#issuecomment-1197427386

   
   ## CI report:
   
   * 0cef10f25e931b40319bc1a9d0d0b33985533b34 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6213: [HUDI-4081][HUDI-4472] Addressing Spark SQL vs Spark DS performance gap

2022-07-27 Thread GitBox


hudi-bot commented on PR #6213:
URL: https://github.com/apache/hudi/pull/6213#issuecomment-1197427299

   
   ## CI report:
   
   * f820f87fa1cb12f1a787e5fec320a52263669b03 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10397)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] alexeykudinkin commented on pull request #6213: [HUDI-4081][HUDI-4472] Addressing Spark SQL vs Spark DS performance gap

2022-07-27 Thread GitBox


alexeykudinkin commented on PR #6213:
URL: https://github.com/apache/hudi/pull/6213#issuecomment-1197426788

   CI is green:
   https://user-images.githubusercontent.com/428277/181381604-212f56dd-3d28-4375-8bd2-9665fb1cb2bd.png;>
   
   
https://dev.azure.com/apache-hudi-ci-org/apache-hudi-ci/_build/results?buildId=10397=results
   
   


-- 
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-unsubscr...@hudi.apache.org

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



[jira] [Created] (HUDI-4490) Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread Rahil Chertara (Jira)
Rahil Chertara created HUDI-4490:


 Summary: Make AWSDmsAvroPayload class backwards compatible
 Key: HUDI-4490
 URL: https://issues.apache.org/jira/browse/HUDI-4490
 Project: Apache Hudi
  Issue Type: Task
Reporter: Rahil Chertara
 Fix For: 0.12.0






--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [hudi] xushiyan commented on a diff in pull request #6228: [HUDI-4488] Improve S3EventsHoodieIncrSource efficiency

2022-07-27 Thread GitBox


xushiyan commented on code in PR #6228:
URL: https://github.com/apache/hudi/pull/6228#discussion_r931617796


##
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java:
##
@@ -172,37 +177,47 @@ public Pair>, String> 
fetchNextBatch(Option lastCkpt
 String s3FS = props.getString(Config.S3_FS_PREFIX, "s3").toLowerCase();
 String s3Prefix = s3FS + "://";
 
-// Extract distinct file keys from s3 meta hoodie table
-final List cloudMetaDf = source
+// Create S3 paths
+final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
+SerializableConfiguration serializableConfiguration = new 
SerializableConfiguration(sparkContext.hadoopConfiguration());
+List cloudFiles = source
 .filter(filter)
 .select("s3.bucket.name", "s3.object.key")
 .distinct()
-.collectAsList();
-// Create S3 paths
-final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
-List cloudFiles = new ArrayList<>();
-for (Row row : cloudMetaDf) {
-  // construct file path, row index 0 refers to bucket and 1 refers to key
-  String bucket = row.getString(0);
-  String filePath = s3Prefix + bucket + "/" + row.getString(1);
-  if (checkExists) {
-FileSystem fs = FSUtils.getFs(s3Prefix + bucket, 
sparkSession.sparkContext().hadoopConfiguration());
-try {
-  if (fs.exists(new Path(filePath))) {
-cloudFiles.add(filePath);
-  }
-} catch (IOException e) {
-  LOG.error(String.format("Error while checking path exists for %s ", 
filePath), e);
-}
-  } else {
-cloudFiles.add(filePath);
-  }
-}
+.rdd().toJavaRDD().mapPartitions(fileListIterator -> {

Review Comment:
   think I mentioned in previous PR: why convert to rdd and can't we 
mapPartitions directly?



##
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java:
##
@@ -172,37 +177,47 @@ public Pair>, String> 
fetchNextBatch(Option lastCkpt
 String s3FS = props.getString(Config.S3_FS_PREFIX, "s3").toLowerCase();
 String s3Prefix = s3FS + "://";
 
-// Extract distinct file keys from s3 meta hoodie table
-final List cloudMetaDf = source
+// Create S3 paths
+final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
+SerializableConfiguration serializableConfiguration = new 
SerializableConfiguration(sparkContext.hadoopConfiguration());
+List cloudFiles = source
 .filter(filter)
 .select("s3.bucket.name", "s3.object.key")
 .distinct()
-.collectAsList();
-// Create S3 paths
-final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
-List cloudFiles = new ArrayList<>();
-for (Row row : cloudMetaDf) {
-  // construct file path, row index 0 refers to bucket and 1 refers to key
-  String bucket = row.getString(0);
-  String filePath = s3Prefix + bucket + "/" + row.getString(1);
-  if (checkExists) {
-FileSystem fs = FSUtils.getFs(s3Prefix + bucket, 
sparkSession.sparkContext().hadoopConfiguration());
-try {
-  if (fs.exists(new Path(filePath))) {
-cloudFiles.add(filePath);
-  }
-} catch (IOException e) {
-  LOG.error(String.format("Error while checking path exists for %s ", 
filePath), e);
-}
-  } else {
-cloudFiles.add(filePath);
-  }
-}
+.rdd().toJavaRDD().mapPartitions(fileListIterator -> {
+  List cloudFilesPerPartition = new ArrayList<>();
+  fileListIterator.forEachRemaining(row -> {
+final Configuration configuration = 
serializableConfiguration.newCopy();
+String bucket = row.getString(0);
+String filePath = s3Prefix + bucket + "/" + row.getString(1);
+try {
+  String decodeUrl = URLDecoder.decode(filePath, 
StandardCharsets.UTF_8.name());
+  if (checkExists) {
+FileSystem fs = FSUtils.getFs(s3Prefix + bucket, 
configuration);
+try {

Review Comment:
   this nested if else try catch block should be simplified. there are some 
hoodie utils that allow throwing



##
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java:
##
@@ -172,37 +177,47 @@ public Pair>, String> 
fetchNextBatch(Option lastCkpt
 String s3FS = props.getString(Config.S3_FS_PREFIX, "s3").toLowerCase();
 String s3Prefix = s3FS + "://";
 
-// Extract distinct file keys from s3 meta hoodie table
-final List cloudMetaDf = source
+// Create S3 paths
+final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
+ 

[GitHub] [hudi] hudi-bot commented on pull request #5771: [HUDI-4071] Relax record key requirement and write with minimal options

2022-07-27 Thread GitBox


hudi-bot commented on PR #5771:
URL: https://github.com/apache/hudi/pull/5771#issuecomment-1197422958

   
   ## CI report:
   
   * 675a63a88336db52078e3826e2589018d0b06826 Azure: 
[SUCCESS](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10398)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[hudi] branch master updated: [HUDI-4126] Disable file splits for Bootstrap real time queries (via InputFormat) (#6219)

2022-07-27 Thread xushiyan
This is an automated email from the ASF dual-hosted git repository.

xushiyan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


The following commit(s) were added to refs/heads/master by this push:
 new 51599af281 [HUDI-4126] Disable file splits for Bootstrap real time 
queries (via InputFormat) (#6219)
51599af281 is described below

commit 51599af2818562b6cea9bd01bd81af363209a2d2
Author: Rahil C <32500120+rahi...@users.noreply.github.com>
AuthorDate: Wed Jul 27 14:58:29 2022 -0700

[HUDI-4126] Disable file splits for Bootstrap real time queries (via 
InputFormat) (#6219)


Co-authored-by: Udit Mehrotra 
Co-authored-by: Raymond Xu <2701446+xushi...@users.noreply.github.com>
---
 .../hudi/hadoop/realtime/HoodieRealtimePath.java   |  2 +-
 .../TestHoodieCopyOnWriteTableInputFormat.java | 60 +++
 .../TestHoodieMergeOnReadTableInputFormat.java | 68 ++
 3 files changed, 129 insertions(+), 1 deletion(-)

diff --git 
a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java
 
b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java
index bba44d5c66..1f1dd1b927 100644
--- 
a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java
+++ 
b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/realtime/HoodieRealtimePath.java
@@ -89,7 +89,7 @@ public class HoodieRealtimePath extends Path {
   }
 
   public boolean isSplitable() {
-return !toString().isEmpty();
+return !toString().isEmpty() && !includeBootstrapFilePath();
   }
 
   public PathWithBootstrapFileStatus getPathWithBootstrapFileStatus() {
diff --git 
a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieCopyOnWriteTableInputFormat.java
 
b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieCopyOnWriteTableInputFormat.java
new file mode 100644
index 00..902778ed1c
--- /dev/null
+++ 
b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieCopyOnWriteTableInputFormat.java
@@ -0,0 +1,60 @@
+/*
+ * 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.hadoop;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Files;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+
+public class TestHoodieCopyOnWriteTableInputFormat {
+
+  @TempDir
+  java.nio.file.Path tempDir;
+  private FileSystem fs;
+
+  @BeforeEach
+  void setUp() throws IOException {
+fs = FileSystem.get(tempDir.toUri(), new Configuration());
+  }
+
+  @AfterEach
+  void tearDown() throws IOException {
+fs.close();
+  }
+
+  @Test
+  void pathNotSplitableForBootstrapScenario() throws IOException {
+URI source = Files.createTempFile(tempDir, "source", ".parquet").toUri();
+URI target = Files.createTempFile(tempDir, "target", ".parquet").toUri();
+PathWithBootstrapFileStatus path = new PathWithBootstrapFileStatus(new 
Path(target), fs.getFileStatus(new Path(source)));
+HoodieCopyOnWriteTableInputFormat cowInputFormat = new 
HoodieCopyOnWriteTableInputFormat();
+assertFalse(cowInputFormat.isSplitable(fs, path), "Path for bootstrap 
should not be splitable.");
+  }
+}
diff --git 
a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieMergeOnReadTableInputFormat.java
 
b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieMergeOnReadTableInputFormat.java
new file mode 100644
index 00..d44f5fbf63
--- /dev/null
+++ 
b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieMergeOnReadTableInputFormat.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 

[GitHub] [hudi] xushiyan merged pull request #6219: [HUDI-4126] Disable file splits for Bootstrap real time queries (via InputFormat)

2022-07-27 Thread GitBox


xushiyan merged PR #6219:
URL: https://github.com/apache/hudi/pull/6219


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] xushiyan commented on pull request #6219: [HUDI-4126] Disable file splits for Bootstrap real time queries (via InputFormat)

2022-07-27 Thread GitBox


xushiyan commented on PR #6219:
URL: https://github.com/apache/hudi/pull/6219#issuecomment-1197413464

   ![Screen Shot 2022-07-27 at 4 57 57 
PM](https://user-images.githubusercontent.com/2701446/181379738-4ee24b41-18ac-4361-becb-a5b83a3ae2ac.png)
   
   CI passed.


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] xushiyan commented on a diff in pull request #6219: [HUDI-4126] Disable file splits for Bootstrap real time queries (via InputFormat)

2022-07-27 Thread GitBox


xushiyan commented on code in PR #6219:
URL: https://github.com/apache/hudi/pull/6219#discussion_r931615083


##
hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieCopyOnWriteTableInputFormat.java:
##
@@ -0,0 +1,60 @@
+/*
+ * 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.hadoop;

Review Comment:
   the test follows src's package 
org.apache.hudi.hadoop.HoodieCopyOnWriteTableInputFormat
   
   plus the API is also protecte



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] yihua commented on a diff in pull request #6219: [HUDI-4126] Disable file splits for Bootstrap real time queries (via InputFormat)

2022-07-27 Thread GitBox


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


##
hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieCopyOnWriteTableInputFormat.java:
##
@@ -0,0 +1,60 @@
+/*
+ * 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.hadoop;

Review Comment:
   Should this be under `org.apache.hudi.hadoop.realtime` as well?



-- 
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-unsubscr...@hudi.apache.org

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



[hudi] branch master updated (924c30c7ea -> cdaec5a8da)

2022-07-27 Thread yihua
This is an automated email from the ASF dual-hosted git repository.

yihua pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/hudi.git


from 924c30c7ea [HUDI-4469] Flip reuse flag to true in 
HoodieBackedTableMetadata to improve file listing (#6214)
 add cdaec5a8da [HUDI-4186] Support Hudi with Spark 3.3.0 (#5943)

No new revisions were added by this update.

Summary of changes:
 .github/workflows/bot.yml  |   5 +-
 .../apache/hudi/config/HoodieStorageConfig.java|  15 ++
 .../org/apache/hudi/config/HoodieWriteConfig.java  |   4 +
 .../HoodieSparkBootstrapSchemaProvider.java|  18 +-
 .../storage/row/HoodieRowParquetWriteSupport.java  |   1 +
 .../scala/org/apache/hudi/HoodieSparkUtils.scala   |   2 +
 .../org/apache/hudi/SparkAdapterSupport.scala  |   4 +-
 .../org/apache/spark/sql/hudi/SparkAdapter.scala   |  48 -
 .../quickstart/TestHoodieFlinkQuickstart.java  |   2 +
 hudi-examples/hudi-examples-spark/pom.xml  |   6 +
 hudi-spark-datasource/README.md|  17 +-
 .../org/apache/hudi/BaseFileOnlyRelation.scala |   6 +-
 .../spark/sql/hudi/analysis/HoodieAnalysis.scala   |  36 ++--
 .../hudi/command/DeleteHoodieTableCommand.scala|   7 +-
 .../spark/sql/parser/HoodieCommonSqlParser.scala   |   8 +
 .../spark/sql/hudi/HoodieSparkSqlTestBase.scala|   5 +-
 .../sql/hudi/TestAlterTableDropPartition.scala |  11 +-
 .../hudi/TestNestedSchemaPruningOptimization.scala |   1 +
 .../sql/hudi/procedure/TestCallCommandParser.scala |  11 +-
 .../org/apache/hudi/Spark2HoodieFileScanRDD.scala} |  13 +-
 .../apache/spark/sql/adapter/Spark2Adapter.scala   |  35 +++-
 .../apache/hudi/spark3/internal/ReflectUtil.java   |   2 +-
 .../spark/sql/adapter/BaseSpark3Adapter.scala  |  17 --
 .../apache/hudi/Spark31HoodieFileScanRDD.scala}|  13 +-
 .../apache/spark/sql/adapter/Spark3_1Adapter.scala |  37 +++-
 .../datasources/Spark31NestedSchemaPruning.scala}  |   2 +-
 ... => Spark312ResolveHudiAlterTableCommand.scala} |   2 +-
 ...scala => HoodieSpark3_1ExtendedSqlParser.scala} |   2 +-
 .../{hudi-spark3 => hudi-spark3.2.x}/pom.xml   |  65 +++---
 .../src/main/antlr4/imports/SqlBase.g4 |   0
 .../apache/hudi/spark/sql/parser/HoodieSqlBase.g4  |   0
 ...Spark32HoodieVectorizedParquetRecordReader.java |   0
 ...org.apache.spark.sql.sources.DataSourceRegister |   0
 .../apache/hudi/Spark32HoodieFileScanRDD.scala}|  13 +-
 .../org/apache/hudi/Spark3DefaultSource.scala  |   0
 .../sql/HoodieSpark32CatalystExpressionUtils.scala |   0
 .../spark/sql/HoodieSpark32CatalystPlanUtils.scala |   0
 .../apache/spark/sql/adapter/Spark3_2Adapter.scala |  34 ++-
 .../apache/spark/sql/avro/AvroDeserializer.scala   |   0
 .../org/apache/spark/sql/avro/AvroSerializer.scala |   0
 .../org/apache/spark/sql/avro/AvroUtils.scala  |   0
 .../sql/avro/HoodieSpark3_2AvroDeserializer.scala  |   0
 .../sql/avro/HoodieSpark3_2AvroSerializer.scala|   0
 .../plans/logical/TimeTravelRelation.scala |   0
 .../sql/connector/catalog/HoodieIdentifier.scala   |   0
 .../datasources/Spark32NestedSchemaPruning.scala}  |   2 +-
 .../parquet/Spark32DataSourceUtils.scala   |   0
 .../parquet/Spark32HoodieParquetFileFormat.scala   |   0
 .../Spark32ResolveHudiAlterTableCommand.scala} |   2 +-
 .../sql/hudi/analysis/HoodieSpark3Analysis.scala   |   0
 .../spark/sql/hudi/catalog/BasicStagedTable.scala  |   0
 .../spark/sql/hudi/catalog/HoodieCatalog.scala |   4 +-
 .../sql/hudi/catalog/HoodieInternalV2Table.scala   |   0
 .../spark/sql/hudi/catalog/HoodieStagedTable.scala |   0
 .../spark/sql/hudi/catalog/TableCreationMode.java  |   0
 .../spark/sql/hudi/command/AlterTableCommand.scala |   0
 .../HoodieSpark3_2ExtendedSqlAstBuilder.scala  |   0
 .../parser/HoodieSpark3_2ExtendedSqlParser.scala   |   0
 .../TestHoodieBulkInsertDataInternalWriter.java|   0
 .../TestHoodieDataSourceInternalBatchWrite.java|   0
 .../hudi/spark3/internal/TestReflectUtil.java  |   0
 .../test/resources/log4j-surefire-quiet.properties |   0
 .../src/test/resources/log4j-surefire.properties   |   0
 .../{hudi-spark3 => hudi-spark3.3.x}/pom.xml   |  14 +-
 .../src/main/antlr4/imports/SqlBase.g4 |   0
 .../apache/hudi/spark/sql/parser/HoodieSqlBase.g4  |   0
 ...park33HoodieVectorizedParquetRecordReader.java} |   4 +-
 ...org.apache.spark.sql.sources.DataSourceRegister |   0
 .../apache/hudi/Spark33HoodieFileScanRDD.scala}|  13 +-
 .../org/apache/hudi/Spark3DefaultSource.scala  |   0
 .../HoodieSpark33CatalystExpressionUtils.scala}|   6 +-
 .../sql/HoodieSpark33CatalystPlanUtils.scala}  |   2 +-
 .../apache/spark/sql/adapter/Spark3_3Adapter.scala |  80 
 .../apache/spark/sql/avro/AvroDeserializer.scala   | 171 +++-
 .../org/apache/spark/sql/avro/AvroSerializer.scala | 151 +-
 .../org/apache/spark/sql/avro/AvroUtils.scala  | 228 

[GitHub] [hudi] yihua merged pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

2022-07-27 Thread GitBox


yihua merged PR #5943:
URL: https://github.com/apache/hudi/pull/5943


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] yihua commented on a diff in pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

2022-07-27 Thread GitBox


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


##
hudi-client/hudi-spark-client/src/main/scala/org/apache/spark/sql/hudi/SparkAdapter.scala:
##
@@ -143,6 +141,38 @@ trait SparkAdapter extends Serializable {
*/
   def createInterpretedPredicate(e: Expression): InterpretedPredicate
 
+  /**
+   * Create instance of [[HoodieFileScanRDD]]
+   * SPARK-37273 FileScanRDD constructor changed in SPARK 3.3
+   */
+  def createHoodieFileScanRDD(sparkSession: SparkSession,
+  readFunction: PartitionedFile => 
Iterator[InternalRow],
+  filePartitions: Seq[FilePartition],
+  readDataSchema: StructType,
+  metadataColumns: Seq[AttributeReference] = 
Seq.empty): FileScanRDD
+
+  /**
+   * Resolve [[DeleteFromTable]]
+   * SPARK-38626 condition is no longer Option in Spark 3.3
+   */
+  def resolveDeleteFromTable(deleteFromTable: Command,
+ resolveExpression: Expression => Expression): 
LogicalPlan
+
+  /**
+   * Extract condition in [[DeleteFromTable]]
+   * SPARK-38626 condition is no longer Option in Spark 3.3
+   */
+  def extractCondition(deleteFromTable: Command): Expression

Review Comment:
   nit: rename to `extractDeleteCondition`?



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] CTTY commented on pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

2022-07-27 Thread GitBox


CTTY commented on PR #5943:
URL: https://github.com/apache/hudi/pull/5943#issuecomment-1197390727

   > Created those Jiras below to follow up on improving the code quality:
   > 
   > * https://issues.apache.org/jira/browse/HUDI-4466  Re-use common code 
between Spark 3.2/3.3
   > * https://issues.apache.org/jira/browse/HUDI-4467 Port borrowed code from 
Spark 3.3
   > * https://issues.apache.org/jira/browse/HUDI-4468 Time travel logic 
simplification
   > * https://issues.apache.org/jira/browse/HUDI-4489 Break down 
HoodieAnalysis rules into Spark-specific components
   > * https://issues.apache.org/jira/browse/HUDI-4481 Refactor 
HoodieCommonSqlParser
   > 
   > @alexeykudinkin @XuQianJin-Stars @yihua
   
   Updated follow-up jiras


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] CTTY commented on a diff in pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

2022-07-27 Thread GitBox


CTTY commented on code in PR #5943:
URL: https://github.com/apache/hudi/pull/5943#discussion_r931598774


##
hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_3Adapter.scala:
##
@@ -0,0 +1,80 @@
+/*
+ * 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.adapter
+
+import org.apache.avro.Schema
+import org.apache.hudi.Spark33HoodieFileScanRDD
+import org.apache.spark.sql.avro._
+import org.apache.spark.sql.catalyst.expressions.{AttributeReference, 
Expression}
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, 
PartitionedFile}
+import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, 
Spark33HoodieParquetFileFormat}
+import org.apache.spark.sql.parser.HoodieSpark3_3ExtendedSqlParser
+import org.apache.spark.sql.types.{DataType, StructType}
+import org.apache.spark.sql.{HoodieCatalystExpressionUtils, 
HoodieCatalystPlansUtils, HoodieSpark33CatalystPlanUtils, 
HoodieSpark33CatalystExpressionUtils, SparkSession}
+
+/**
+ * Implementation of [[SparkAdapter]] for Spark 3.3.x branch
+ */
+class Spark3_3Adapter extends BaseSpark3Adapter {
+
+  override def getCatalystExpressionUtils: HoodieCatalystExpressionUtils = 
HoodieSpark33CatalystExpressionUtils
+
+  override def getCatalystPlanUtils: HoodieCatalystPlansUtils = 
HoodieSpark33CatalystPlanUtils
+
+  override def createAvroSerializer(rootCatalystType: DataType, rootAvroType: 
Schema, nullable: Boolean): HoodieAvroSerializer =
+new HoodieSpark3_3AvroSerializer(rootCatalystType, rootAvroType, nullable)
+
+  override def createAvroDeserializer(rootAvroType: Schema, rootCatalystType: 
DataType): HoodieAvroDeserializer =
+new HoodieSpark3_3AvroDeserializer(rootAvroType, rootCatalystType)
+
+  override def createExtendedSparkParser: Option[(SparkSession, 
ParserInterface) => ParserInterface] = {
+Some(
+  (spark: SparkSession, delegate: ParserInterface) => new 
HoodieSpark3_3ExtendedSqlParser(spark, delegate)
+)
+  }
+
+  override def createHoodieParquetFileFormat(appendPartitionValues: Boolean): 
Option[ParquetFileFormat] = {
+Some(new Spark33HoodieParquetFileFormat(appendPartitionValues))
+  }
+
+  override def createHoodieFileScanRDD(sparkSession: SparkSession,
+   readFunction: PartitionedFile => 
Iterator[InternalRow],
+   filePartitions: Seq[FilePartition],
+   readDataSchema: StructType,
+   metadataColumns: 
Seq[AttributeReference] = Seq.empty): FileScanRDD = {
+new Spark33HoodieFileScanRDD(sparkSession, readFunction, filePartitions, 
readDataSchema, metadataColumns)
+  }
+
+  override def resolveDeleteFromTable(deleteFromTable: Command,
+  resolveExpression: Expression => 
Expression): DeleteFromTable = {
+val deleteFromTableCommand = deleteFromTable.asInstanceOf[DeleteFromTable]
+DeleteFromTable(deleteFromTableCommand.table, 
resolveExpression(deleteFromTableCommand.condition))
+  }
+
+  override def extractCondition(deleteFromTable: Command): Expression = {
+deleteFromTable.asInstanceOf[DeleteFromTable].condition
+  }
+
+  override def getQueryParserFromExtendedSqlParser(session: SparkSession, 
delegate: ParserInterface,
+   sqlText: String): 
LogicalPlan = {
+new HoodieSpark3_3ExtendedSqlParser(session, delegate).parseQuery(sqlText)

Review Comment:
   > Created those Jiras below to follow up on improving the code quality:
   > 
   > * https://issues.apache.org/jira/browse/HUDI-4466  Re-use common code 
between Spark 3.2/3.3
   > * https://issues.apache.org/jira/browse/HUDI-4467 Port borrowed code from 
Spark 3.3
   > * https://issues.apache.org/jira/browse/HUDI-4468 Time travel logic 
simplification
   > * https://issues.apache.org/jira/browse/HUDI-4489 Break down 
HoodieAnalysis rules into Spark-specific components
   > * 

[GitHub] [hudi] yihua commented on a diff in pull request #6228: [HUDI-4488] Improve S3 file listing efficiency

2022-07-27 Thread GitBox


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


##
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java:
##
@@ -172,37 +177,47 @@ public Pair>, String> 
fetchNextBatch(Option lastCkpt
 String s3FS = props.getString(Config.S3_FS_PREFIX, "s3").toLowerCase();
 String s3Prefix = s3FS + "://";
 
-// Extract distinct file keys from s3 meta hoodie table
-final List cloudMetaDf = source
+// Create S3 paths
+final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
+SerializableConfiguration serializableConfiguration = new 
SerializableConfiguration(sparkContext.hadoopConfiguration());
+List cloudFiles = source
 .filter(filter)
 .select("s3.bucket.name", "s3.object.key")
 .distinct()
-.collectAsList();
-// Create S3 paths
-final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
-List cloudFiles = new ArrayList<>();
-for (Row row : cloudMetaDf) {
-  // construct file path, row index 0 refers to bucket and 1 refers to key
-  String bucket = row.getString(0);
-  String filePath = s3Prefix + bucket + "/" + row.getString(1);
-  if (checkExists) {
-FileSystem fs = FSUtils.getFs(s3Prefix + bucket, 
sparkSession.sparkContext().hadoopConfiguration());
-try {
-  if (fs.exists(new Path(filePath))) {
-cloudFiles.add(filePath);
-  }
-} catch (IOException e) {
-  LOG.error(String.format("Error while checking path exists for %s ", 
filePath), e);
-}
-  } else {
-cloudFiles.add(filePath);
-  }
-}
+.rdd().toJavaRDD().mapPartitions(fileListIterator -> {
+  List cloudFilesPerPartition = new ArrayList<>();
+  fileListIterator.forEachRemaining(row -> {
+final Configuration configuration = 
serializableConfiguration.newCopy();

Review Comment:
   Might be good to have a feature flag to control which logic to use?  Or 
that's overkill?  cc @xushiyan @codope 



##
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java:
##
@@ -172,37 +177,47 @@ public Pair>, String> 
fetchNextBatch(Option lastCkpt
 String s3FS = props.getString(Config.S3_FS_PREFIX, "s3").toLowerCase();
 String s3Prefix = s3FS + "://";
 
-// Extract distinct file keys from s3 meta hoodie table
-final List cloudMetaDf = source
+// Create S3 paths
+final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
+SerializableConfiguration serializableConfiguration = new 
SerializableConfiguration(sparkContext.hadoopConfiguration());
+List cloudFiles = source
 .filter(filter)
 .select("s3.bucket.name", "s3.object.key")
 .distinct()
-.collectAsList();
-// Create S3 paths
-final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
-List cloudFiles = new ArrayList<>();
-for (Row row : cloudMetaDf) {
-  // construct file path, row index 0 refers to bucket and 1 refers to key
-  String bucket = row.getString(0);
-  String filePath = s3Prefix + bucket + "/" + row.getString(1);
-  if (checkExists) {
-FileSystem fs = FSUtils.getFs(s3Prefix + bucket, 
sparkSession.sparkContext().hadoopConfiguration());
-try {
-  if (fs.exists(new Path(filePath))) {
-cloudFiles.add(filePath);
-  }
-} catch (IOException e) {
-  LOG.error(String.format("Error while checking path exists for %s ", 
filePath), e);
-}
-  } else {
-cloudFiles.add(filePath);
-  }
-}
+.rdd().toJavaRDD().mapPartitions(fileListIterator -> {
+  List cloudFilesPerPartition = new ArrayList<>();
+  fileListIterator.forEachRemaining(row -> {
+final Configuration configuration = 
serializableConfiguration.newCopy();

Review Comment:
   Could this be outside the loop instead of copying the config in each 
iteration?



##
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java:
##
@@ -172,37 +177,47 @@ public Pair>, String> 
fetchNextBatch(Option lastCkpt
 String s3FS = props.getString(Config.S3_FS_PREFIX, "s3").toLowerCase();
 String s3Prefix = s3FS + "://";
 
-// Extract distinct file keys from s3 meta hoodie table
-final List cloudMetaDf = source
+// Create S3 paths
+final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
+SerializableConfiguration serializableConfiguration = new 
SerializableConfiguration(sparkContext.hadoopConfiguration());
+List cloudFiles = source
 .filter(filter)
  

[jira] [Created] (HUDI-4489) Break down HoodieAnalysis rules into Spark-specific components

2022-07-27 Thread Alexey Kudinkin (Jira)
Alexey Kudinkin created HUDI-4489:
-

 Summary: Break down HoodieAnalysis rules into Spark-specific 
components
 Key: HUDI-4489
 URL: https://issues.apache.org/jira/browse/HUDI-4489
 Project: Apache Hudi
  Issue Type: Bug
Reporter: Alexey Kudinkin
Assignee: Alexey Kudinkin
 Fix For: 0.13.0
 Attachments: image-2022-07-27-14-24-03-190.png

Currently, `HoodieAnalysis` hosts Rules that are going to be universally 
applied to all Spark versions even though they don't have to:

For ex, Hudi back-ports `DeleteFromTable`, `UpdateTable` commands to Spark 2, 
requiring corresponding resolution to occur:

!image-2022-07-27-14-24-03-190.png|width=512,height=144!

 

However, this rule should not be applied for Spark 3, which has its own rules 
applied to DeleteFromTable alas other commands



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [hudi] rahil-c commented on pull request #6229: Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


rahil-c commented on PR #6229:
URL: https://github.com/apache/hudi/pull/6229#issuecomment-1197384633

   cc @xushiyan 


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] rahil-c opened a new pull request, #6229: Make AWSDmsAvroPayload class backwards compatible

2022-07-27 Thread GitBox


rahil-c opened a new pull request, #6229:
URL: https://github.com/apache/hudi/pull/6229

   ## *Tips*
   - *Thank you very much for contributing to Apache Hudi.*
   - *Please review https://hudi.apache.org/contribute/how-to-contribute before 
opening a pull request.*
   
   ## What is the purpose of the pull request
   
   *(For example: This pull request adds quick-start document.)*
   
   ## Brief change log
   
   *(for example:)*
 - *Modify AnnotationLocation checkstyle rule in checkstyle.xml*
   
   ## 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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] alexeykudinkin commented on pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

2022-07-27 Thread GitBox


alexeykudinkin commented on PR #5943:
URL: https://github.com/apache/hudi/pull/5943#issuecomment-1197380346

   @CTTY please add the Jiras in the description so that they're more easily 
discoverable


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


hudi-bot commented on PR #6016:
URL: https://github.com/apache/hudi/pull/6016#issuecomment-1197377856

   
   ## CI report:
   
   * 40e76396c15d5934c90f8163b670c95ef79fa3fa Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10402)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6228: [HUDI-4488] Improve S3 file listing efficiency

2022-07-27 Thread GitBox


hudi-bot commented on PR #6228:
URL: https://github.com/apache/hudi/pull/6228#issuecomment-1197374231

   
   ## CI report:
   
   * 0cc2dbb39e432baf741bb3dd94c6d627cb250297 UNKNOWN
   * a70c45452f5ff6184c668c4209c29b9d2cd598d9 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10404)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

2022-07-27 Thread GitBox


alexeykudinkin commented on code in PR #5943:
URL: https://github.com/apache/hudi/pull/5943#discussion_r931585854


##
hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala:
##
@@ -122,6 +123,30 @@ class Spark2Adapter extends SparkAdapter {
 InterpretedPredicate.create(e)
   }
 
+  override def createHoodieFileScanRDD(sparkSession: SparkSession,
+   readFunction: PartitionedFile => 
Iterator[InternalRow],
+   filePartitions: Seq[FilePartition],
+   readDataSchema: StructType,
+   metadataColumns: 
Seq[AttributeReference] = Seq.empty): FileScanRDD = {
+new Spark2HoodieFileScanRDD(sparkSession, readFunction, filePartitions)
+  }
+
+  override def resolveDeleteFromTable(deleteFromTable: Command,
+  resolveExpression: Expression => 
Expression): DeleteFromTable = {
+val deleteFromTableCommand = deleteFromTable.asInstanceOf[DeleteFromTable]
+val resolvedCondition = 
deleteFromTableCommand.condition.map(resolveExpression)
+DeleteFromTable(deleteFromTableCommand.table, resolvedCondition)
+  }
+
+  override def extractCondition(deleteFromTable: Command): Expression = {

Review Comment:
   Let's also return `Option` instead of 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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

2022-07-27 Thread GitBox


alexeykudinkin commented on code in PR #5943:
URL: https://github.com/apache/hudi/pull/5943#discussion_r931585204


##
hudi-spark-datasource/hudi-spark2/src/main/scala/org/apache/spark/sql/adapter/Spark2Adapter.scala:
##
@@ -122,6 +123,30 @@ class Spark2Adapter extends SparkAdapter {
 InterpretedPredicate.create(e)
   }
 
+  override def createHoodieFileScanRDD(sparkSession: SparkSession,
+   readFunction: PartitionedFile => 
Iterator[InternalRow],
+   filePartitions: Seq[FilePartition],
+   readDataSchema: StructType,
+   metadataColumns: 
Seq[AttributeReference] = Seq.empty): FileScanRDD = {
+new Spark2HoodieFileScanRDD(sparkSession, readFunction, filePartitions)
+  }
+
+  override def resolveDeleteFromTable(deleteFromTable: Command,

Review Comment:
   I don't see why we can't: 
   
   - We get rid of the method completely 
   - We use `extractCondition` to extract condition and then do everything else 
(resolution, etc) in the caller



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] alexeykudinkin commented on a diff in pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

2022-07-27 Thread GitBox


alexeykudinkin commented on code in PR #5943:
URL: https://github.com/apache/hudi/pull/5943#discussion_r931583487


##
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/parser/HoodieCommonSqlParser.scala:
##
@@ -57,6 +57,14 @@ class HoodieCommonSqlParser(session: SparkSession, delegate: 
ParserInterface)
 
   override def parseDataType(sqlText: String): DataType = 
delegate.parseDataType(sqlText)
 
+  /* SPARK-37266 Added parseQuery to ParserInterface in Spark 3.3.0. This is a 
patch to prevent
+   hackers from tampering text with persistent view, it won't be called in 
older Spark
+   Don't mark this as override for backward compatibility
+   Can't use sparkExtendedParser directly here due to the same reason */

Review Comment:
   As discussed on Slack, let's instead of doing parsing in `SparkAdapter` 
create `ExtendedParserInterface`, where we can place this new `parseQuery` 
method and that could be used in Hudi's code-base (this is similar to how 
`HoodieCatalystExpressionUtils` set up)
   



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6228: [HUDI-4488] Improve S3 file listing efficiency

2022-07-27 Thread GitBox


hudi-bot commented on PR #6228:
URL: https://github.com/apache/hudi/pull/6228#issuecomment-1197369996

   
   ## CI report:
   
   * 0cc2dbb39e432baf741bb3dd94c6d627cb250297 UNKNOWN
   * a70c45452f5ff6184c668c4209c29b9d2cd598d9 UNKNOWN
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6213: [HUDI-4081][HUDI-4472] Addressing Spark SQL vs Spark DS performance gap

2022-07-27 Thread GitBox


hudi-bot commented on PR #6213:
URL: https://github.com/apache/hudi/pull/6213#issuecomment-1197369889

   
   ## CI report:
   
   * f820f87fa1cb12f1a787e5fec320a52263669b03 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10397)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6210: [HUDI-4464] Clear warnings in Azure CI

2022-07-27 Thread GitBox


hudi-bot commented on PR #6210:
URL: https://github.com/apache/hudi/pull/6210#issuecomment-1197369854

   
   ## CI report:
   
   * c0d3947dc56adc15f4a8090536dfccd2ca6b4bc3 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10318)
 
   * fd505a3e40b653d9c3d12350c37d5b203f25b9b6 Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10403)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] hudi-bot commented on pull request #6016: [HUDI-4465] Optimizing file-listing sequence of Metadata Table

2022-07-27 Thread GitBox


hudi-bot commented on PR #6016:
URL: https://github.com/apache/hudi/pull/6016#issuecomment-1197369475

   
   ## CI report:
   
   * fe477a848ec801ed01dc8fea53a3a8237f9b8034 Azure: 
[FAILURE](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10371)
 
   * 40e76396c15d5934c90f8163b670c95ef79fa3fa Azure: 
[PENDING](https://dev.azure.com/apache-hudi-ci-org/785b6ef4-2f42-4a89-8f0e-5f0d7039a0cc/_build/results?buildId=10402)
 
   
   
   Bot commands
 @hudi-bot supports the following commands:
   
- `@hudi-bot run azure` re-run the last Azure build
   


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] vamshigv commented on a diff in pull request #6228: [HUDI-4488] Improve S3 file listing efficiency

2022-07-27 Thread GitBox


vamshigv commented on code in PR #6228:
URL: https://github.com/apache/hudi/pull/6228#discussion_r931582094


##
hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/S3EventsHoodieIncrSource.java:
##
@@ -172,37 +177,47 @@ public Pair>, String> 
fetchNextBatch(Option lastCkpt
 String s3FS = props.getString(Config.S3_FS_PREFIX, "s3").toLowerCase();
 String s3Prefix = s3FS + "://";
 
-// Extract distinct file keys from s3 meta hoodie table
-final List cloudMetaDf = source
+// Create S3 paths
+final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
+SerializableConfiguration serializableConfiguration = new 
SerializableConfiguration(sparkContext.hadoopConfiguration());
+List cloudFiles = source
 .filter(filter)
 .select("s3.bucket.name", "s3.object.key")
 .distinct()
-.collectAsList();
-// Create S3 paths
-final boolean checkExists = props.getBoolean(Config.ENABLE_EXISTS_CHECK, 
Config.DEFAULT_ENABLE_EXISTS_CHECK);
-List cloudFiles = new ArrayList<>();
-for (Row row : cloudMetaDf) {
-  // construct file path, row index 0 refers to bucket and 1 refers to key
-  String bucket = row.getString(0);
-  String filePath = s3Prefix + bucket + "/" + row.getString(1);
-  if (checkExists) {
-FileSystem fs = FSUtils.getFs(s3Prefix + bucket, 
sparkSession.sparkContext().hadoopConfiguration());
-try {
-  if (fs.exists(new Path(filePath))) {
-cloudFiles.add(filePath);
-  }
-} catch (IOException e) {
-  LOG.error(String.format("Error while checking path exists for %s ", 
filePath), e);
-}
-  } else {
-cloudFiles.add(filePath);
-  }
-}
+.rdd().toJavaRDD().mapPartitions(fileListIterator -> {
+  List cloudFilesPerPartition = new ArrayList<>();
+  fileListIterator.forEachRemaining(row -> {
+final Configuration configuration = 
serializableConfiguration.newCopy();

Review Comment:
   Configuration here is not used for strict read hence copy creation might be 
more safer.



-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] vamshigv commented on pull request #6176: [HUDI-4445] S3 Incremental source improvements

2022-07-27 Thread GitBox


vamshigv commented on PR #6176:
URL: https://github.com/apache/hudi/pull/6176#issuecomment-1197368170

   Priority is to land https://github.com/apache/hudi/pull/6228 ahead of this 
while this can make it to the next release.


-- 
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-unsubscr...@hudi.apache.org

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



[jira] [Updated] (HUDI-4488) Improve S3 File listing efficiency

2022-07-27 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HUDI-4488?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated HUDI-4488:
-
Labels: pull-request-available  (was: )

> Improve S3 File listing efficiency
> --
>
> Key: HUDI-4488
> URL: https://issues.apache.org/jira/browse/HUDI-4488
> Project: Apache Hudi
>  Issue Type: Sub-task
>Reporter: Vamshi Gudavarthi
>Priority: Major
>  Labels: pull-request-available
> Fix For: 0.12.0
>
>
> Improve S3 file listing my moving the check to executor from driver.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [hudi] vamshigv commented on pull request #6228: [HUDI-4488] Improve S3 file listing efficiency

2022-07-27 Thread GitBox


vamshigv commented on PR #6228:
URL: https://github.com/apache/hudi/pull/6228#issuecomment-1197367486

   This is a small change that can have great impact on performance. Hence we 
would like to close this and land this first while we iterate on 
https://github.com/apache/hudi/pull/6176 


-- 
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-unsubscr...@hudi.apache.org

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



[GitHub] [hudi] CTTY commented on a diff in pull request #5943: [HUDI-4186] Support Hudi with Spark 3.3.0

2022-07-27 Thread GitBox


CTTY commented on code in PR #5943:
URL: https://github.com/apache/hudi/pull/5943#discussion_r931580643


##
hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/parser/HoodieCommonSqlParser.scala:
##
@@ -57,6 +57,14 @@ class HoodieCommonSqlParser(session: SparkSession, delegate: 
ParserInterface)
 
   override def parseDataType(sqlText: String): DataType = 
delegate.parseDataType(sqlText)
 
+  /* SPARK-37266 Added parseQuery to ParserInterface in Spark 3.3.0. This is a 
patch to prevent
+   hackers from tampering text with persistent view, it won't be called in 
older Spark
+   Don't mark this as override for backward compatibility
+   Can't use sparkExtendedParser directly here due to the same reason */
+  def parseQuery(sqlText: String): LogicalPlan = parse(sqlText) { parser =>

Review Comment:
   I reused the code flow from `parsePlan` method under the same class here. 
Calling `parse` might not be needed here. good point



-- 
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-unsubscr...@hudi.apache.org

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



  1   2   3   >