1919package org .apache .hudi .table ;
2020
2121import org .apache .hudi .common .model .DefaultHoodieRecordPayload ;
22- import org .apache .hudi .common .model .EventTimeAvroPayload ;
2322import org .apache .hudi .common .util .StringUtils ;
2423import org .apache .hudi .configuration .FlinkOptions ;
2524import org .apache .hudi .configuration .OptionsResolver ;
2625import org .apache .hudi .exception .HoodieValidationException ;
27- import org .apache .hudi .hive .MultiPartKeysValueExtractor ;
2826import org .apache .hudi .index .HoodieIndex ;
2927import org .apache .hudi .keygen .ComplexAvroKeyGenerator ;
3028import org .apache .hudi .keygen .NonpartitionedAvroKeyGenerator ;
3836import org .apache .flink .table .api .ValidationException ;
3937import org .apache .flink .table .api .constraints .UniqueConstraint ;
4038import org .apache .flink .table .catalog .CatalogTable ;
39+ import org .apache .flink .table .catalog .ObjectIdentifier ;
4140import org .apache .flink .table .catalog .ResolvedSchema ;
4241import org .apache .flink .table .connector .sink .DynamicTableSink ;
4342import org .apache .flink .table .connector .source .DynamicTableSource ;
@@ -71,7 +70,7 @@ public DynamicTableSource createDynamicTableSource(Context context) {
7170 Configuration conf = FlinkOptions .fromMap (context .getCatalogTable ().getOptions ());
7271 ResolvedSchema schema = context .getCatalogTable ().getResolvedSchema ();
7372 sanityCheck (conf , schema );
74- setupConfOptions (conf , context .getObjectIdentifier (). getObjectName () , context .getCatalogTable (), schema );
73+ setupConfOptions (conf , context .getObjectIdentifier (), context .getCatalogTable (), schema );
7574
7675 Path path = new Path (conf .getOptional (FlinkOptions .PATH ).orElseThrow (() ->
7776 new ValidationException ("Option [path] should not be empty." )));
@@ -90,7 +89,7 @@ public DynamicTableSink createDynamicTableSink(Context context) {
9089 "Option [path] should not be empty." );
9190 ResolvedSchema schema = context .getCatalogTable ().getResolvedSchema ();
9291 sanityCheck (conf , schema );
93- setupConfOptions (conf , context .getObjectIdentifier (). getObjectName () , context .getCatalogTable (), schema );
92+ setupConfOptions (conf , context .getObjectIdentifier (), context .getCatalogTable (), schema );
9493 return new HoodieTableSink (conf , schema );
9594 }
9695
@@ -154,35 +153,30 @@ private void sanityCheck(Configuration conf, ResolvedSchema schema) {
154153 throw new HoodieValidationException ("Field " + preCombineField + " does not exist in the table schema."
155154 + "Please check '" + FlinkOptions .PRECOMBINE_FIELD .key () + "' option." );
156155 }
157- } else if (FlinkOptions .isDefaultValueDefined (conf , FlinkOptions .PAYLOAD_CLASS_NAME )) {
158- // if precombine field is specified but payload clazz is default,
159- // use DefaultHoodieRecordPayload to make sure the precombine field is always taken for
160- // comparing.
161- conf .setString (FlinkOptions .PAYLOAD_CLASS_NAME , EventTimeAvroPayload .class .getName ());
162156 }
163157 }
164158
165159 /**
166- * Sets up the config options based on the table definition, for e.g the table name, primary key.
160+ * Sets up the config options based on the table definition, for e.g, the table name, primary key.
167161 *
168- * @param conf The configuration to setup
169- * @param tableName The table name
162+ * @param conf The configuration to set up
163+ * @param tablePath The table path
170164 * @param table The catalog table
171165 * @param schema The physical schema
172166 */
173167 private static void setupConfOptions (
174168 Configuration conf ,
175- String tableName ,
169+ ObjectIdentifier tablePath ,
176170 CatalogTable table ,
177171 ResolvedSchema schema ) {
178172 // table name
179- conf .setString (FlinkOptions .TABLE_NAME .key (), tableName );
173+ conf .setString (FlinkOptions .TABLE_NAME .key (), tablePath . getObjectName () );
180174 // hoodie key about options
181175 setupHoodieKeyOptions (conf , table );
182176 // compaction options
183177 setupCompactionOptions (conf );
184178 // hive options
185- setupHiveOptions (conf );
179+ setupHiveOptions (conf , tablePath );
186180 // read options
187181 setupReadOptions (conf );
188182 // write options
@@ -309,10 +303,12 @@ private static void setupCompactionOptions(Configuration conf) {
309303 /**
310304 * Sets up the hive options from the table definition.
311305 */
312- private static void setupHiveOptions (Configuration conf ) {
313- if (!conf .getBoolean (FlinkOptions .HIVE_STYLE_PARTITIONING )
314- && FlinkOptions .isDefaultValueDefined (conf , FlinkOptions .HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME )) {
315- conf .setString (FlinkOptions .HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME , MultiPartKeysValueExtractor .class .getName ());
306+ private static void setupHiveOptions (Configuration conf , ObjectIdentifier tablePath ) {
307+ if (!conf .contains (FlinkOptions .HIVE_SYNC_DB )) {
308+ conf .setString (FlinkOptions .HIVE_SYNC_DB , tablePath .getDatabaseName ());
309+ }
310+ if (!conf .contains (FlinkOptions .HIVE_SYNC_TABLE )) {
311+ conf .setString (FlinkOptions .HIVE_SYNC_TABLE , tablePath .getObjectName ());
316312 }
317313 }
318314
0 commit comments