You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues-all@impala.apache.org by "bharath v (JIRA)" <ji...@apache.org> on 2018/09/24 23:40:00 UTC

[jira] [Updated] (IMPALA-7615) Partition metadata mismatch should be handled gracefully in local catalog mode.

     [ https://issues.apache.org/jira/browse/IMPALA-7615?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

bharath v updated IMPALA-7615:
------------------------------
    Description: 
*This is a Catalog v2 only improvement*

An RPC to fetch partition metadata for a partition ID that does not exist on the Catalog server currently throws IAE.
{noformat}
@Override
  public TGetPartialCatalogObjectResponse getPartialInfo(
      TGetPartialCatalogObjectRequest req) throws TableLoadingException {
      for (long partId : partIds) {
        HdfsPartition part = partitionMap_.get(partId);
        Preconditions.checkArgument(part != null, "Partition id %s does not exist",  <------
            partId);
        TPartialPartitionInfo partInfo = new TPartialPartitionInfo(partId);

        if (req.table_info_selector.want_partition_names) {
          partInfo.setName(part.getPartitionName());
        }

        if (req.table_info_selector.want_partition_metadata) {
          partInfo.hms_partition = part.toHmsPartition();
{noformat}
This is undesirable since such exceptions are not transparently retried in the frontend. Instead we should fix this code path to throw InconsistentMetadataException, similar to what we do for other code paths that handle such inconsistent metadata like version changes.

An example stack trace that hits this issue looks like follows,
{noformat}
org.apache.impala.catalog.local.LocalCatalogException: Could not load partitions for table partition_level_tests.store_sales
at org.apache.impala.catalog.local.LocalFsTable.loadPartitions(LocalFsTable.java:399)
at org.apache.impala.catalog.FeCatalogUtils.loadAllPartitions(FeCatalogUtils.java:207)
at org.apache.impala.catalog.local.LocalFsTable.getMajorityFormat(LocalFsTable.java:244)
at org.apache.impala.planner.HdfsTableSink.computeResourceProfile(HdfsTableSink.java:75)
at org.apache.impala.planner.PlanFragment.computeResourceProfile(PlanFragment.java:233)
at org.apache.impala.planner.Planner.computeResourceReqs(Planner.java:365)
at org.apache.impala.service.Frontend.createExecRequest(Frontend.java:1020)
at org.apache.impala.service.Frontend.doCreateExecRequest(Frontend.java:1162)
at org.apache.impala.service.Frontend.createExecRequest(Frontend.java:1077)
at org.apache.impala.service.JniFrontend.createExecRequest(JniFrontend.java:156)
Caused by: org.apache.thrift.TException: TGetPartialCatalogObjectResponse(status:TStatus(status_code:GENERAL, error_msgs:[IllegalArgumentException: Partition id 10084 does not exist]), lookup_status:OK)
at org.apache.impala.catalog.local.CatalogdMetaProvider.sendRequest(CatalogdMetaProvider.java:322)
at org.apache.impala.catalog.local.CatalogdMetaProvider.loadPartitionsFromCatalogd(CatalogdMetaProvider.java:644)
at org.apache.impala.catalog.local.CatalogdMetaProvider.loadPartitionsByRefs(CatalogdMetaProvider.java:610)
at org.apache.impala.catalog.local.LocalFsTable.loadPartitions(LocalFsTable.java:395)
... 9 more{noformat}

  was:
*This is a Catalog v2 only improvement*

An RPC to fetch partition metadata for a partition ID that does not exist on the Catalog server currently throws IAE.
{noformat}
@Override
  public TGetPartialCatalogObjectResponse getPartialInfo(
      TGetPartialCatalogObjectRequest req) throws TableLoadingException {
      for (long partId : partIds) {
        HdfsPartition part = partitionMap_.get(partId);
        Preconditions.checkArgument(part != null, "Partition id %s does not exist",  <------
            partId);
        TPartialPartitionInfo partInfo = new TPartialPartitionInfo(partId);

        if (req.table_info_selector.want_partition_names) {
          partInfo.setName(part.getPartitionName());
        }

        if (req.table_info_selector.want_partition_metadata) {
          partInfo.hms_partition = part.toHmsPartition();
{noformat}
This is undesirable since such exceptions are not transparently retried in the frontend. Instead we should fix this code path to throw InconsistentMetadataException, similar to what we do for other code paths that handle such inconsistent metadata like version changes.


> Partition metadata mismatch should be handled gracefully in local catalog mode.
> -------------------------------------------------------------------------------
>
>                 Key: IMPALA-7615
>                 URL: https://issues.apache.org/jira/browse/IMPALA-7615
>             Project: IMPALA
>          Issue Type: Improvement
>          Components: Frontend
>    Affects Versions: Impala 3.1.0
>            Reporter: bharath v
>            Priority: Major
>
> *This is a Catalog v2 only improvement*
> An RPC to fetch partition metadata for a partition ID that does not exist on the Catalog server currently throws IAE.
> {noformat}
> @Override
>   public TGetPartialCatalogObjectResponse getPartialInfo(
>       TGetPartialCatalogObjectRequest req) throws TableLoadingException {
>       for (long partId : partIds) {
>         HdfsPartition part = partitionMap_.get(partId);
>         Preconditions.checkArgument(part != null, "Partition id %s does not exist",  <------
>             partId);
>         TPartialPartitionInfo partInfo = new TPartialPartitionInfo(partId);
>         if (req.table_info_selector.want_partition_names) {
>           partInfo.setName(part.getPartitionName());
>         }
>         if (req.table_info_selector.want_partition_metadata) {
>           partInfo.hms_partition = part.toHmsPartition();
> {noformat}
> This is undesirable since such exceptions are not transparently retried in the frontend. Instead we should fix this code path to throw InconsistentMetadataException, similar to what we do for other code paths that handle such inconsistent metadata like version changes.
> An example stack trace that hits this issue looks like follows,
> {noformat}
> org.apache.impala.catalog.local.LocalCatalogException: Could not load partitions for table partition_level_tests.store_sales
> at org.apache.impala.catalog.local.LocalFsTable.loadPartitions(LocalFsTable.java:399)
> at org.apache.impala.catalog.FeCatalogUtils.loadAllPartitions(FeCatalogUtils.java:207)
> at org.apache.impala.catalog.local.LocalFsTable.getMajorityFormat(LocalFsTable.java:244)
> at org.apache.impala.planner.HdfsTableSink.computeResourceProfile(HdfsTableSink.java:75)
> at org.apache.impala.planner.PlanFragment.computeResourceProfile(PlanFragment.java:233)
> at org.apache.impala.planner.Planner.computeResourceReqs(Planner.java:365)
> at org.apache.impala.service.Frontend.createExecRequest(Frontend.java:1020)
> at org.apache.impala.service.Frontend.doCreateExecRequest(Frontend.java:1162)
> at org.apache.impala.service.Frontend.createExecRequest(Frontend.java:1077)
> at org.apache.impala.service.JniFrontend.createExecRequest(JniFrontend.java:156)
> Caused by: org.apache.thrift.TException: TGetPartialCatalogObjectResponse(status:TStatus(status_code:GENERAL, error_msgs:[IllegalArgumentException: Partition id 10084 does not exist]), lookup_status:OK)
> at org.apache.impala.catalog.local.CatalogdMetaProvider.sendRequest(CatalogdMetaProvider.java:322)
> at org.apache.impala.catalog.local.CatalogdMetaProvider.loadPartitionsFromCatalogd(CatalogdMetaProvider.java:644)
> at org.apache.impala.catalog.local.CatalogdMetaProvider.loadPartitionsByRefs(CatalogdMetaProvider.java:610)
> at org.apache.impala.catalog.local.LocalFsTable.loadPartitions(LocalFsTable.java:395)
> ... 9 more{noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-all-unsubscribe@impala.apache.org
For additional commands, e-mail: issues-all-help@impala.apache.org