[ 
https://issues.apache.org/jira/browse/IMPALA-6671?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16565959#comment-16565959
 ] 

bharath v commented on IMPALA-6671:
-----------------------------------

Another thing to keep in mind is the "SYNC_DDL" based propagation. I'm not 
totally sure if your proposal works with it. We promise a catalog version 
upfront that is expected to contain the result of the DDL and coordinators wait 
for that minimum version to be propagated. With your proposal, if we keep 
pushing tables into the next round of updates, wondering if that breaks 
anything.

> Metadata operations that modify a table blocks topic updates for other 
> unrelated operations
> -------------------------------------------------------------------------------------------
>
>                 Key: IMPALA-6671
>                 URL: https://issues.apache.org/jira/browse/IMPALA-6671
>             Project: IMPALA
>          Issue Type: Bug
>          Components: Catalog
>    Affects Versions: Impala 2.10.0, Impala 2.11.0, Impala 2.12.0
>            Reporter: Mostafa Mokhtar
>            Priority: Critical
>              Labels: catalog-server, perfomance
>
> Metadata operations that mutate the state of a table like "compute stats foo" 
> or "alter recover partitions" block topic updates for read only operations 
> against unrelated tables as "describe bar".
> Thread for blocked operation
> {code}
> "Thread-7" prio=10 tid=0x0000000011613000 nid=0x21b3b waiting on condition 
> [0x00007f5f2ef52000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00007f6f57ff0240> (a 
> java.util.concurrent.locks.ReentrantLock$NonfairSync)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
>         at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:834)
>         at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:867)
>         at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1197)
>         at 
> java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:214)
>         at 
> java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:290)
>         at 
> org.apache.impala.catalog.CatalogServiceCatalog.addTableToCatalogDeltaHelper(CatalogServiceCatalog.java:639)
>         at 
> org.apache.impala.catalog.CatalogServiceCatalog.addTableToCatalogDelta(CatalogServiceCatalog.java:611)
>         at 
> org.apache.impala.catalog.CatalogServiceCatalog.addDatabaseToCatalogDelta(CatalogServiceCatalog.java:567)
>         at 
> org.apache.impala.catalog.CatalogServiceCatalog.getCatalogDelta(CatalogServiceCatalog.java:449)
>         at 
> org.apache.impala.service.JniCatalog.getCatalogDelta(JniCatalog.java:126)
> {code}
> Thread for blocking operation 
> {code}
> "Thread-130" prio=10 tid=0x00000000113d5800 nid=0x2499d runnable 
> [0x00007f5ef80d0000]
>    java.lang.Thread.State: RUNNABLE
>         at java.net.SocketInputStream.socketRead0(Native Method)
>         at java.net.SocketInputStream.read(SocketInputStream.java:152)
>         at java.net.SocketInputStream.read(SocketInputStream.java:122)
>         at java.io.BufferedInputStream.fill(BufferedInputStream.java:235)
>         at java.io.BufferedInputStream.read1(BufferedInputStream.java:275)
>         at java.io.BufferedInputStream.read(BufferedInputStream.java:334)
>         - locked <0x00007f5fffcd9f18> (a java.io.BufferedInputStream)
>         at 
> org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:127)
>         at org.apache.thrift.transport.TTransport.readAll(TTransport.java:84)
>         at 
> org.apache.thrift.transport.TSaslTransport.readLength(TSaslTransport.java:346)
>         at 
> org.apache.thrift.transport.TSaslTransport.readFrame(TSaslTransport.java:423)
>         at 
> org.apache.thrift.transport.TSaslTransport.read(TSaslTransport.java:405)
>         at 
> org.apache.thrift.transport.TSaslClientTransport.read(TSaslClientTransport.java:37)
>         at org.apache.thrift.transport.TTransport.readAll(TTransport.java:84)
>         at 
> org.apache.hadoop.hive.thrift.TFilterTransport.readAll(TFilterTransport.java:62)
>         at 
> org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:378)
>         at 
> org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:297)
>         at 
> org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:204)
>         at 
> org.apache.thrift.TServiceClient.receiveBase(TServiceClient.java:69)
>         at 
> org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.recv_add_partitions_req(ThriftHiveMetastore.java:1639)
>         at 
> org.apache.hadoop.hive.metastore.api.ThriftHiveMetastore$Client.add_partitions_req(ThriftHiveMetastore.java:1626)
>         at 
> org.apache.hadoop.hive.metastore.HiveMetaStoreClient.add_partitions(HiveMetaStoreClient.java:609)
>         at sun.reflect.GeneratedMethodAccessor18.invoke(Unknown Source)
>         at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>         at java.lang.reflect.Method.invoke(Method.java:606)
>         at 
> org.apache.hadoop.hive.metastore.RetryingMetaStoreClient.invoke(RetryingMetaStoreClient.java:101)
>         at com.sun.proxy.$Proxy10.add_partitions(Unknown Source)
>         at 
> org.apache.impala.service.CatalogOpExecutor.alterTableRecoverPartitions(CatalogOpExecutor.java:2651)
>         at 
> org.apache.impala.service.CatalogOpExecutor.alterTable(CatalogOpExecutor.java:525)
>         at 
> org.apache.impala.service.CatalogOpExecutor.execDdlRequest(CatalogOpExecutor.java:262)
>         at org.apache.impala.service.JniCatalog.execDdl(JniCatalog.java:146)
> {code}



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

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to