1818
1919package org .apache .hudi .table .catalog ;
2020
21+ import org .apache .hudi .client .HoodieFlinkWriteClient ;
2122import org .apache .hudi .common .fs .FSUtils ;
2223import org .apache .hudi .common .model .HoodieFileFormat ;
2324import org .apache .hudi .common .table .HoodieTableMetaClient ;
25+ import org .apache .hudi .common .table .timeline .HoodieActiveTimeline ;
2426import org .apache .hudi .common .util .StringUtils ;
2527import org .apache .hudi .common .util .collection .Pair ;
28+ import org .apache .hudi .config .HoodieWriteConfig ;
2629import org .apache .hudi .configuration .FlinkOptions ;
2730import org .apache .hudi .configuration .OptionsResolver ;
2831import org .apache .hudi .exception .HoodieCatalogException ;
32+ import org .apache .hudi .exception .HoodieMetadataException ;
2933import org .apache .hudi .hadoop .utils .HoodieInputFormatUtils ;
3034import org .apache .hudi .sync .common .util .ConfigUtils ;
3135import org .apache .hudi .table .format .FilePathUtils ;
7478import org .apache .hadoop .hive .metastore .api .Database ;
7579import org .apache .hadoop .hive .metastore .api .FieldSchema ;
7680import org .apache .hadoop .hive .metastore .api .InvalidOperationException ;
81+ import org .apache .hadoop .hive .metastore .api .MetaException ;
7782import org .apache .hadoop .hive .metastore .api .NoSuchObjectException ;
7883import org .apache .hadoop .hive .metastore .api .PrincipalType ;
7984import org .apache .hadoop .hive .metastore .api .SerDeInfo ;
8792import org .slf4j .LoggerFactory ;
8893
8994import java .io .IOException ;
95+ import java .util .ArrayList ;
9096import java .util .Arrays ;
9197import java .util .Collections ;
9298import java .util .HashMap ;
@@ -488,7 +494,8 @@ private void initTableIfNotExists(ObjectPath tablePath, CatalogTable catalogTabl
488494 }
489495 }
490496
491- private String inferTablePath (ObjectPath tablePath , CatalogBaseTable table ) {
497+ @ VisibleForTesting
498+ public String inferTablePath (ObjectPath tablePath , CatalogBaseTable table ) {
492499 String location = table .getOptions ().getOrDefault (PATH .key (), "" );
493500 if (StringUtils .isNullOrEmpty (location )) {
494501 try {
@@ -777,7 +784,44 @@ public void createPartition(
777784 public void dropPartition (
778785 ObjectPath tablePath , CatalogPartitionSpec partitionSpec , boolean ignoreIfNotExists )
779786 throws PartitionNotExistException , CatalogException {
780- throw new HoodieCatalogException ("Not supported." );
787+ checkNotNull (tablePath , "Table path cannot be null" );
788+ checkNotNull (partitionSpec , "CatalogPartitionSpec cannot be null" );
789+
790+ HoodieFlinkWriteClient <?> writeClient = null ;
791+ try {
792+ CatalogBaseTable table = getTable (tablePath );
793+ writeClient = createWriteClient (tablePath , table );
794+ boolean hiveStylePartitioning = Boolean .parseBoolean (table .getOptions ().get (FlinkOptions .HIVE_STYLE_PARTITIONING .key ()));
795+ writeClient .deletePartitions (
796+ Collections .singletonList (HoodieCatalogUtil .inferPartitionPath (hiveStylePartitioning , partitionSpec )),
797+ HoodieActiveTimeline .createNewInstantTime ())
798+ .forEach (writeStatus -> {
799+ if (writeStatus .hasErrors ()) {
800+ throw new HoodieMetadataException (String .format ("Failed to commit metadata table records at file id %s." , writeStatus .getFileId ()));
801+ }
802+ });
803+
804+ client .dropPartition (
805+ tablePath .getDatabaseName (),
806+ tablePath .getObjectName (),
807+ getOrderedFullPartitionValues (
808+ partitionSpec , HiveSchemaUtils .getFieldNames (getHiveTable (tablePath ).getPartitionKeys ()), tablePath ),
809+ true );
810+ } catch (NoSuchObjectException e ) {
811+ if (!ignoreIfNotExists ) {
812+ throw new PartitionNotExistException (getName (), tablePath , partitionSpec , e );
813+ }
814+ } catch (MetaException | TableNotExistException | PartitionSpecInvalidException e ) {
815+ throw new PartitionNotExistException (getName (), tablePath , partitionSpec , e );
816+ } catch (Exception e ) {
817+ throw new CatalogException (
818+ String .format (
819+ "Failed to drop partition %s of table %s" , partitionSpec , tablePath ));
820+ } finally {
821+ if (writeClient != null ) {
822+ writeClient .close ();
823+ }
824+ }
781825 }
782826
783827 @ Override
@@ -790,6 +834,45 @@ public void alterPartition(
790834 throw new HoodieCatalogException ("Not supported." );
791835 }
792836
837+ /**
838+ * Get a list of ordered partition values by re-arranging them based on the given list of
839+ * partition keys. If the partition value is null, it'll be converted into default partition
840+ * name.
841+ *
842+ * @param partitionSpec a partition spec.
843+ * @param partitionKeys a list of partition keys.
844+ * @param tablePath path of the table to which the partition belongs.
845+ * @return A list of partition values ordered according to partitionKeys.
846+ * @throws PartitionSpecInvalidException thrown if partitionSpec and partitionKeys have
847+ * different sizes, or any key in partitionKeys doesn't exist in partitionSpec.
848+ */
849+ @ VisibleForTesting
850+ public List <String > getOrderedFullPartitionValues (
851+ CatalogPartitionSpec partitionSpec , List <String > partitionKeys , ObjectPath tablePath )
852+ throws PartitionSpecInvalidException {
853+ Map <String , String > spec = partitionSpec .getPartitionSpec ();
854+ if (spec .size () != partitionKeys .size ()) {
855+ throw new PartitionSpecInvalidException (
856+ getName (), partitionKeys , tablePath , partitionSpec );
857+ }
858+
859+ List <String > values = new ArrayList <>(spec .size ());
860+ for (String key : partitionKeys ) {
861+ if (!spec .containsKey (key )) {
862+ throw new PartitionSpecInvalidException (
863+ getName (), partitionKeys , tablePath , partitionSpec );
864+ } else {
865+ String value = spec .get (key );
866+ if (value == null ) {
867+ value = getHiveConf ().getVar (HiveConf .ConfVars .DEFAULTPARTITIONNAME );
868+ }
869+ values .add (value );
870+ }
871+ }
872+
873+ return values ;
874+ }
875+
793876 @ Override
794877 public List <String > listFunctions (String databaseName )
795878 throws DatabaseNotExistException , CatalogException {
@@ -906,4 +989,23 @@ private Map<String, String> supplementOptions(
906989 return newOptions ;
907990 }
908991 }
992+
993+ private HoodieFlinkWriteClient <?> createWriteClient (
994+ ObjectPath tablePath ,
995+ CatalogBaseTable table ) throws Exception {
996+ Map <String , String > options = table .getOptions ();
997+ // enable auto-commit though ~
998+ options .put (HoodieWriteConfig .AUTO_COMMIT_ENABLE .key (), "true" );
999+ return StreamerUtil .createWriteClient (
1000+ Configuration .fromMap (options )
1001+ .set (FlinkOptions .TABLE_NAME , tablePath .getObjectName ())
1002+ .set (FlinkOptions .SOURCE_AVRO_SCHEMA ,
1003+ HoodieTableMetaClient .builder ().setBasePath (inferTablePath (tablePath , table )).setConf (hiveConf ).build ()
1004+ .getTableConfig ().getTableCreateSchema ().get ().toString ()));
1005+ }
1006+
1007+ @ VisibleForTesting
1008+ public IMetaStoreClient getClient () {
1009+ return client ;
1010+ }
9091011}
0 commit comments