bowenli86 commented on a change in pull request #8449: [FLINK-12235][hive] Support partition related operations in HiveCatalog URL: https://github.com/apache/flink/pull/8449#discussion_r284856201
########## File path: flink-connectors/flink-connector-hive/src/main/java/org/apache/flink/table/catalog/hive/HiveCatalogBase.java ########## @@ -407,4 +443,201 @@ private Table getHiveTable(ObjectPath tablePath) throws TableNotExistException { String.format("Failed to get table %s from Hive metastore", tablePath.getFullName()), e); } } + + // ------ partitions ------ + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + try { + Table hiveTable = getHiveTable(tablePath); + return client.getPartition(tablePath.getDatabaseName(), tablePath.getObjectName(), + getFullPartitionValues(tablePath, partitionSpec, getFieldNames(hiveTable.getPartitionKeys()))) != null; + } catch (NoSuchObjectException | TableNotExistException | PartitionSpecInvalidException e) { + return false; + } catch (TException e) { + throw new CatalogException( + String.format("Failed to get partition %s of table %s", partitionSpec, tablePath), e); + } + } + + @Override + public void createPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition partition, boolean ignoreIfExists) + throws TableNotExistException, TableNotPartitionedException, PartitionSpecInvalidException, PartitionAlreadyExistsException, CatalogException { + + validateCatalogPartition(partition); + + Table hiveTable = getHiveTable(tablePath); + + ensurePartitionedTable(tablePath, hiveTable); + + try { + client.add_partition(createHivePartition(hiveTable, partitionSpec, partition)); + } catch (AlreadyExistsException e) { + if (!ignoreIfExists) { + throw new PartitionAlreadyExistsException(catalogName, tablePath, partitionSpec); + } + } catch (TException e) { + throw new CatalogException( + String.format("Failed to create partition %s of table %s", partitionSpec, tablePath)); + } + } + + @Override + public void dropPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + try { + Table hiveTable = getHiveTable(tablePath); + ensurePartitionedTable(tablePath, hiveTable); + client.dropPartition(tablePath.getDatabaseName(), tablePath.getObjectName(), + getFullPartitionValues(tablePath, partitionSpec, getFieldNames(hiveTable.getPartitionKeys())), true); + } catch (NoSuchObjectException e) { + if (!ignoreIfNotExists) { + throw new PartitionNotExistException(catalogName, tablePath, partitionSpec); + } + } catch (TException e) { + throw new CatalogException( + String.format("Failed to drop partition %s of table %s", partitionSpec, tablePath)); + } catch (TableNotPartitionedException | TableNotExistException | PartitionSpecInvalidException e) { + throw new PartitionNotExistException(catalogName, tablePath, partitionSpec, e); + } + } + + @Override + public List<CatalogPartitionSpec> listPartitions(ObjectPath tablePath) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + Table hiveTable = getHiveTable(tablePath); + + ensurePartitionedTable(tablePath, hiveTable); + + try { + return client.listPartitionNames(tablePath.getDatabaseName(), tablePath.getObjectName(), (short) -1).stream() + .map(HiveCatalogBase::createPartitionSpec).collect(Collectors.toList()); + } catch (TException e) { + throw new CatalogException( + String.format("Failed to list partitions of table %s", tablePath), e); + } + } + + @Override + public List<CatalogPartitionSpec> listPartitions(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + Table hiveTable = getHiveTable(tablePath); + + ensurePartitionedTable(tablePath, hiveTable); + + try { + // partition spec can be partial + List<String> partialVals = MetaStoreUtils.getPvals(hiveTable.getPartitionKeys(), partitionSpec.getPartitionSpec()); + return client.listPartitionNames(tablePath.getDatabaseName(), tablePath.getObjectName(), partialVals, + (short) -1).stream().map(HiveCatalogBase::createPartitionSpec).collect(Collectors.toList()); + } catch (TException e) { + throw new CatalogException( + String.format("Failed to list partitions of table %s", tablePath), e); + } + } + + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws PartitionNotExistException, CatalogException { + try { + Table hiveTable = getHiveTable(tablePath); + ensurePartitionedTable(tablePath, hiveTable); + Partition hivePartition = client.getPartition(tablePath.getDatabaseName(), tablePath.getObjectName(), + getFullPartitionValues(tablePath, partitionSpec, getFieldNames(hiveTable.getPartitionKeys()))); + return createCatalogPartition(hivePartition); + } catch (NoSuchObjectException e) { + throw new PartitionNotExistException(catalogName, tablePath, partitionSpec); + } catch (TException e) { + throw new CatalogException( + String.format("Failed to get partition %s of table %s", partitionSpec, tablePath), e); + } catch (TableNotPartitionedException | TableNotExistException | PartitionSpecInvalidException e) { + throw new PartitionNotExistException(catalogName, tablePath, partitionSpec, e); + } + } + + @Override + public void alterPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec, CatalogPartition newPartition, boolean ignoreIfNotExists) + throws PartitionNotExistException, CatalogException { + // Explicitly check if the partition exists or not + // because alter_partition() doesn't throw NoSuchObjectException like dropPartition() when the target doesn't exist + if (partitionExists(tablePath, partitionSpec)) { + try { + Table hiveTable = getHiveTable(tablePath); + ensurePartitionedTable(tablePath, hiveTable); + client.alter_partition( + tablePath.getDatabaseName(), + tablePath.getObjectName(), + createHivePartition(hiveTable, partitionSpec, newPartition) + ); + } catch (TException e) { + throw new CatalogException( + String.format("Failed to alter existing partition with new partition %s of table %s", + partitionSpec, tablePath), e); + } catch (TableNotPartitionedException | TableNotExistException | PartitionSpecInvalidException e) { + throw new PartitionNotExistException(catalogName, tablePath, partitionSpec, e); + } + } else if (!ignoreIfNotExists) { + throw new PartitionNotExistException(catalogName, tablePath, partitionSpec); + } + } + + private void ensurePartitionedTable(ObjectPath tablePath, Table hiveTable) throws TableNotPartitionedException { + if (hiveTable.getPartitionKeysSize() == 0) { + throw new TableNotPartitionedException(catalogName, tablePath); + } + } + + /** + * Get field names from field schemas. + */ + static List<String> getFieldNames(List<FieldSchema> fieldSchemas) { + List<String> names = new ArrayList<>(fieldSchemas.size()); + for (FieldSchema fs : fieldSchemas) { + names.add(fs.getName()); + } + return names; + } + + /** + * Creates a {@link CatalogPartitionSpec} from a Hive partition name string. + * Example of Hive partition name string - "name=bob/year=2019" + */ + private static CatalogPartitionSpec createPartitionSpec(String hivePartitionName) { + String[] partKeyVals = hivePartitionName.split("/"); + Map<String, String> spec = new HashMap<>(partKeyVals.length); + for (String keyVal : partKeyVals) { + String[] kv = keyVal.split("="); + spec.put(kv[0], kv[1]); + } + return new CatalogPartitionSpec(spec); + } + + /** + * Get a list of ordered partition values by re-arranging them based on the given list of partition keys. + * Review comment: minor: missing a param for `tablePath` ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: us...@infra.apache.org With regards, Apache Git Services