20
20
package org .apache .hudi .hadoop ;
21
21
22
22
import org .apache .hudi .avro .HoodieAvroUtils ;
23
- import org .apache .hudi .client .HoodieJavaWriteClient ;
24
- import org .apache .hudi .client .common .HoodieJavaEngineContext ;
25
23
import org .apache .hudi .common .config .HoodieMemoryConfig ;
26
- import org .apache .hudi .common .config .HoodieReaderConfig ;
27
- import org .apache .hudi .common .config .RecordMergeMode ;
28
- import org .apache .hudi .common .engine .EngineType ;
29
24
import org .apache .hudi .common .engine .HoodieReaderContext ;
30
- import org .apache .hudi .common .model .HoodieAvroRecord ;
31
- import org .apache .hudi .common .model .HoodieRecord ;
32
- import org .apache .hudi .common .model .HoodieRecordPayload ;
33
25
import org .apache .hudi .common .table .HoodieTableMetaClient ;
34
- import org .apache .hudi .common .table .read .CustomPayloadForTesting ;
35
- import org .apache .hudi .common .table .read .TestHoodieFileGroupReaderBase ;
26
+ import org .apache .hudi .common .table .read .HoodieFileGroupReaderOnJavaTestBase ;
36
27
import org .apache .hudi .common .testutils .HoodieTestDataGenerator ;
37
28
import org .apache .hudi .common .testutils .HoodieTestUtils ;
38
29
import org .apache .hudi .common .testutils .minicluster .HdfsTestService ;
39
- import org .apache .hudi .config .HoodieWriteConfig ;
40
- import org .apache .hudi .hadoop .hive .HoodieCombineHiveInputFormat ;
41
- import org .apache .hudi .hadoop .realtime .HoodieParquetRealtimeInputFormat ;
42
- import org .apache .hudi .hadoop .realtime .HoodieRealtimeRecordReader ;
43
30
import org .apache .hudi .hadoop .utils .ObjectInspectorCache ;
44
31
import org .apache .hudi .storage .HoodieStorage ;
45
32
import org .apache .hudi .storage .StorageConfiguration ;
46
33
import org .apache .hudi .storage .hadoop .HoodieHadoopStorage ;
47
34
import org .apache .hudi .testutils .ArrayWritableTestUtil ;
48
- import org .apache .hudi .testutils .HoodieJavaClientTestHarness ;
49
35
50
36
import org .apache .avro .Schema ;
51
37
import org .apache .hadoop .conf .Configuration ;
52
38
import org .apache .hadoop .fs .FileSystem ;
53
- import org .apache .hadoop .fs .Path ;
54
39
import org .apache .hadoop .hive .metastore .api .hive_metastoreConstants ;
55
- import org .apache .hadoop .hive .ql .exec .Utilities ;
56
- import org .apache .hadoop .hive .ql .exec .mr .ExecMapper ;
57
40
import org .apache .hadoop .hive .ql .io .parquet .MapredParquetInputFormat ;
58
- import org .apache .hadoop .hive .ql .plan .MapredWork ;
59
- import org .apache .hadoop .hive .ql .plan .PartitionDesc ;
60
- import org .apache .hadoop .hive .ql .plan .TableDesc ;
61
41
import org .apache .hadoop .hive .serde2 .ColumnProjectionUtils ;
62
42
import org .apache .hadoop .io .ArrayWritable ;
63
- import org .apache .hadoop .io .NullWritable ;
64
- import org .apache .hadoop .mapred .FileInputFormat ;
65
- import org .apache .hadoop .mapred .InputSplit ;
66
43
import org .apache .hadoop .mapred .JobConf ;
67
- import org .apache .hadoop .mapred .RecordReader ;
68
- import org .apache .hadoop .mapred .Reporter ;
69
44
import org .junit .jupiter .api .AfterAll ;
70
45
import org .junit .jupiter .api .BeforeAll ;
71
46
72
47
import java .io .IOException ;
73
- import java .util .ArrayList ;
74
- import java .util .Arrays ;
75
- import java .util .HashMap ;
76
- import java .util .LinkedHashMap ;
77
48
import java .util .List ;
78
- import java .util .Map ;
79
49
import java .util .stream .Collectors ;
80
50
81
- import static org .apache .hadoop .hive .ql .exec .Utilities .HAS_MAP_WORK ;
82
- import static org .apache .hadoop .hive .ql .exec .Utilities .MAPRED_MAPPER_CLASS ;
83
51
import static org .apache .hudi .hadoop .HoodieFileGroupReaderBasedRecordReader .getRecordKeyField ;
84
52
import static org .apache .hudi .hadoop .HoodieFileGroupReaderBasedRecordReader .getStoredPartitionFieldNames ;
85
- import static org .junit .jupiter .api .Assertions .assertEquals ;
86
53
87
- public class TestHoodieFileGroupReaderOnHive extends TestHoodieFileGroupReaderBase <ArrayWritable > {
54
+ public class TestHoodieFileGroupReaderOnHive extends HoodieFileGroupReaderOnJavaTestBase <ArrayWritable > {
88
55
89
56
private static final String PARTITION_COLUMN = "datestr" ;
90
57
private static JobConf baseJobConf ;
@@ -97,7 +64,7 @@ public class TestHoodieFileGroupReaderOnHive extends TestHoodieFileGroupReaderBa
97
64
private static final boolean USE_FAKE_PARTITION = true ;
98
65
99
66
@ BeforeAll
100
- public static void setUpClass () throws IOException , InterruptedException {
67
+ public static void setUpClass () throws IOException {
101
68
// Append is not supported in LocalFileSystem. HDFS needs to be setup.
102
69
hdfsTestService = new HdfsTestService ();
103
70
fs = hdfsTestService .start (true ).getFileSystem ();
@@ -123,140 +90,20 @@ public StorageConfiguration<?> getStorageConf() {
123
90
}
124
91
125
92
@ Override
126
- public String getBasePath () {
127
- return tempDir .toAbsolutePath () + "/myTable" ;
128
- }
129
-
130
- @ Override
131
- public HoodieReaderContext <ArrayWritable > getHoodieReaderContext (String tablePath , Schema avroSchema , StorageConfiguration <?> storageConf ) {
93
+ public HoodieReaderContext <ArrayWritable > getHoodieReaderContext (String tablePath , Schema avroSchema , StorageConfiguration <?> storageConf , HoodieTableMetaClient metaClient ) {
132
94
HoodieFileGroupReaderBasedRecordReader .HiveReaderCreator readerCreator = (inputSplit , jobConf ) -> new MapredParquetInputFormat ().getRecordReader (inputSplit , jobConf , null );
133
- HoodieTableMetaClient metaClient = HoodieTableMetaClient .builder ().setConf (storageConf ).setBasePath (tablePath ).build ();
134
95
JobConf jobConf = new JobConf (storageConf .unwrapAs (Configuration .class ));
135
96
setupJobconf (jobConf );
136
97
return new HiveHoodieReaderContext (readerCreator , getRecordKeyField (metaClient ),
137
98
getStoredPartitionFieldNames (new JobConf (storageConf .unwrapAs (Configuration .class )), avroSchema ),
138
99
new ObjectInspectorCache (avroSchema , jobConf ), storageConf );
139
100
}
140
101
141
- @ Override
142
- public String getCustomPayload () {
143
- return CustomPayloadForTesting .class .getName ();
144
- }
145
-
146
- @ Override
147
- public void commitToTable (List <HoodieRecord > recordList , String operation , Map <String , String > writeConfigs ) {
148
- HoodieWriteConfig writeConfig = HoodieWriteConfig .newBuilder ()
149
- .withEngineType (EngineType .JAVA )
150
- .withEmbeddedTimelineServerEnabled (false )
151
- .withProps (writeConfigs )
152
- .withPath (getBasePath ())
153
- .withSchema (HoodieTestDataGenerator .TRIP_EXAMPLE_SCHEMA )
154
- .build ();
155
-
156
- HoodieJavaClientTestHarness .TestJavaTaskContextSupplier taskContextSupplier = new HoodieJavaClientTestHarness .TestJavaTaskContextSupplier ();
157
- HoodieJavaEngineContext context = new HoodieJavaEngineContext (getStorageConf (), taskContextSupplier );
158
- //init table if not exists
159
- Path basePath = new Path (getBasePath ());
160
- try {
161
- try (FileSystem lfs = basePath .getFileSystem (baseJobConf )) {
162
- boolean basepathExists = lfs .exists (basePath );
163
- boolean operationIsInsert = operation .equalsIgnoreCase ("insert" );
164
- if (!basepathExists || operationIsInsert ) {
165
- if (basepathExists ) {
166
- lfs .delete (new Path (getBasePath ()), true );
167
- }
168
- Map <String , Object > initConfigs = new HashMap <>(writeConfigs );
169
- HoodieTableMetaClient .TableBuilder builder = HoodieTableMetaClient .newTableBuilder ()
170
- .setTableType (writeConfigs .getOrDefault ("hoodie.datasource.write.table.type" , "MERGE_ON_READ" ))
171
- .setTableName (writeConfigs .get ("hoodie.table.name" ))
172
- .setPartitionFields (writeConfigs .getOrDefault ("hoodie.datasource.write.partitionpath.field" , "" ))
173
- .setRecordMergeMode (RecordMergeMode .getValue (writeConfigs .get ("hoodie.record.merge.mode" )))
174
- .set (initConfigs );
175
- if (writeConfigs .containsKey ("hoodie.datasource.write.payload.class" )) {
176
- builder = builder .setPayloadClassName (writeConfigs .get ("hoodie.datasource.write.payload.class" ));
177
- }
178
- builder .initTable (storageConf , getBasePath ());
179
- }
180
- }
181
- } catch (IOException e ) {
182
- throw new RuntimeException (e );
183
- }
184
-
185
- try (HoodieJavaWriteClient writeClient = new HoodieJavaWriteClient (context , writeConfig )) {
186
- String instantTime = writeClient .createNewInstantTime ();
187
- writeClient .startCommitWithTime (instantTime );
188
- // Make a copy of the records for writing. The writer will clear out the data field.
189
- List <HoodieRecord > recordsCopy = new ArrayList <>(recordList .size ());
190
- recordList .forEach (hoodieRecord -> recordsCopy .add (new HoodieAvroRecord <>(hoodieRecord .getKey (), (HoodieRecordPayload ) hoodieRecord .getData ())));
191
- if (operation .toLowerCase ().equals ("insert" )) {
192
- writeClient .insert (recordsCopy , instantTime );
193
- } else {
194
- writeClient .upsert (recordsCopy , instantTime );
195
- }
196
- }
197
- }
198
-
199
102
@ Override
200
103
public void assertRecordsEqual (Schema schema , ArrayWritable expected , ArrayWritable actual ) {
201
104
ArrayWritableTestUtil .assertArrayWritableEqual (schema , expected , actual , false );
202
105
}
203
106
204
- private static boolean isLogFileRec (HoodieReaderContext <ArrayWritable > readerContext , Schema schema , ArrayWritable record ) {
205
- return !readerContext .getValue (record , schema , HoodieRecord .FILENAME_METADATA_FIELD ).toString ().contains (".parquet" );
206
- }
207
-
208
- private static String createUniqueKey (HoodieReaderContext <ArrayWritable > readerContext , Schema schema , ArrayWritable record , boolean isSkipMerge ) {
209
- if (isSkipMerge ) {
210
- return readerContext .getRecordKey (record , schema ) + "_" + readerContext .getValue (record , schema , HoodieRecord .COMMIT_TIME_METADATA_FIELD ).toString ();
211
- } else {
212
- return readerContext .getRecordKey (record , schema );
213
- }
214
- }
215
-
216
- private RecordReader <NullWritable , ArrayWritable > createRecordReader (String tablePath , boolean isSkipMerge ) throws IOException {
217
- JobConf jobConf = new JobConf (baseJobConf );
218
- jobConf .set (HoodieReaderConfig .FILE_GROUP_READER_ENABLED .key (), "false" );
219
- jobConf .set (HoodieRealtimeRecordReader .REALTIME_SKIP_MERGE_PROP , String .valueOf (isSkipMerge ));
220
-
221
- TableDesc tblDesc = Utilities .defaultTd ;
222
- // Set the input format
223
- tblDesc .setInputFileFormatClass (HoodieParquetRealtimeInputFormat .class );
224
- LinkedHashMap <Path , PartitionDesc > pt = new LinkedHashMap <>();
225
- LinkedHashMap <Path , ArrayList <String >> talias = new LinkedHashMap <>();
226
-
227
- PartitionDesc partDesc = new PartitionDesc (tblDesc , null );
228
-
229
- pt .put (new Path (tablePath ), partDesc );
230
-
231
- ArrayList <String > arrayList = new ArrayList <>();
232
- arrayList .add (tablePath );
233
- talias .put (new Path (tablePath ), arrayList );
234
-
235
- MapredWork mrwork = new MapredWork ();
236
- mrwork .getMapWork ().setPathToPartitionInfo (pt );
237
- mrwork .getMapWork ().setPathToAliases (talias );
238
-
239
- Path mapWorkPath = new Path (tablePath );
240
- Utilities .setMapRedWork (jobConf , mrwork , mapWorkPath );
241
-
242
- // Add three partition path to InputPaths
243
- Path [] partitionDirArray = new Path [HoodieTestDataGenerator .DEFAULT_PARTITION_PATHS .length ];
244
- Arrays .stream (HoodieTestDataGenerator .DEFAULT_PARTITION_PATHS ).map (s -> new Path (tablePath , s )).collect (Collectors .toList ()).toArray (partitionDirArray );
245
- FileInputFormat .setInputPaths (jobConf , partitionDirArray );
246
- jobConf .set (HAS_MAP_WORK , "true" );
247
- // The following config tells Hive to choose ExecMapper to read the MAP_WORK
248
- jobConf .set (MAPRED_MAPPER_CLASS , ExecMapper .class .getName ());
249
- // setting the split size to be 3 to create one split for 3 file groups
250
- jobConf .set (org .apache .hadoop .mapreduce .lib .input .FileInputFormat .SPLIT_MAXSIZE , "128000000" );
251
- setupJobconf (jobConf );
252
-
253
- HoodieCombineHiveInputFormat combineHiveInputFormat = new HoodieCombineHiveInputFormat ();
254
- InputSplit [] splits = combineHiveInputFormat .getSplits (jobConf , 1 );
255
-
256
- assertEquals (1 , splits .length );
257
- return combineHiveInputFormat .getRecordReader (splits [0 ], jobConf , Reporter .NULL );
258
- }
259
-
260
107
private void setupJobconf (JobConf jobConf ) {
261
108
Schema schema = HoodieAvroUtils .addMetadataFields (HoodieTestDataGenerator .AVRO_SCHEMA );
262
109
List <Schema .Field > fields = schema .getFields ();
0 commit comments