|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.hudi |
| 20 | + |
| 21 | +import org.apache.hadoop.conf.Configuration |
| 22 | +import org.apache.hudi.TestParquetReaderCompatibility.NullabilityEnum.{NotNullable, Nullability, Nullable} |
| 23 | +import org.apache.hudi.TestParquetReaderCompatibility.{SparkSetting, TestScenario, ThreeLevel, TwoLevel} |
| 24 | +import org.apache.hudi.client.common.HoodieSparkEngineContext |
| 25 | +import org.apache.hudi.common.config.HoodieMetadataConfig |
| 26 | +import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType |
| 27 | +import org.apache.hudi.common.table.ParquetTableSchemaResolver |
| 28 | +import org.apache.hudi.common.testutils.HoodieTestUtils |
| 29 | +import org.apache.hudi.config.HoodieWriteConfig |
| 30 | +import org.apache.hudi.io.storage.HoodieIOFactory |
| 31 | +import org.apache.hudi.metadata.HoodieBackedTableMetadata |
| 32 | +import org.apache.hudi.storage.StoragePath |
| 33 | +import org.apache.hudi.testutils.HoodieClientTestUtils |
| 34 | +import org.apache.spark.sql.{Row, SaveMode, SparkSession} |
| 35 | +import org.apache.spark.sql.types.{ArrayType, LongType, StringType, StructField, StructType} |
| 36 | +import org.apache.hudi.common.util.ConfigUtils.DEFAULT_HUDI_CONFIG_FOR_READER |
| 37 | +import org.apache.hudi.storage.hadoop.{HadoopStorageConfiguration, HoodieHadoopStorage} |
| 38 | +import org.apache.parquet.schema.OriginalType |
| 39 | +import org.apache.spark.SparkConf |
| 40 | +import org.junit.jupiter.params.ParameterizedTest |
| 41 | +import org.junit.jupiter.params.provider.MethodSource |
| 42 | + |
| 43 | +import java.util.Collections |
| 44 | +import scala.collection.mutable |
| 45 | +import scala.collection.JavaConverters._ |
| 46 | + |
| 47 | +object TestParquetReaderCompatibility { |
| 48 | + val listFieldName = "internal_list" |
| 49 | + abstract class SparkSetting { |
| 50 | + def value: String |
| 51 | + def overrideConf(conf: SparkConf): Unit |
| 52 | + } |
| 53 | + |
| 54 | + // Explicitly set 2 level |
| 55 | + case object TwoLevel extends SparkSetting { |
| 56 | + val value: String = "TwoLevel" |
| 57 | + def overrideConf(conf: SparkConf): Unit = { |
| 58 | + conf.set("spark.hadoop.parquet.avro.write-old-list-structure", true.toString) |
| 59 | + } |
| 60 | + } |
| 61 | + |
| 62 | + // Explicitly set 3 level |
| 63 | + case object ThreeLevel extends SparkSetting { |
| 64 | + val value: String = "ThreeLevel" |
| 65 | + |
| 66 | + def overrideConf(conf: SparkConf): Unit = { |
| 67 | + conf.set("spark.hadoop.parquet.avro.write-old-list-structure", false.toString) |
| 68 | + } |
| 69 | + } |
| 70 | + |
| 71 | + // Set nothing(likely most users do so) - default is 2 level inside Avro code. |
| 72 | + case object Default extends SparkSetting { |
| 73 | + def value: String = "TwoLevel" |
| 74 | + |
| 75 | + def overrideConf(conf: SparkConf): Unit = {} |
| 76 | + } |
| 77 | + |
| 78 | + val cases: Seq[SparkSetting] = Seq(TwoLevel, ThreeLevel, Default) |
| 79 | + |
| 80 | + object NullabilityEnum extends Enumeration { |
| 81 | + type Nullability = Value |
| 82 | + val Nullable: NullabilityEnum.Value = Value("Nullable") |
| 83 | + val NotNullable: NullabilityEnum.Value = Value("NotNullable") |
| 84 | + } |
| 85 | + |
| 86 | + case class TestScenario(initialLevel: SparkSetting, listNullability: NullabilityEnum.Nullability, targetLevel: SparkSetting, itemsNullability: NullabilityEnum.Nullability) |
| 87 | + |
| 88 | + /** |
| 89 | + * Here we are generating all possible combinations of settings, including default. |
| 90 | + **/ |
| 91 | + def allPossibleCombinations: java.util.stream.Stream[TestScenario] = { |
| 92 | + val allPossibleCombinations = for ( |
| 93 | + initialLevel <- cases; |
| 94 | + listNullability <- NullabilityEnum.values.toSeq; |
| 95 | + targetLevel <- cases; |
| 96 | + itemsNullability <- NullabilityEnum.values.toSeq |
| 97 | + ) yield TestScenario(initialLevel, listNullability, targetLevel, itemsNullability) |
| 98 | + allPossibleCombinations.filter { |
| 99 | + case c => { |
| 100 | + val notAllowedNulls = Seq(TwoLevel, Default) |
| 101 | + // It's not allowed to have NULLs inside lists for 2 level lists(this matches explicit setting or default). |
| 102 | + !(c.itemsNullability == Nullable && (notAllowedNulls.contains(c.targetLevel) || notAllowedNulls.contains(c.initialLevel))) |
| 103 | + } |
| 104 | + }.asJava.stream() |
| 105 | + } |
| 106 | + |
| 107 | + def selectedCombinations: java.util.stream.Stream[TestScenario] = { |
| 108 | + Seq( |
| 109 | + // This scenario leads to silent dataloss mentioned here - https://github.com/apache/hudi/pull/11450 - basically all arrays |
| 110 | + // which are not updated in the incoming batch are set to null. |
| 111 | + TestScenario(initialLevel = TwoLevel, listNullability = Nullable, targetLevel = ThreeLevel, itemsNullability = NotNullable), |
| 112 | + // This scenario leads to exception mentioned here https://github.com/apache/hudi/pull/11450 - the only difference with silent dataloss |
| 113 | + // is that writer does not allow wrongly-read null to be written into new file, so write fails. |
| 114 | + TestScenario(initialLevel = TwoLevel, listNullability = NotNullable, targetLevel = ThreeLevel, itemsNullability = NotNullable), |
| 115 | + // This is reverse version of scenario TwoLevel -> ThreeLevel with nullable list value - leads to silent data loss. |
| 116 | + TestScenario(initialLevel = ThreeLevel, listNullability = Nullable, targetLevel = TwoLevel, itemsNullability = NotNullable), |
| 117 | + // This is reverse version of scenario TwoLevel -> ThreeLevel with not nullable list value - leads to exception. |
| 118 | + TestScenario(initialLevel = ThreeLevel, listNullability = NotNullable, targetLevel = TwoLevel, itemsNullability = NotNullable) |
| 119 | + ).asJava.stream() |
| 120 | + } |
| 121 | + def testSource: java.util.stream.Stream[TestScenario] = if(runAllPossible) { |
| 122 | + allPossibleCombinations |
| 123 | + } else { |
| 124 | + selectedCombinations |
| 125 | + } |
| 126 | + |
| 127 | + /** |
| 128 | + * Change the value to run on highlighted ones. |
| 129 | + **/ |
| 130 | + def runAllPossible = true |
| 131 | +} |
| 132 | + |
| 133 | +/** |
| 134 | + * Ensure after switch from reading file with schema with which file was written to deduced schema(RFC 46) |
| 135 | + * different list levels can interoperate. |
| 136 | + **/ |
| 137 | +class TestParquetReaderCompatibility extends HoodieSparkWriterTestBase { |
| 138 | + /* |
| 139 | + * Generate schema with required nullability constraints. |
| 140 | + * The interesting part is that if list is the last element in the schema - different errors will be thrown. |
| 141 | + **/ |
| 142 | + private def getSchemaWithParameters(listNullability: Nullability, listElementNullability: Nullability): StructType = { |
| 143 | + val listNullable = listNullability == Nullable |
| 144 | + val listElementsNullable = listElementNullability == Nullable |
| 145 | + val schema = StructType(Array( |
| 146 | + StructField("key", LongType, nullable = false), |
| 147 | + StructField("partition", StringType, nullable = false), |
| 148 | + StructField(TestParquetReaderCompatibility.listFieldName, ArrayType(LongType, listElementsNullable), listNullable), |
| 149 | + StructField("ts", LongType, nullable = false) |
| 150 | + )) |
| 151 | + schema |
| 152 | + } |
| 153 | + private def defaultPartition = "p1" |
| 154 | + |
| 155 | + private def generateRowsWithParameters(listNullability: Nullability, listElementNullability: Nullability, combineValue: Long = 1L, dummyCount: Int = 10): Map[Long, Row] = { |
| 156 | + val listNullable = listNullability == Nullable |
| 157 | + val listElementsNullable = listElementNullability == Nullable |
| 158 | + val res = mutable.Map[Long, Row]() |
| 159 | + var key = 1L |
| 160 | + for (_ <- 1 to dummyCount) { |
| 161 | + res += key -> Row(key, defaultPartition, Seq(100L), combineValue) |
| 162 | + key += 1 |
| 163 | + } |
| 164 | + res += key -> Row(key, defaultPartition, Seq(1L, 2L), combineValue) |
| 165 | + key += 1 |
| 166 | + if (listNullable) { |
| 167 | + res += key -> Row(key, defaultPartition, null, combineValue) |
| 168 | + key += 1 |
| 169 | + } |
| 170 | + if (listElementsNullable) { |
| 171 | + res += key -> Row(key, defaultPartition, Seq(1L, null), combineValue) |
| 172 | + key += 1 |
| 173 | + } |
| 174 | + res.toMap |
| 175 | + } |
| 176 | + |
| 177 | + private def createSparkSessionWithListLevel(listType: SparkSetting): SparkSession = { |
| 178 | + val conf = new SparkConf() |
| 179 | + listType.overrideConf(conf) |
| 180 | + val spark = SparkSession.builder() |
| 181 | + .config(HoodieClientTestUtils.getSparkConfForTest("hoodie_test")) |
| 182 | + .config(conf) |
| 183 | + .getOrCreate() |
| 184 | + spark |
| 185 | + } |
| 186 | + |
| 187 | + /** |
| 188 | + * Test interoperability of different parquet list types and their nullability. |
| 189 | + **/ |
| 190 | + @ParameterizedTest |
| 191 | + @MethodSource(Array("testSource")) |
| 192 | + def testAvroListUpdate(input: TestScenario): Unit = { |
| 193 | + spark.stop() |
| 194 | + val path = tempBasePath + "_avro_list_update" |
| 195 | + val options = Map( |
| 196 | + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "key", |
| 197 | + DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts", |
| 198 | + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition", |
| 199 | + HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName, |
| 200 | + "path" -> path |
| 201 | + ) |
| 202 | + val initialLevel = input.initialLevel |
| 203 | + val listNullability = input.listNullability |
| 204 | + val targetLevel = input.targetLevel |
| 205 | + val itemsNullability = input.itemsNullability |
| 206 | + val structType = getSchemaWithParameters(listNullability, itemsNullability) |
| 207 | + val initialRecords = generateRowsWithParameters(listNullability, itemsNullability) |
| 208 | + |
| 209 | + val firstWriteSession = createSparkSessionWithListLevel(initialLevel) |
| 210 | + try { |
| 211 | + HoodieSparkSqlWriter.write( |
| 212 | + firstWriteSession.sqlContext, |
| 213 | + SaveMode.Overwrite, |
| 214 | + options, |
| 215 | + firstWriteSession.createDataFrame(firstWriteSession.sparkContext.parallelize(initialRecords.values.toSeq), structType) |
| 216 | + ) |
| 217 | + |
| 218 | + val firstWriteLevels = getListLevelsFromPath(firstWriteSession, path) |
| 219 | + assert(firstWriteLevels.size == 1, s"Expected only one level, got $firstWriteLevels") |
| 220 | + assert(firstWriteLevels.head == initialLevel.value, s"Expected level $initialLevel, got $firstWriteLevels") |
| 221 | + } finally { |
| 222 | + firstWriteSession.close() |
| 223 | + } |
| 224 | + |
| 225 | + val updateRecords = generateRowsWithParameters(listNullability, itemsNullability, 2L, 1) |
| 226 | + val secondWriteSession = createSparkSessionWithListLevel(targetLevel) |
| 227 | + var expectedRecordsWithSchema: Seq[Row] = Seq() |
| 228 | + try { |
| 229 | + HoodieSparkSqlWriter.write( |
| 230 | + secondWriteSession.sqlContext, |
| 231 | + SaveMode.Append, |
| 232 | + options, |
| 233 | + secondWriteSession.createDataFrame(secondWriteSession.sparkContext.parallelize(updateRecords.values.toSeq), structType) |
| 234 | + ) |
| 235 | + val secondWriteLevels = getListLevelsFromPath(secondWriteSession, path) |
| 236 | + assert(secondWriteLevels.size == 1, s"Expected only one level, got $secondWriteLevels") |
| 237 | + assert(secondWriteLevels.head == targetLevel.value, s"Expected level $targetLevel, got $secondWriteLevels") |
| 238 | + |
| 239 | + val expectedRecords = (initialRecords ++ updateRecords).values.toSeq |
| 240 | + expectedRecordsWithSchema = dropMetaFields( |
| 241 | + secondWriteSession.createDataFrame(secondWriteSession.sparkContext.parallelize(expectedRecords), structType) |
| 242 | + ).collect().toSeq |
| 243 | + } finally { |
| 244 | + secondWriteSession.close() |
| 245 | + } |
| 246 | + |
| 247 | + val readSessionWithInitLevel = createSparkSessionWithListLevel(initialLevel) |
| 248 | + try { |
| 249 | + compareResults(expectedRecordsWithSchema, readSessionWithInitLevel, path) |
| 250 | + } finally { |
| 251 | + readSessionWithInitLevel.close() |
| 252 | + } |
| 253 | + |
| 254 | + val readSessionWithTargetLevel = createSparkSessionWithListLevel(targetLevel) |
| 255 | + try { |
| 256 | + compareResults(expectedRecordsWithSchema, readSessionWithTargetLevel, path) |
| 257 | + } finally { |
| 258 | + readSessionWithTargetLevel.close() |
| 259 | + } |
| 260 | + |
| 261 | + initSparkContext() |
| 262 | + } |
| 263 | + |
| 264 | + /** |
| 265 | + * For some reason order of fields is different, |
| 266 | + * so produces difference like |
| 267 | + * Difference: Expected [2,p1,WrappedArray(1, 2),2], got [2,WrappedArray(1, 2),2,p1] |
| 268 | + * Difference: Expected [3,p1,WrappedArray(1, null),2], got [3,WrappedArray(1, null),2,p1] |
| 269 | + * So using manual comparison by ensuring length is the same, then extracting fields by names and comparing them. |
| 270 | + * This will not work for nested structs, but it's a simple test. |
| 271 | + */ |
| 272 | + def compareIndividualRows(first: Row, second: Row): Boolean = { |
| 273 | + if (first.length != second.length) { |
| 274 | + false |
| 275 | + } else { |
| 276 | + first.schema.fieldNames.forall { field => |
| 277 | + val firstIndex = first.fieldIndex(field) |
| 278 | + val secondIndex = second.fieldIndex(field) |
| 279 | + first.get(firstIndex) == second.get(secondIndex) |
| 280 | + } |
| 281 | + } |
| 282 | + } |
| 283 | + |
| 284 | + private def compareResults(expectedRecords: Seq[Row], sparkSession: SparkSession, path: String): Unit = { |
| 285 | + implicit object RowOrdering extends Ordering[Row] { |
| 286 | + def compare(a: Row, b: Row): Int = { |
| 287 | + val firstId = a.getLong(a.fieldIndex("key")) |
| 288 | + val secondId = b.getLong(b.fieldIndex("key")) |
| 289 | + firstId.compareTo(secondId) |
| 290 | + } |
| 291 | + } |
| 292 | + val expectedSorted = expectedRecords.sorted |
| 293 | + val readRecords = dropMetaFields(sparkSession.read.format("hudi").load(path)).collect().toSeq.sorted |
| 294 | + assert(readRecords.length == expectedSorted.length, s"Expected ${expectedSorted.length} records, got ${readRecords.length}") |
| 295 | + val recordsEqual = readRecords.zip(expectedSorted).forall { |
| 296 | + case (first, second) => compareIndividualRows(first, second) |
| 297 | + } |
| 298 | + val explanationStr = if (!recordsEqual) { |
| 299 | + readRecords.zipWithIndex.map { |
| 300 | + case (row, index) => { |
| 301 | + val expectedRow = expectedSorted(index) |
| 302 | + if (row != expectedRow) { |
| 303 | + s"Difference: Expected $expectedRow, got $row" |
| 304 | + } else { |
| 305 | + s"Equals: expected $expectedRow, got $row" |
| 306 | + } |
| 307 | + } |
| 308 | + }.mkString("\n") |
| 309 | + } else { |
| 310 | + "" |
| 311 | + } |
| 312 | + assert(recordsEqual, explanationStr) |
| 313 | + } |
| 314 | + |
| 315 | + private def getListLevelsFromPath(spark: SparkSession, path: String): Set[String] = { |
| 316 | + val engineContext = new HoodieSparkEngineContext(spark.sparkContext, spark.sqlContext) |
| 317 | + val metadataConfig = HoodieMetadataConfig.newBuilder().enable(true).build() |
| 318 | + val baseTableMetadata = new HoodieBackedTableMetadata( |
| 319 | + engineContext, HoodieTestUtils.getDefaultStorage, metadataConfig, s"$path", false) |
| 320 | + val fileStatuses = baseTableMetadata.getAllFilesInPartitions(Collections.singletonList(s"$path/$defaultPartition")) |
| 321 | + |
| 322 | + fileStatuses.asScala.flatMap(_._2.asScala).map(_.getPath).map(path => getListType(spark.sparkContext.hadoopConfiguration, path)).toSet |
| 323 | + } |
| 324 | + |
| 325 | + private def getListType(hadoopConf: Configuration, path: StoragePath): String = { |
| 326 | + val reader = HoodieIOFactory.getIOFactory(new HoodieHadoopStorage(path, new HadoopStorageConfiguration(hadoopConf))).getReaderFactory(HoodieRecordType.AVRO).getFileReader(DEFAULT_HUDI_CONFIG_FOR_READER, path) |
| 327 | + val schema = ParquetTableSchemaResolver.convertAvroSchemaToParquet(reader.getSchema, hadoopConf) |
| 328 | + |
| 329 | + val list = schema.getFields.asScala.find(_.getName == TestParquetReaderCompatibility.listFieldName).get |
| 330 | + val groupType = list.asGroupType() |
| 331 | + val originalType = groupType.getOriginalType |
| 332 | + val isThreeLevel = originalType == OriginalType.LIST && !(groupType.getType(0).getName == "array") |
| 333 | + |
| 334 | + if (isThreeLevel) { |
| 335 | + ThreeLevel.value |
| 336 | + } else { |
| 337 | + TwoLevel.value |
| 338 | + } |
| 339 | + } |
| 340 | + |
| 341 | +} |
0 commit comments