Skip to content

Commit

Permalink
Refactor test to use other primitives
Browse files Browse the repository at this point in the history
  • Loading branch information
VitoMakarevich committed Jun 17, 2024
1 parent 93c6d3d commit 6d56ddc
Showing 1 changed file with 9 additions and 12 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,22 +19,19 @@
package org.apache.hudi

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.TestParquetReaderCompatibility.NullabilityEnum.{NotNullable, Nullability, Nullable}
import org.apache.hudi.TestParquetReaderCompatibility.{SparkSetting, TestScenario, ThreeLevel, TwoLevel}
import org.apache.hudi.client.common.HoodieSparkEngineContext
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType
import org.apache.hudi.common.table.ParquetTableSchemaResolver
import org.apache.hudi.common.testutils.HoodieTestUtils
import org.apache.hudi.common.table.TableSchemaResolver
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.io.storage.HoodieIOFactory
import org.apache.hudi.io.storage.HoodieFileReaderFactory
import org.apache.hudi.metadata.HoodieBackedTableMetadata
import org.apache.hudi.storage.StoragePath
import org.apache.hudi.testutils.HoodieClientTestUtils
import org.apache.spark.sql.{Row, SaveMode, SparkSession}
import org.apache.spark.sql.types.{ArrayType, LongType, StringType, StructField, StructType}
import org.apache.hudi.common.util.ConfigUtils.DEFAULT_HUDI_CONFIG_FOR_READER
import org.apache.hudi.storage.hadoop.{HadoopStorageConfiguration, HoodieHadoopStorage}
import org.apache.parquet.schema.OriginalType
import org.apache.spark.SparkConf
import org.junit.jupiter.params.ParameterizedTest
Expand Down Expand Up @@ -134,7 +131,7 @@ object TestParquetReaderCompatibility {
* Ensure after switch from reading file with schema with which file was written to deduced schema(RFC 46)
* different list levels can interoperate.
**/
class TestParquetReaderCompatibility extends HoodieSparkWriterTestBase {
class TestParquetReaderCompatibility extends TestHoodieSparkSqlWriter {
/*
* Generate schema with required nullability constraints.
* The interesting part is that if list is the last element in the schema - different errors will be thrown.
Expand Down Expand Up @@ -316,15 +313,15 @@ class TestParquetReaderCompatibility extends HoodieSparkWriterTestBase {
val engineContext = new HoodieSparkEngineContext(spark.sparkContext, spark.sqlContext)
val metadataConfig = HoodieMetadataConfig.newBuilder().enable(true).build()
val baseTableMetadata = new HoodieBackedTableMetadata(
engineContext, HoodieTestUtils.getDefaultStorage, metadataConfig, s"$path", false)
engineContext, metadataConfig, s"$path", false)
val fileStatuses = baseTableMetadata.getAllFilesInPartitions(Collections.singletonList(s"$path/$defaultPartition"))

fileStatuses.asScala.flatMap(_._2.asScala).map(_.getPath).map(path => getListType(spark.sparkContext.hadoopConfiguration, path)).toSet
fileStatuses.asScala.flatMap(_._2.toList).map(_.getPath).map(path => getListType(spark.sparkContext.hadoopConfiguration, path)).toSet
}

private def getListType(hadoopConf: Configuration, path: StoragePath): String = {
val reader = HoodieIOFactory.getIOFactory(new HoodieHadoopStorage(path, new HadoopStorageConfiguration(hadoopConf))).getReaderFactory(HoodieRecordType.AVRO).getFileReader(DEFAULT_HUDI_CONFIG_FOR_READER, path)
val schema = ParquetTableSchemaResolver.convertAvroSchemaToParquet(reader.getSchema, hadoopConf)
private def getListType(hadoopConf: Configuration, path: Path): String = {
val reader = HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO).getFileReader(hadoopConf, path)
val schema = TableSchemaResolver.convertAvroSchemaToParquet(reader.getSchema, hadoopConf)

val list = schema.getFields.asScala.find(_.getName == TestParquetReaderCompatibility.listFieldName).get
val groupType = list.asGroupType()
Expand Down

0 comments on commit 6d56ddc

Please sign in to comment.