18
18
package org .apache .seatunnel .connectors .seatunnel .file .source .reader ;
19
19
20
20
import org .apache .seatunnel .api .source .Collector ;
21
+ import org .apache .seatunnel .api .table .catalog .TablePath ;
21
22
import org .apache .seatunnel .api .table .type .ArrayType ;
22
23
import org .apache .seatunnel .api .table .type .BasicType ;
23
24
import org .apache .seatunnel .api .table .type .DecimalType ;
28
29
import org .apache .seatunnel .api .table .type .SeaTunnelRow ;
29
30
import org .apache .seatunnel .api .table .type .SeaTunnelRowType ;
30
31
import org .apache .seatunnel .api .table .type .SqlType ;
32
+ import org .apache .seatunnel .common .exception .CommonError ;
31
33
import org .apache .seatunnel .common .exception .CommonErrorCodeDeprecated ;
32
34
import org .apache .seatunnel .connectors .seatunnel .file .exception .FileConnectorErrorCode ;
33
35
import org .apache .seatunnel .connectors .seatunnel .file .exception .FileConnectorException ;
67
69
import java .util .List ;
68
70
import java .util .Map ;
69
71
import java .util .concurrent .TimeUnit ;
72
+ import java .util .stream .IntStream ;
70
73
71
74
@ Slf4j
72
75
public class ParquetReadStrategy extends AbstractReadStrategy {
@@ -75,6 +78,7 @@ public class ParquetReadStrategy extends AbstractReadStrategy {
75
78
private static final long NANOS_PER_MILLISECOND = 1000000 ;
76
79
private static final long MILLIS_PER_DAY = TimeUnit .DAYS .toMillis (1L );
77
80
private static final long JULIAN_DAY_NUMBER_FOR_UNIX_EPOCH = 2440588 ;
81
+ private static final String PARQUET = "Parquet" ;
78
82
79
83
private int [] indexes ;
80
84
@@ -234,6 +238,12 @@ private Object resolveObject(Object field, SeaTunnelDataType<?> fieldType) {
234
238
235
239
@ Override
236
240
public SeaTunnelRowType getSeaTunnelRowTypeInfo (String path ) throws FileConnectorException {
241
+ return getSeaTunnelRowTypeInfo (TablePath .DEFAULT , path );
242
+ }
243
+
244
+ @ Override
245
+ public SeaTunnelRowType getSeaTunnelRowTypeInfo (TablePath tablePath , String path )
246
+ throws FileConnectorException {
237
247
ParquetMetadata metadata ;
238
248
try (ParquetFileReader reader =
239
249
hadoopFileSystemProxy .doWithHadoopAuth (
@@ -259,19 +269,22 @@ public SeaTunnelRowType getSeaTunnelRowTypeInfo(String path) throws FileConnecto
259
269
String [] fields = new String [readColumns .size ()];
260
270
SeaTunnelDataType <?>[] types = new SeaTunnelDataType [readColumns .size ()];
261
271
indexes = new int [readColumns .size ()];
262
- for (int i = 0 ; i < readColumns .size (); i ++) {
263
- fields [i ] = readColumns .get (i );
264
- Type type = originalSchema .getType (fields [i ]);
265
- int fieldIndex = originalSchema .getFieldIndex (fields [i ]);
266
- indexes [i ] = fieldIndex ;
267
- types [i ] = parquetType2SeaTunnelType (type );
268
- }
272
+ buildColumnsWithErrorCheck (
273
+ tablePath ,
274
+ IntStream .range (0 , readColumns .size ()).iterator (),
275
+ i -> {
276
+ fields [i ] = readColumns .get (i );
277
+ Type type = originalSchema .getType (fields [i ]);
278
+ int fieldIndex = originalSchema .getFieldIndex (fields [i ]);
279
+ indexes [i ] = fieldIndex ;
280
+ types [i ] = parquetType2SeaTunnelType (type , fields [i ]);
281
+ });
269
282
seaTunnelRowType = new SeaTunnelRowType (fields , types );
270
283
seaTunnelRowTypeWithPartition = mergePartitionTypes (path , seaTunnelRowType );
271
284
return getActualSeaTunnelRowTypeInfo ();
272
285
}
273
286
274
- private SeaTunnelDataType <?> parquetType2SeaTunnelType (Type type ) {
287
+ private SeaTunnelDataType <?> parquetType2SeaTunnelType (Type type , String name ) {
275
288
if (type .isPrimitive ()) {
276
289
switch (type .asPrimitiveType ().getPrimitiveTypeName ()) {
277
290
case INT32 :
@@ -287,9 +300,8 @@ private SeaTunnelDataType<?> parquetType2SeaTunnelType(Type type) {
287
300
case DATE :
288
301
return LocalTimeType .LOCAL_DATE_TYPE ;
289
302
default :
290
- String errorMsg = String .format ("Not support this type [%s]" , type );
291
- throw new FileConnectorException (
292
- CommonErrorCodeDeprecated .UNSUPPORTED_DATA_TYPE , errorMsg );
303
+ throw CommonError .convertToSeaTunnelTypeError (
304
+ PARQUET , type .toString (), name );
293
305
}
294
306
case INT64 :
295
307
if (type .asPrimitiveType ().getOriginalType () == OriginalType .TIMESTAMP_MILLIS ) {
@@ -324,9 +336,7 @@ private SeaTunnelDataType<?> parquetType2SeaTunnelType(Type type) {
324
336
int scale = Integer .parseInt (splits [1 ]);
325
337
return new DecimalType (precision , scale );
326
338
default :
327
- String errorMsg = String .format ("Not support this type [%s]" , type );
328
- throw new FileConnectorException (
329
- CommonErrorCodeDeprecated .UNSUPPORTED_DATA_TYPE , errorMsg );
339
+ throw CommonError .convertToSeaTunnelTypeError ("Parquet" , type .toString (), name );
330
340
}
331
341
} else {
332
342
LogicalTypeAnnotation logicalTypeAnnotation =
@@ -339,7 +349,7 @@ private SeaTunnelDataType<?> parquetType2SeaTunnelType(Type type) {
339
349
for (int i = 0 ; i < fields .size (); i ++) {
340
350
Type fieldType = fields .get (i );
341
351
SeaTunnelDataType <?> seaTunnelDataType =
342
- parquetType2SeaTunnelType (fields .get (i ));
352
+ parquetType2SeaTunnelType (fields .get (i ), name );
343
353
fieldNames [i ] = fieldType .getName ();
344
354
seaTunnelDataTypes [i ] = seaTunnelDataType ;
345
355
}
@@ -349,9 +359,9 @@ private SeaTunnelDataType<?> parquetType2SeaTunnelType(Type type) {
349
359
case MAP :
350
360
GroupType groupType = type .asGroupType ().getType (0 ).asGroupType ();
351
361
SeaTunnelDataType <?> keyType =
352
- parquetType2SeaTunnelType (groupType .getType (0 ));
362
+ parquetType2SeaTunnelType (groupType .getType (0 ), name );
353
363
SeaTunnelDataType <?> valueType =
354
- parquetType2SeaTunnelType (groupType .getType (1 ));
364
+ parquetType2SeaTunnelType (groupType .getType (1 ), name );
355
365
return new MapType <>(keyType , valueType );
356
366
case LIST :
357
367
Type elementType ;
@@ -360,7 +370,8 @@ private SeaTunnelDataType<?> parquetType2SeaTunnelType(Type type) {
360
370
} catch (Exception e ) {
361
371
elementType = type .asGroupType ().getType (0 );
362
372
}
363
- SeaTunnelDataType <?> fieldType = parquetType2SeaTunnelType (elementType );
373
+ SeaTunnelDataType <?> fieldType =
374
+ parquetType2SeaTunnelType (elementType , name );
364
375
switch (fieldType .getSqlType ()) {
365
376
case STRING :
366
377
return ArrayType .STRING_ARRAY_TYPE ;
@@ -379,17 +390,12 @@ private SeaTunnelDataType<?> parquetType2SeaTunnelType(Type type) {
379
390
case DOUBLE :
380
391
return ArrayType .DOUBLE_ARRAY_TYPE ;
381
392
default :
382
- String errorMsg =
383
- String .format (
384
- "SeaTunnel array type not supported this genericType [%s] yet" ,
385
- fieldType );
386
- throw new FileConnectorException (
387
- CommonErrorCodeDeprecated .UNSUPPORTED_DATA_TYPE , errorMsg );
393
+ throw CommonError .convertToSeaTunnelTypeError (
394
+ PARQUET , type .toString (), name );
388
395
}
389
396
default :
390
- throw new FileConnectorException (
391
- CommonErrorCodeDeprecated .UNSUPPORTED_DATA_TYPE ,
392
- "SeaTunnel file connector not support this nest type" );
397
+ throw CommonError .convertToSeaTunnelTypeError (
398
+ PARQUET , type .toString (), name );
393
399
}
394
400
}
395
401
}
0 commit comments