@@ -115,55 +115,81 @@ public void validateRecordsInFileGroup(String tablePath,
115
115
116
116
private static Stream <Arguments > testArguments () {
117
117
return Stream .of (
118
- arguments (RecordMergeMode .COMMIT_TIME_ORDERING , "avro" ),
119
- arguments (RecordMergeMode .COMMIT_TIME_ORDERING , "parquet" ),
120
- arguments (RecordMergeMode .EVENT_TIME_ORDERING , "avro" ),
121
- arguments (RecordMergeMode .EVENT_TIME_ORDERING , "parquet" ),
122
- arguments (RecordMergeMode .CUSTOM , "avro" ),
123
- arguments (RecordMergeMode .CUSTOM , "parquet" )
118
+ arguments (RecordMergeMode .COMMIT_TIME_ORDERING , "avro" , false ),
119
+ arguments (RecordMergeMode .COMMIT_TIME_ORDERING , "parquet" , false ),
120
+ arguments (RecordMergeMode .EVENT_TIME_ORDERING , "avro" , false ),
121
+ arguments (RecordMergeMode .EVENT_TIME_ORDERING , "parquet" , false ),
122
+ arguments (RecordMergeMode .CUSTOM , "avro" , false ),
123
+ arguments (RecordMergeMode .CUSTOM , "parquet" , false ),
124
+ arguments (RecordMergeMode .COMMIT_TIME_ORDERING , "avro" , true ),
125
+ arguments (RecordMergeMode .COMMIT_TIME_ORDERING , "parquet" , true ),
126
+ arguments (RecordMergeMode .EVENT_TIME_ORDERING , "avro" , true ),
127
+ arguments (RecordMergeMode .EVENT_TIME_ORDERING , "parquet" , true ),
128
+ arguments (RecordMergeMode .CUSTOM , "avro" , true ),
129
+ arguments (RecordMergeMode .CUSTOM , "parquet" , true )
124
130
);
125
131
}
126
132
127
133
@ ParameterizedTest
128
134
@ MethodSource ("testArguments" )
129
- public void testReadFileGroupInMergeOnReadTable (RecordMergeMode recordMergeMode , String logDataBlockFormat ) throws Exception {
130
- Map <String , String > writeConfigs = new HashMap <>(getCommonConfigs (recordMergeMode ));
135
+ public void testReadFileGroupInMergeOnReadTable (RecordMergeMode recordMergeMode ,
136
+ String logDataBlockFormat ,
137
+ Boolean useNestedKey ) throws Exception {
138
+ Map <String , String > writeConfigs = new HashMap <>(getCommonConfigs (recordMergeMode , useNestedKey ));
131
139
writeConfigs .put (HoodieStorageConfig .LOGFILE_DATA_BLOCK_FORMAT .key (), logDataBlockFormat );
132
140
133
141
try (HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator (0xDEEF )) {
142
+ List <HoodieRecord > baseRecords = useNestedKey
143
+ ? dataGen .generateInsertsWithKeyFieldSpecified ("001" , "fare" , 100 )
144
+ : dataGen .generateInserts ("001" , 100 );
145
+
134
146
// One commit; reading one file group containing a base file only
135
- commitToTable (dataGen . generateInserts ( "001" , 100 ) , INSERT .value (), writeConfigs );
147
+ commitToTable (baseRecords , INSERT .value (), writeConfigs );
136
148
validateOutputFromFileGroupReader (
137
149
getStorageConf (), getBasePath (), dataGen .getPartitionPaths (), true , 0 , recordMergeMode );
138
150
139
151
// Two commits; reading one file group containing a base file and a log file
140
- commitToTable (dataGen .generateUpdates ("002" , 100 ), UPSERT .value (), writeConfigs );
152
+ List <HoodieRecord > firstUpdate = useNestedKey
153
+ ? dataGen .generateUpdates ("002" , baseRecords , "fare" )
154
+ : dataGen .generateUpdates ("002" , 100 );
155
+ commitToTable (firstUpdate , UPSERT .value (), writeConfigs );
141
156
validateOutputFromFileGroupReader (
142
157
getStorageConf (), getBasePath (), dataGen .getPartitionPaths (), true , 1 , recordMergeMode );
143
158
144
159
// Three commits; reading one file group containing a base file and two log files
145
- commitToTable (dataGen .generateUpdates ("003" , 100 ), UPSERT .value (), writeConfigs );
160
+ List <HoodieRecord > secondUpdates = useNestedKey
161
+ ? dataGen .generateUpdates ("003" , baseRecords , "fare" )
162
+ : dataGen .generateUpdates ("003" , 100 );
163
+ commitToTable (secondUpdates , UPSERT .value (), writeConfigs );
146
164
validateOutputFromFileGroupReader (
147
165
getStorageConf (), getBasePath (), dataGen .getPartitionPaths (), true , 2 , recordMergeMode );
148
166
}
149
167
}
150
168
151
169
@ ParameterizedTest
152
170
@ MethodSource ("testArguments" )
153
- public void testReadLogFilesOnlyInMergeOnReadTable (RecordMergeMode recordMergeMode , String logDataBlockFormat ) throws Exception {
154
- Map <String , String > writeConfigs = new HashMap <>(getCommonConfigs (recordMergeMode ));
171
+ public void testReadLogFilesOnlyInMergeOnReadTable (RecordMergeMode recordMergeMode ,
172
+ String logDataBlockFormat ,
173
+ Boolean useNestedKey ) throws Exception {
174
+ Map <String , String > writeConfigs = new HashMap <>(getCommonConfigs (recordMergeMode , useNestedKey ));
155
175
writeConfigs .put (HoodieStorageConfig .LOGFILE_DATA_BLOCK_FORMAT .key (), logDataBlockFormat );
156
176
// Use InMemoryIndex to generate log only mor table
157
177
writeConfigs .put ("hoodie.index.type" , "INMEMORY" );
158
178
159
179
try (HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator (0xDEEF )) {
160
- // One commit; reading one file group containing a base file only
161
- commitToTable (dataGen .generateInserts ("001" , 100 ), INSERT .value (), writeConfigs );
180
+ // One commit; reading one file group containing one log file
181
+ List <HoodieRecord > baseRecords = useNestedKey
182
+ ? dataGen .generateInsertsWithKeyFieldSpecified ("001" , "fare" , 100 )
183
+ : dataGen .generateInserts ("001" , 100 );
184
+ commitToTable (baseRecords , INSERT .value (), writeConfigs );
162
185
validateOutputFromFileGroupReader (
163
186
getStorageConf (), getBasePath (), dataGen .getPartitionPaths (), false , 1 , recordMergeMode );
164
187
165
- // Two commits; reading one file group containing a base file and a log file
166
- commitToTable (dataGen .generateUpdates ("002" , 100 ), UPSERT .value (), writeConfigs );
188
+ // Two commits; reading two log files
189
+ List <HoodieRecord > updateRecords = useNestedKey
190
+ ? dataGen .generateUpdates ("002" , baseRecords , "fare" )
191
+ : dataGen .generateUpdates ("002" , 100 );
192
+ commitToTable (updateRecords , UPSERT .value (), writeConfigs );
167
193
validateOutputFromFileGroupReader (
168
194
getStorageConf (), getBasePath (), dataGen .getPartitionPaths (), false , 2 , recordMergeMode );
169
195
}
@@ -172,7 +198,7 @@ public void testReadLogFilesOnlyInMergeOnReadTable(RecordMergeMode recordMergeMo
172
198
@ ParameterizedTest
173
199
@ EnumSource (value = ExternalSpillableMap .DiskMapType .class )
174
200
public void testSpillableMapUsage (ExternalSpillableMap .DiskMapType diskMapType ) throws Exception {
175
- Map <String , String > writeConfigs = new HashMap <>(getCommonConfigs (RecordMergeMode .COMMIT_TIME_ORDERING ));
201
+ Map <String , String > writeConfigs = new HashMap <>(getCommonConfigs (RecordMergeMode .COMMIT_TIME_ORDERING , false ));
176
202
Option <Schema .Type > orderingFieldType = Option .of (Schema .Type .STRING );
177
203
try (HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator (0xDEEF )) {
178
204
commitToTable (dataGen .generateInserts ("001" , 100 ), INSERT .value (), writeConfigs );
@@ -230,9 +256,11 @@ public void testSpillableMapUsage(ExternalSpillableMap.DiskMapType diskMapType)
230
256
}
231
257
}
232
258
233
- private Map <String , String > getCommonConfigs (RecordMergeMode recordMergeMode ) {
259
+ private Map <String , String > getCommonConfigs (RecordMergeMode recordMergeMode , Boolean useNestedKey ) {
234
260
Map <String , String > configMapping = new HashMap <>();
235
- configMapping .put (KeyGeneratorOptions .RECORDKEY_FIELD_NAME .key (), "_row_key" );
261
+ configMapping .put (
262
+ KeyGeneratorOptions .RECORDKEY_FIELD_NAME .key (),
263
+ useNestedKey ? "fare" : "_row_key" );
236
264
configMapping .put (KeyGeneratorOptions .PARTITIONPATH_FIELD_NAME .key (), "partition_path" );
237
265
configMapping .put ("hoodie.datasource.write.precombine.field" , "timestamp" );
238
266
configMapping .put ("hoodie.payload.ordering.field" , "timestamp" );
0 commit comments