yihua commented on code in PR #13882:
URL: https://github.com/apache/hudi/pull/13882#discussion_r2345444460
##########
hudi-client/hudi-spark-client/src/main/scala/org/apache/hudi/SparkAdapterSupport.scala:
##########
@@ -43,4 +43,6 @@ object SparkAdapterSupport {
getClass.getClassLoader.loadClass(adapterClass)
.newInstance().asInstanceOf[SparkAdapter]
}
+
+ def getSparkAdapter: SparkAdapter = sparkAdapter
Review Comment:
In Java code, this is used to get the spark adapter
`SparkAdapterSupport$.MODULE$.sparkAdapter()` in other classes
##########
hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/io/storage/row/HoodieRowParquetWriteSupport.java:
##########
@@ -18,47 +18,149 @@
package org.apache.hudi.io.storage.row;
+import org.apache.hudi.HoodieSparkUtils;
+import org.apache.hudi.SparkAdapterSupport;
+import org.apache.hudi.avro.HoodieAvroUtils;
import org.apache.hudi.avro.HoodieBloomFilterWriteSupport;
import org.apache.hudi.common.bloom.BloomFilter;
import org.apache.hudi.common.config.HoodieConfig;
import org.apache.hudi.common.config.HoodieStorageConfig;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
+import org.apache.hudi.config.HoodieWriteConfig;
+import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter;
+import org.apache.hudi.internal.schema.utils.SerDeHelper;
+import org.apache.avro.LogicalType;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
import org.apache.hadoop.conf.Configuration;
+import org.apache.parquet.avro.AvroSchemaConverter;
+import org.apache.parquet.avro.AvroWriteSupport;
import org.apache.parquet.hadoop.api.WriteSupport;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.io.api.RecordConsumer;
+import org.apache.parquet.schema.MessageType;
+import org.apache.spark.sql.HoodieInternalRowUtils;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.expressions.SpecializedGetters;
+import org.apache.spark.sql.catalyst.util.ArrayData;
+import org.apache.spark.sql.catalyst.util.DateTimeUtils;
+import org.apache.spark.sql.catalyst.util.MapData;
+import org.apache.spark.sql.execution.datasources.DataSourceUtils;
import org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport;
+import org.apache.spark.sql.internal.SQLConf;
+import org.apache.spark.sql.types.ArrayType;
+import org.apache.spark.sql.types.DataType;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Decimal;
+import org.apache.spark.sql.types.DecimalType;
+import org.apache.spark.sql.types.MapType;
import org.apache.spark.sql.types.StructType;
+import org.apache.spark.sql.types.TimestampNTZType;
import org.apache.spark.unsafe.types.UTF8String;
+import org.apache.spark.util.VersionUtils;
import java.util.Collections;
+import java.util.HashMap;
import java.util.Map;
+import java.util.stream.IntStream;
+import scala.Enumeration;
+import scala.Function1;
+
+import static org.apache.hudi.avro.AvroSchemaUtils.resolveNullableSchema;
import static
org.apache.hudi.common.config.HoodieStorageConfig.PARQUET_FIELD_ID_WRITE_ENABLED;
/**
- * Hoodie Write Support for directly writing Row to Parquet.
+ * Hoodie Write Support for directly writing Row to Parquet and adding the
Hudi bloom index to the file metadata.
+ * The implementation is adapted from Spark's {@link
org.apache.spark.sql.execution.datasources.parquet.ParquetWriteSupport} but has
the following major differences:
+ * <ul>
+ * <li>Decimals are always written with the legacy format to ensure
compatibility with parquet-avro and other readers</li>
+ * <li>Writing 2-level or 3-level lists is configurable with either the Hudi
option, hoodie.parquet.writelegacyformat.enabled, or the parquet-avro option,
parquet.avro.write-old-list-structure
+ * to ensure consistency across writer paths.</li>
+ * <li>The scale of the timestamps is determined by the Hudi writer schema
instead of relying on Spark configuration</li>
+ * </ul>
*/
-public class HoodieRowParquetWriteSupport extends ParquetWriteSupport {
+public class HoodieRowParquetWriteSupport extends WriteSupport<InternalRow> {
+ private static final Schema MAP_KEY_SCHEMA =
Schema.create(Schema.Type.STRING);
+ private static final String MAP_REPEATED_NAME = "key_value";
+ private static final String MAP_KEY_NAME = "key";
+ private static final String MAP_VALUE_NAME = "value";
private final Configuration hadoopConf;
private final Option<HoodieBloomFilterWriteSupport<UTF8String>>
bloomFilterWriteSupportOpt;
+ private final byte[] decimalBuffer = new
byte[Decimal.minBytesForPrecision()[DecimalType.MAX_PRECISION()]];
+ private final Enumeration.Value datetimeRebaseMode = (Enumeration.Value)
SparkAdapterSupport.getSparkAdapter().getDateTimeRebaseMode();
Review Comment:
nit: use `SparkAdapterSupport$.MODULE$.sparkAdapter()` so there is no need
to add a new API in `SparkAdapterSupport`, to be consistent with the usage in
other Java classes
##########
hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/adapter/Spark3_5Adapter.scala:
##########
@@ -38,7 +38,7 @@ import
org.apache.spark.sql.execution.datasources.orc.Spark35OrcReader
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat,
Spark35LegacyHoodieParquetFileFormat, Spark35ParquetReader}
import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
import org.apache.spark.sql.hudi.analysis.TableValuedFunctions
-import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf}
Review Comment:
So the class is moved in Spark thus the adapter pattern.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]