yihua commented on code in PR #11192: URL: https://github.com/apache/hudi/pull/11192#discussion_r1597285676
########## hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/HoodieHadoopIOFactory.java: ########## @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.io.hadoop.HoodieAvroFileReaderFactory; +import org.apache.hudi.io.hadoop.HoodieAvroFileWriterFactory; + +public class HoodieHadoopIOFactory extends HoodieIOFactory { Review Comment: Javadocs on what is returned, and what the Avro vs Spark record type means. ########## hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieIOFactory.java: ########## @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hudi.common.config.HoodieStorageConfig; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.util.ReflectionUtils; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.storage.StorageConfiguration; + +/** + * Base class to get HoodieFileReaderFactory and HoodieFileWriterFactory + */ +public abstract class HoodieIOFactory { + + public static HoodieIOFactory getIOFactory(StorageConfiguration<?> storageConf) { + String ioFactoryClass = storageConf.getString(HoodieStorageConfig.HOODIE_IO_FACTORY_CLASS.key()) + .orElse(HoodieStorageConfig.HOODIE_IO_FACTORY_CLASS.defaultValue()); + return getIOFactory(ioFactoryClass); + } + + private static HoodieIOFactory getIOFactory(String ioFactoryClass) { + try { + Class<?> clazz = + ReflectionUtils.getClass(ioFactoryClass); + return (HoodieIOFactory) clazz.newInstance(); + } catch (IllegalArgumentException | IllegalAccessException | InstantiationException e) { + throw new HoodieException("Unable to create " + ioFactoryClass, e); + } + } + + public HoodieFileReaderFactory getReaderFactory(HoodieRecord.HoodieRecordType recordType) { Review Comment: I'm wondering for different record type, should they be using two different `HoodieIOFactory` implementation instead one implementation class redirecting to different reader/writer factories internally? ########## hudi-hadoop-common/src/main/java/org/apache/hudi/io/storage/HoodieHadoopIOFactory.java: ########## @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.io.storage; + +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.io.hadoop.HoodieAvroFileReaderFactory; +import org.apache.hudi.io.hadoop.HoodieAvroFileWriterFactory; + +public class HoodieHadoopIOFactory extends HoodieIOFactory { + + @Override + public HoodieFileReaderFactory getReaderFactory(HoodieRecord.HoodieRecordType recordType) { + if (recordType == HoodieRecord.HoodieRecordType.AVRO) { + return new HoodieAvroFileReaderFactory(); + } + return super.getReaderFactory(recordType); Review Comment: Is SPARK record type not supported? ########## hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala: ########## @@ -64,6 +65,9 @@ class DefaultSource extends RelationProvider // Enable "passPartitionByAsOptions" to support "write.partitionBy(...)" spark.conf.set("spark.sql.legacy.sources.write.passPartitionByAsOptions", "true") } + //always use spark io factory + spark.sparkContext.hadoopConfiguration.set(HoodieStorageConfig.HOODIE_IO_FACTORY_CLASS.key(), + classOf[HoodieSparkIOFactory].getName) Review Comment: Does this affect the writer path? or only the reader path? ########## hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DefaultSource.scala: ########## @@ -64,6 +65,9 @@ class DefaultSource extends RelationProvider // Enable "passPartitionByAsOptions" to support "write.partitionBy(...)" spark.conf.set("spark.sql.legacy.sources.write.passPartitionByAsOptions", "true") } + //always use spark io factory + spark.sparkContext.hadoopConfiguration.set(HoodieStorageConfig.HOODIE_IO_FACTORY_CLASS.key(), Review Comment: ```suggestion // Always use spark io factory spark.sparkContext.hadoopConfiguration.set(HoodieStorageConfig.HOODIE_IO_FACTORY_CLASS.key(), ``` -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: commits-unsubscr...@hudi.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org