@@ -19,13 +19,16 @@ package org.apache.hudi.functional
1919
2020import org .apache .hudi .DataSourceWriteOptions ._
2121import org .apache .hudi .common .model .HoodieTableType
22+ import org .apache .hudi .common .table .{HoodieTableMetaClient , TableSchemaResolver }
2223import org .apache .hudi .common .table .timeline .HoodieActiveTimeline
2324import org .apache .hudi .config .HoodieWriteConfig
2425import org .apache .hudi .keygen .{ComplexKeyGenerator , NonpartitionedKeyGenerator }
2526import org .apache .hudi .testutils .HoodieClientTestBase
2627import org .apache .hudi .{DataSourceReadOptions , DataSourceWriteOptions }
28+
2729import org .apache .spark .sql .{Row , SaveMode , SparkSession }
28- import org .junit .jupiter .api .Assertions .{assertEquals , assertTrue }
30+
31+ import org .junit .jupiter .api .Assertions .{assertEquals , assertNotNull , assertNull , assertTrue }
2932import org .junit .jupiter .api .{AfterEach , BeforeEach }
3033import org .junit .jupiter .params .ParameterizedTest
3134import org .junit .jupiter .params .provider .EnumSource
@@ -39,9 +42,8 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
3942 " hoodie.upsert.shuffle.parallelism" -> " 4" ,
4043 " hoodie.bulkinsert.shuffle.parallelism" -> " 2" ,
4144 " hoodie.delete.shuffle.parallelism" -> " 1" ,
42- DataSourceWriteOptions .RECORDKEY_FIELD .key -> " _row_key" ,
43- DataSourceWriteOptions .PARTITIONPATH_FIELD .key -> " partition" ,
44- DataSourceWriteOptions .PRECOMBINE_FIELD .key -> " timestamp" ,
45+ DataSourceWriteOptions .RECORDKEY_FIELD .key -> " id" ,
46+ DataSourceWriteOptions .PRECOMBINE_FIELD .key -> " version" ,
4547 HoodieWriteConfig .TBL_NAME .key -> " hoodie_test"
4648 )
4749
@@ -72,8 +74,6 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
7274 df1.write.format(" hudi" )
7375 .options(commonOpts)
7476 .option(DataSourceWriteOptions .TABLE_TYPE .key, tableType.name())
75- .option(RECORDKEY_FIELD .key, " id" )
76- .option(PRECOMBINE_FIELD .key, " version" )
7777 .option(PARTITIONPATH_FIELD .key, " " )
7878 .option(KEYGENERATOR_CLASS_NAME .key, classOf [NonpartitionedKeyGenerator ].getName)
7979 .mode(SaveMode .Overwrite )
@@ -86,8 +86,6 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
8686 df2.write.format(" hudi" )
8787 .options(commonOpts)
8888 .option(DataSourceWriteOptions .TABLE_TYPE .key, tableType.name())
89- .option(RECORDKEY_FIELD .key, " id" )
90- .option(PRECOMBINE_FIELD .key, " version" )
9189 .option(PARTITIONPATH_FIELD .key, " " )
9290 .option(KEYGENERATOR_CLASS_NAME .key, classOf [NonpartitionedKeyGenerator ].getName)
9391 .mode(SaveMode .Append )
@@ -100,8 +98,6 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
10098 df3.write.format(" hudi" )
10199 .options(commonOpts)
102100 .option(DataSourceWriteOptions .TABLE_TYPE .key, tableType.name())
103- .option(RECORDKEY_FIELD .key, " id" )
104- .option(PRECOMBINE_FIELD .key, " version" )
105101 .option(PARTITIONPATH_FIELD .key, " " )
106102 .option(KEYGENERATOR_CLASS_NAME .key, classOf [NonpartitionedKeyGenerator ].getName)
107103 .mode(SaveMode .Append )
@@ -228,4 +224,84 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
228224 val format = new SimpleDateFormat (" yyyy-MM-dd" )
229225 format.format(date)
230226 }
227+
228+ @ ParameterizedTest
229+ @ EnumSource (value = classOf [HoodieTableType ])
230+ def testTimeTravelQueryWithSchemaEvolution (tableType : HoodieTableType ): Unit = {
231+ initMetaClient(tableType)
232+ val _spark = spark
233+ import _spark .implicits ._
234+
235+ // First write
236+ val df1 = Seq ((1 , " a1" , 10 , 1000 )).toDF(" id" , " name" , " value" , " version" )
237+ df1.write.format(" hudi" )
238+ .options(commonOpts)
239+ .option(DataSourceWriteOptions .TABLE_TYPE .key, tableType.name())
240+ .option(PARTITIONPATH_FIELD .key, " " )
241+ .mode(SaveMode .Overwrite )
242+ .save(basePath)
243+
244+ metaClient = HoodieTableMetaClient .builder()
245+ .setBasePath(basePath)
246+ .setConf(spark.sessionState.newHadoopConf)
247+ .build()
248+ val firstCommit = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().getTimestamp
249+
250+ // Second write
251+ val df2 = Seq ((1 , " a1" , 12 , 1001 , " 2022" )).toDF(" id" , " name" , " value" , " version" , " year" )
252+ df2.write.format(" hudi" )
253+ .options(commonOpts)
254+ .option(DataSourceWriteOptions .TABLE_TYPE .key, tableType.name())
255+ .option(PARTITIONPATH_FIELD .key, " " )
256+ .mode(SaveMode .Append )
257+ .save(basePath)
258+ metaClient.reloadActiveTimeline()
259+ val secondCommit = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().getTimestamp
260+
261+ // Third write
262+ val df3 = Seq ((1 , " a1" , 13 , 1002 , " 2022" , " 08" )).toDF(" id" , " name" , " value" , " version" , " year" , " month" )
263+ df3.write.format(" hudi" )
264+ .options(commonOpts)
265+ .option(DataSourceWriteOptions .TABLE_TYPE .key, tableType.name())
266+ .option(PARTITIONPATH_FIELD .key, " " )
267+ .mode(SaveMode .Append )
268+ .save(basePath)
269+ metaClient.reloadActiveTimeline()
270+ val thirdCommit = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().getTimestamp
271+
272+ val tableSchemaResolver = new TableSchemaResolver (metaClient)
273+
274+ // Query as of firstCommitTime
275+ val result1 = spark.read.format(" hudi" )
276+ .option(DataSourceReadOptions .TIME_TRAVEL_AS_OF_INSTANT .key, firstCommit)
277+ .load(basePath)
278+ .select(" id" , " name" , " value" , " version" )
279+ .take(1 )(0 )
280+ assertEquals(Row (1 , " a1" , 10 , 1000 ), result1)
281+ val schema1 = tableSchemaResolver.getTableAvroSchema(firstCommit)
282+ assertNull(schema1.getField(" year" ))
283+ assertNull(schema1.getField(" month" ))
284+
285+ // Query as of secondCommitTime
286+ val result2 = spark.read.format(" hudi" )
287+ .option(DataSourceReadOptions .TIME_TRAVEL_AS_OF_INSTANT .key, secondCommit)
288+ .load(basePath)
289+ .select(" id" , " name" , " value" , " version" , " year" )
290+ .take(1 )(0 )
291+ assertEquals(Row (1 , " a1" , 12 , 1001 , " 2022" ), result2)
292+ val schema2 = tableSchemaResolver.getTableAvroSchema(secondCommit)
293+ assertNotNull(schema2.getField(" year" ))
294+ assertNull(schema2.getField(" month" ))
295+
296+ // Query as of thirdCommitTime
297+ val result3 = spark.read.format(" hudi" )
298+ .option(DataSourceReadOptions .TIME_TRAVEL_AS_OF_INSTANT .key, thirdCommit)
299+ .load(basePath)
300+ .select(" id" , " name" , " value" , " version" , " year" , " month" )
301+ .take(1 )(0 )
302+ assertEquals(Row (1 , " a1" , 13 , 1002 , " 2022" , " 08" ), result3)
303+ val schema3 = tableSchemaResolver.getTableAvroSchema(thirdCommit)
304+ assertNotNull(schema3.getField(" year" ))
305+ assertNotNull(schema3.getField(" month" ))
306+ }
231307}
0 commit comments