3232import org .apache .hudi .util .StreamerUtil ;
3333
3434import org .apache .avro .Schema ;
35+
36+ import org .apache .flink .annotation .VisibleForTesting ;
3537import org .apache .flink .configuration .Configuration ;
3638import org .apache .flink .sql .parser .hive .ddl .SqlAlterHiveDatabase ;
3739import org .apache .flink .sql .parser .hive .ddl .SqlAlterHiveDatabaseOwner ;
4749import org .apache .flink .table .catalog .CatalogTable ;
4850import org .apache .flink .table .catalog .CatalogView ;
4951import org .apache .flink .table .catalog .ObjectPath ;
50- import org .apache .flink .table .catalog .ResolvedCatalogTable ;
51- import org .apache .flink .table .catalog .ResolvedSchema ;
5252import org .apache .flink .table .catalog .exceptions .CatalogException ;
5353import org .apache .flink .table .catalog .exceptions .DatabaseAlreadyExistException ;
5454import org .apache .flink .table .catalog .exceptions .DatabaseNotEmptyException ;
@@ -352,7 +352,8 @@ private org.apache.avro.Schema getLatestTableSchema(String path) {
352352 return null ;
353353 }
354354
355- private Table getHiveTable (ObjectPath tablePath ) throws TableNotExistException {
355+ @ VisibleForTesting
356+ public Table getHiveTable (ObjectPath tablePath ) throws TableNotExistException {
356357 try {
357358 Table hiveTable = client .getTable (tablePath .getDatabaseName (), tablePath .getObjectName ());
358359 return checkHoodieTable (hiveTable );
@@ -426,7 +427,7 @@ public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ig
426427 }
427428
428429 if (table instanceof CatalogView ) {
429- throw new HoodieCatalogException ("view not supported ." );
430+ throw new HoodieCatalogException ("Hoodie catalog does not support to CREATE VIEW ." );
430431 }
431432
432433 try {
@@ -436,7 +437,7 @@ public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ig
436437 //create hive table
437438 client .createTable (hiveTable );
438439 //init hoodie metaClient
439- initTableIfNotExists (tablePath , (ResolvedCatalogTable ) table );
440+ initTableIfNotExists (tablePath , (CatalogTable ) table );
440441 } catch (AlreadyExistsException e ) {
441442 if (!ignoreIfExists ) {
442443 throw new TableAlreadyExistException (getName (), tablePath , e );
@@ -447,12 +448,10 @@ public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ig
447448 }
448449 }
449450
450- private void initTableIfNotExists (ObjectPath tablePath , ResolvedCatalogTable catalogTable ) {
451- Map <String , String > options = catalogTable .getOptions ();
452- Configuration flinkConf = Configuration .fromMap (catalogTable .getOptions ());
451+ private void initTableIfNotExists (ObjectPath tablePath , CatalogTable catalogTable ) {
452+ Configuration flinkConf = Configuration .fromMap (applyOptionsHook (catalogTable .getOptions ()));
453453 flinkConf .addAllToProperties (hiveConf .getAllProperties ());
454- ResolvedSchema resolvedSchema = catalogTable .getResolvedSchema ();
455- final String avroSchema = AvroSchemaConverter .convertToSchema (resolvedSchema .toPhysicalRowDataType ().getLogicalType ()).toString ();
454+ final String avroSchema = AvroSchemaConverter .convertToSchema (catalogTable .getSchema ().toPhysicalRowDataType ().getLogicalType ()).toString ();
456455 flinkConf .setString (FlinkOptions .SOURCE_AVRO_SCHEMA , avroSchema );
457456
458457 // stores two copies of options:
@@ -461,16 +460,23 @@ private void initTableIfNotExists(ObjectPath tablePath, ResolvedCatalogTable cat
461460 // because the HoodieTableMetaClient is a heavy impl, we try to avoid initializing it
462461 // when calling #getTable.
463462
464- final String pkColumns = String .join ("," , resolvedSchema .getPrimaryKey ().get ().getColumns ());
465- flinkConf .setString (FlinkOptions .RECORD_KEY_FIELD , pkColumns );
466- options .put (TableOptionProperties .PK_CONSTRAINT_NAME , resolvedSchema .getPrimaryKey ().get ().getName ());
467- options .put (TableOptionProperties .PK_COLUMNS , pkColumns );
463+ if (catalogTable .getUnresolvedSchema ().getPrimaryKey ().isPresent ()) {
464+ final String pkColumns = String .join ("," , catalogTable .getUnresolvedSchema ().getPrimaryKey ().get ().getColumnNames ());
465+ String recordKey = flinkConf .get (FlinkOptions .RECORD_KEY_FIELD );
466+ if (!Objects .equals (pkColumns , recordKey )) {
467+ throw new HoodieCatalogException (String .format ("%s and %s are the different" , pkColumns , recordKey ));
468+ }
469+ }
468470
469471 if (catalogTable .isPartitioned ()) {
470472 final String partitions = String .join ("," , catalogTable .getPartitionKeys ());
471473 flinkConf .setString (FlinkOptions .PARTITION_PATH_FIELD , partitions );
472- options .put (TableOptionProperties .PARTITION_COLUMNS , partitions );
473474 }
475+
476+ if (!flinkConf .getOptional (PATH ).isPresent ()) {
477+ flinkConf .setString (PATH , inferTablePath (tablePath , catalogTable ));
478+ }
479+
474480 flinkConf .setString (FlinkOptions .TABLE_NAME , tablePath .getObjectName ());
475481 try {
476482 StreamerUtil .initTableIfNotExists (flinkConf );
@@ -485,14 +491,27 @@ private String inferTablePath(ObjectPath tablePath, CatalogBaseTable table) {
485491 try {
486492 Path dbLocation = new Path (client .getDatabase (tablePath .getDatabaseName ()).getLocationUri ());
487493 location = new Path (dbLocation , tablePath .getObjectName ()).toString ();
488- table .getOptions ().put (PATH .key (), location );
489494 } catch (TException e ) {
490495 throw new HoodieCatalogException (String .format ("Failed to infer hoodie table path for table %s" , tablePath ), e );
491496 }
492497 }
493498 return location ;
494499 }
495500
501+ private Map <String , String > applyOptionsHook (Map <String , String > options ) {
502+ Map <String , String > properties = new HashMap <>(options );
503+ if (!options .containsKey (FlinkOptions .RECORD_KEY_FIELD )) {
504+ properties .put (FlinkOptions .RECORD_KEY_FIELD .key (), FlinkOptions .RECORD_KEY_FIELD .defaultValue ());
505+ }
506+ if (!options .containsKey (FlinkOptions .PRECOMBINE_FIELD )) {
507+ properties .put (FlinkOptions .PRECOMBINE_FIELD .key (), FlinkOptions .PRECOMBINE_FIELD .defaultValue ());
508+ }
509+ if (!options .containsKey (FlinkOptions .TABLE_TYPE )) {
510+ properties .put (FlinkOptions .TABLE_TYPE .key (), FlinkOptions .TABLE_TYPE .defaultValue ());
511+ }
512+ return properties ;
513+ }
514+
496515 private Table instantiateHiveTable (ObjectPath tablePath , CatalogBaseTable table , String location , boolean useRealTimeInputFormat ) throws IOException {
497516 // let Hive set default parameters for us, e.g. serialization.format
498517 Table hiveTable =
@@ -501,7 +520,7 @@ private Table instantiateHiveTable(ObjectPath tablePath, CatalogBaseTable table,
501520 hiveTable .setOwner (UserGroupInformation .getCurrentUser ().getUserName ());
502521 hiveTable .setCreateTime ((int ) (System .currentTimeMillis () / 1000 ));
503522
504- Map <String , String > properties = new HashMap <> (table .getOptions ());
523+ Map <String , String > properties = applyOptionsHook (table .getOptions ());
505524 properties .put ("EXTERNAL" , "TRUE" );
506525 // Table comment
507526 if (table .getComment () != null ) {
@@ -551,20 +570,23 @@ private Table instantiateHiveTable(ObjectPath tablePath, CatalogBaseTable table,
551570
552571 hiveTable .setSd (sd );
553572
554- properties .putAll (TableOptionProperties .translateFlinkTableProperties2Spark ((ResolvedCatalogTable ) table , hiveConf ));
573+ properties .putAll (TableOptionProperties .translateFlinkTableProperties2Spark ((CatalogTable ) table , hiveConf ));
555574
556575 //set pk
557- ResolvedSchema resolvedSchema = ((ResolvedCatalogTable ) table ).getResolvedSchema ();
558- if (resolvedSchema .getPrimaryKey ().isPresent ()) {
559- String pkColumns = String .join ("," , resolvedSchema .getPrimaryKey ().get ().getColumns ());
576+ if (table .getUnresolvedSchema ().getPrimaryKey ().isPresent ()) {
577+ String pkColumns = String .join ("," , table .getUnresolvedSchema ().getPrimaryKey ().get ().getColumnNames ());
560578 String recordKey = properties .getOrDefault (FlinkOptions .RECORD_KEY_FIELD .key (), FlinkOptions .RECORD_KEY_FIELD .defaultValue ());
561579 if (!Objects .equals (pkColumns , recordKey )) {
562580 throw new HoodieCatalogException (String .format ("%s and %s are the different" , pkColumns , recordKey ));
563581 }
564- properties .put (PK_CONSTRAINT_NAME , resolvedSchema . getPrimaryKey ().get ().getName ());
582+ properties .put (PK_CONSTRAINT_NAME , table . getUnresolvedSchema (). getPrimaryKey ().get ().getConstraintName ());
565583 properties .put (PK_COLUMNS , pkColumns );
566584 }
567585
586+ if (!properties .containsKey (FlinkOptions .PATH .key ())) {
587+ properties .put (FlinkOptions .PATH .key (), location );
588+ }
589+
568590 hiveTable .setParameters (properties );
569591 return hiveTable ;
570592 }
@@ -588,7 +610,7 @@ public List<String> listTables(String databaseName)
588610 @ Override
589611 public List <String > listViews (String databaseName )
590612 throws DatabaseNotExistException , CatalogException {
591- throw new HoodieCatalogException ("view not supported. " );
613+ throw new HoodieCatalogException ("Hoodie catalog does not support to listViews " );
592614 }
593615
594616 @ Override
@@ -684,7 +706,7 @@ public void alterTable(
684706 throw new HoodieCatalogException (String .format ("The %s is not hoodie table" , tablePath .getObjectName ()));
685707 }
686708 if (newCatalogTable instanceof CatalogView ) {
687- throw new HoodieCatalogException ("view not supported. " );
709+ throw new HoodieCatalogException ("Hoodie catalog does not support to ALTER VIEW " );
688710 }
689711
690712 try {
0 commit comments