You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@phoenix.apache.org by Nick Dimiduk <nd...@apache.org> on 2016/05/09 03:39:54 UTC

Re: Write path blocked by MetaDataEndpoint acquiring region lock

Switching from user to dev, user to BCC.

Hi fellas,

I suffered an ingest outage in production this weekend with the symptoms
discussed here; I need to revive this thread.

After speaking a bit offline with Arun, I believe my use of the
ROW_TIMESTAMP feature in combination with a modest total data volume is the
culprit. Prod was unblocked by performing the following actions, though I
must admit I don't know if one of these alone would have been enough:

1. increasing phoenix.stats.guidepost.width to 500mb
2. setting phoenix.stats.useCurrentTime to false
3. dropping all data from SYSTEM.STATS for my user tables

Just as my previous resolution of increasing
phoenix.coprocessor.maxMetaDataCacheSize, I fear these steps are merely
kicking the can down the road. As our user load increases, so too will our
data size and thus more regions and an increasing size of the metadata
cache.

If this is indeed related to user-managed timestamps, as appears to be the
case for both Arun and myself, it means it will also bite users using the
new transactions feature in 4.7. Given the popularity of this new feature,
I believe it's critical we identify a resolution.

I think, as a minimum, we should move the critical section of refreshing
SYSTEM.CATALOG and SYSTEM.STATS out of the regionserver coprocessor
rowlock. Frankly, it's unacceptable to be making RPC calls under such
circumstances. A background thread that refreshes these caches on some
interval is more appropriate. If we require writes be interrupted in order
to accept online schema changes, I propose a ZK watcher/notification as an
alternative design choice.

I'm happy to provide more context, details, logs, jstacks, &c as needed.
Looking forward to a timely resolution.

Thanks,
Nick

On Wed, Mar 23, 2016 at 9:20 AM, Thangamani, Arun <Ar...@cdk.com>
wrote:

> Hey Nick, at least as far as PHOENIX-2607 is concerned, traveling back
> in time to insert data is the fundamental cause of the issue; that is, even
> after we insert the correct data in cache, we ignore whats in the cache
> next time around, and start rebuilding every time. This is by design and
> implementation of timestamps.
> I haven’t had the chance to completely check how UPDATE_CACHE_FREQUENCY
> works yet (James Suggestion) , I am hoping to check that in the next few
> weeks and close out PHOENIX-2607
>
> But in your situation, why are we rebuilding often or not finding meta
> data in cache?, there are few suspects I can think off (from the phoenix
> source code)
> 1) multi tenancy
> 2) guava library version, maybe accidentally a older version is getting
> pulled in at runtime
> 3) client/stat/table timestamp – whatever is getting build for metadata
> cache has timestamp that are different than what we are expecting?
> 4) the cache implementation by itself has a bug getting triggered by your
> use case
>
> I used the same table definition as my prod table, created a local
> instance of phoenix and attached a debugger to see why we needed the
> constant rebuild of meta data.
>
> Sorry, I wish I could help more, but if you can share your table
> definition, I can keep an eye in the next few weeks when I play with
> PHOENIX-2607.
>
> Thanks
> -Arun
>
> From: Nick Dimiduk <nd...@gmail.com>
> Date: Friday, March 18, 2016 at 11:32 AM
> To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
> Cc: James Taylor <ja...@apache.org>, Lars Hofhansl <la...@apache.org>,
> "Thangamani, Arun" <Ar...@cdk.com>
>
> Subject: Re: Write path blocked by MetaDataEndpoint acquiring region lock
>
> Spinning back around here, it seems my configuration change helped, but
> hasn't solved the problem. Jobs are no longer dying from RPC timeouts but I
> still see significant RPC latency spikes associated with SYSTEM.CATALOG.
> Hopefully I can make time to investigate further next week.
>
> @Arun did you gain any more insight into these symptoms on your side?
>
> On Mon, Feb 29, 2016 at 5:03 PM, Nick Dimiduk <nd...@gmail.com> wrote:
>
>> Is 1000 a good default?
>>>
>>
>> I'm sure it depends a lot on one's workload.
>>
>> I added some debug logging around the metaDataCache and and acquisition
>> of the rowlock. Checking into the one host with excessive RPC call time, I
>> do indeed see MetaDataEndpointImpl logging cache evictions happening
>> frequently. Looks like the estimatedSize of the stats for one of my tables
>> is pushing 11mb and another table is not far behind. I bumped the value
>> of phoenix.coprocessor.maxMetaDataCacheSize to 100mb, will let that soak
>> for a couple days.
>>
>> Let's get in some extra debug logging folks can enable to see what's
>> going on in there; there's currently no visibility (stats or logging)
>> around this cache. Maybe stats would be better? Better still would be a
>> cache that can dynamically resize to accommodate increasing table (stats)
>> sizes and/or increasing number of tables. I also wonder if it's worth
>> pinning SYSTEM.CATALOG and SYSTEM.STATS to the same host, and
>> short-circuiting the RPC call between them. At the very least let's cache
>> the HBase Connection object so we're not recreating it with each stats
>> table lookup.
>>
>> Thanks,
>> Nick
>>
>> On Wed, Feb 17, 2016 at 9:42 AM, Nick Dimiduk <nd...@gmail.com> wrote:
>>>
>>>> Thanks for the context Arun.
>>>>
>>>> For what it's worth, I greatly increased the batch size (from default
>>>> 1,000 to 500,000), which i believe reduced contention on the lock and
>>>> allowed ingest to catch up.
>>>>
>>>> On Tue, Feb 16, 2016 at 9:14 PM, Thangamani, Arun <
>>>> Arun.Thangamani@cdk.com> wrote:
>>>>
>>>>> Sorry I had pressed Control + Enter a little earlier than I wanted to,
>>>>> corrections inline. Thanks
>>>>>
>>>>> From: "Thangamani, Arun" <Ar...@cdk.com>
>>>>> Reply-To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>>>> Date: Tuesday, February 16, 2016 at 8:38 PM
>>>>> To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>>>> Cc: James Taylor <ja...@apache.org>, Lars Hofhansl <
>>>>> larsh@apache.org>
>>>>> Subject: Re: Write path blocked by MetaDataEndpoint acquiring region
>>>>> lock
>>>>>
>>>>> Hey Nick,
>>>>>
>>>>> Looks like you are failing to find your table in meta data cache, if
>>>>> you don’t find it in the meta data cache, we end up rebuilding the metadata
>>>>> from both the SYSTEM.CATALOG and SYSTEM.STATS tables.
>>>>> The rebuilding process for the meta data is a scan on both the tables.
>>>>>
>>>>> So, we will end up going to zookeeper to find the region and execute
>>>>> the scan on the region, it is an expensive operation, that explains the
>>>>> calls to a specific region server and zookeeper
>>>>>
>>>>> Similar thing happens in PHOENIX-2607, but that is specifiically
>>>>> related to timestamps, if the client timestamp is less than or equal to the
>>>>> stats timestamp of the table, we will end up doing the above rebuilding
>>>>> process repeatedly for every batch of insert from PhoenixMapReduceUtil
>>>>> (Batch size default is 1000 rows). I don’t believe you have a timestamp
>>>>> issue, but looks like you have the same repeated lookups and related scans
>>>>> for rebuilding that happen in PHOENIX-2607
>>>>>
>>>>> James has suggested a workaround for the meta data cache refresh using UPDATE_CACHE_FREQUENCY
>>>>> variable while defining the table, and it will probably help  (we are
>>>>> trying it out for timestamp issue)
>>>>>
>>>>> Hope this helps.
>>>>>
>>>>> Thanks
>>>>> Arun
>>>>>
>>>>> Please look at the following lines in MetadataCacheImpl
>>>>> private PTable doGetTable(byte[] key, long clientTimeStamp, RowLock
>>>>> rowLock) throws IOException, SQLException {
>>>>>
>>>>>     ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
>>>>>     Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
>>>>>             GlobalCache.getInstance(this.env).getMetaDataCache();
>>>>>     PTable table = (PTable)metaDataCache.getIfPresent(cacheKey);
>>>>>     // We only cache the latest, so we'll end up building the table with every call if the
>>>>>     // client connection has specified an SCN.
>>>>>     // TODO: If we indicate to the client that we're returning an older version, but there's a
>>>>>     // newer version available, the client
>>>>>     // can safely not call this, since we only allow modifications to the latest.
>>>>>     if (table != null && table.getTimeStamp() < clientTimeStamp) {
>>>>>         // Table on client is up-to-date with table on server, so just return
>>>>>         if (isTableDeleted(table)) {
>>>>>             return null;
>>>>>         }
>>>>>         return table;
>>>>>     }
>>>>>     // Ask Lars about the expense of this call - if we don't take the lock, we still won't get
>>>>>     // partial results
>>>>>     // get the co-processor environment
>>>>>     // TODO: check that key is within region.getStartKey() and region.getEndKey()
>>>>>     // and return special code to force client to lookup region from meta.
>>>>>     Region region = env.getRegion();
>>>>>     /*
>>>>>      * Lock directly on key, though it may be an index table. This will just prevent a table
>>>>>      * from getting rebuilt too often.
>>>>>      */
>>>>>     final boolean wasLocked = (rowLock != null);
>>>>>     if (!wasLocked) {
>>>>>         rowLock = region.getRowLock(key, true);
>>>>>         if (rowLock == null) {
>>>>>             throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
>>>>>         }
>>>>>     }
>>>>>     try {
>>>>>         // Try cache again in case we were waiting on a lock
>>>>>         table = (PTable)metaDataCache.getIfPresent(cacheKey);
>>>>>         // We only cache the latest, so we'll end up building the table with every call if the
>>>>>         // client connection has specified an SCN.
>>>>>         // TODO: If we indicate to the client that we're returning an older version, but there's
>>>>>         // a newer version available, the client
>>>>>         // can safely not call this, since we only allow modifications to the latest.
>>>>>         if (table != null && table.getTimeStamp() < clientTimeStamp) {
>>>>>             // Table on client is up-to-date with table on server, so just return
>>>>>             if (isTableDeleted(table)) {
>>>>>                 return null;
>>>>>             }
>>>>>             return table;
>>>>>         }
>>>>>         // Query for the latest table first, since it's not cached
>>>>>         table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP);
>>>>>         if (table != null && table.getTimeStamp() < clientTimeStamp) {
>>>>>             return table;
>>>>>         }
>>>>>         // Otherwise, query for an older version of the table - it won't be cached
>>>>>         return buildTable(key, cacheKey, region, clientTimeStamp);
>>>>>     } finally {
>>>>>         if (!wasLocked) rowLock.release();
>>>>>     }
>>>>> }
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> From: Nick Dimiduk <nd...@gmail.com>
>>>>> Reply-To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>>>> Date: Tuesday, February 16, 2016 at 5:58 PM
>>>>> To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>>>> Cc: James Taylor <ja...@apache.org>, Lars Hofhansl <
>>>>> larsh@apache.org>
>>>>> Subject: Write path blocked by MetaDataEndpoint acquiring region lock
>>>>>
>>>>> Hello,
>>>>>
>>>>> I have a high throughput ingest pipeline that's seised up. My ingest
>>>>> application ultimately crashes, contains the following stack trace [0].
>>>>>
>>>>> Independently, I noticed that the RPC call time of one of the machines
>>>>> was significantly higher than others (95pct at multiple seconds vs 10's of
>>>>> ms). I grabbed the RS log and a couple jstacks from the process. In the
>>>>> logs I see handler threads creating ZK connections excessively (~50 INFO
>>>>> lines per second). The jstacks show handler threads parked while taking
>>>>> region row locks, calling HRegion.getRowLockInternal() via
>>>>> MetaDataEndpointImpl.doGetTable() [1]. The one handler thread I see that's
>>>>> in the same MetaDataEndpointImpl area but not under lock appears to be
>>>>> making an RPC to read the statistics table [2].
>>>>>
>>>>> I believe these two occurrences are related.
>>>>>
>>>>> My working theory is that the metaDataCache object is performing
>>>>> poorly for some reason. This results in excessive meta data lookups, some
>>>>> of which appear to require making an RPC call while under lock.
>>>>>
>>>>> What can I do to relive pressure on this rowlock? Looking at the code
>>>>> around the lock, this looks like it's populating to a connection-level
>>>>> cache of schema metadata. The host with the high RPC call time is hosting
>>>>> the SYSTEM.CATALOG table region. I see some configuration settings that may
>>>>> be related (ie, phoenix.coprocessor.maxMetaDataCacheSize), but I see no way
>>>>> to get debug information about cache size or evictions from this Guava
>>>>> cache instance.
>>>>>
>>>>> I'll be digging into this further, but I appreciate any pointers you
>>>>> may have.
>>>>>
>>>>> BTW, this is Phoenix 4.6.0 + HBase 1.1.2.
>>>>>
>>>>> Thanks a lot,
>>>>> -n
>>>>>
>>>>> [0]: client-side stack
>>>>> Caused by: org.apache.phoenix.exception.PhoenixIOException:
>>>>> Interrupted calling coprocessor service
>>>>> org.apache.phoenix.coprocessor.generated.MetaDataProtos$MetaDataService for
>>>>> row \x00<schema>\x00<user_table>
>>>>>         at
>>>>> org.apache.phoenix.util.ServerUtil.parseServerException(ServerUtil.java:108)
>>>>>         at
>>>>> org.apache.phoenix.query.ConnectionQueryServicesImpl.metaDataCoprocessorExec(ConnectionQueryServicesImpl.java:1053)
>>>>>         at
>>>>> org.apache.phoenix.query.ConnectionQueryServicesImpl.metaDataCoprocessorExec(ConnectionQueryServicesImpl.java:1016)
>>>>>         at
>>>>> org.apache.phoenix.query.ConnectionQueryServicesImpl.getTable(ConnectionQueryServicesImpl.java:1289)
>>>>>         at
>>>>> org.apache.phoenix.schema.MetaDataClient.updateCache(MetaDataClient.java:446)
>>>>>         at
>>>>> org.apache.phoenix.schema.MetaDataClient.updateCache(MetaDataClient.java:389)
>>>>>         at
>>>>> org.apache.phoenix.schema.MetaDataClient.updateCache(MetaDataClient.java:385)
>>>>>         at
>>>>> org.apache.phoenix.execute.MutationState.validate(MutationState.java:369)
>>>>>         at
>>>>> org.apache.phoenix.execute.MutationState.commit(MutationState.java:417)
>>>>>         at
>>>>> org.apache.phoenix.jdbc.PhoenixConnection$3.call(PhoenixConnection.java:482)
>>>>>         at
>>>>> org.apache.phoenix.jdbc.PhoenixConnection$3.call(PhoenixConnection.java:479)
>>>>>         at org.apache.phoenix.call.CallRunner.run(CallRunner.java:53)
>>>>>         at
>>>>> org.apache.phoenix.jdbc.PhoenixConnection.commit(PhoenixConnection.java:479)
>>>>>         at
>>>>> org.apache.phoenix.mapreduce.PhoenixRecordWriter.write(PhoenixRecordWriter.java:84)
>>>>>
>>>>> [1] rs handlers blocked stack
>>>>> "B.defaultRpcServer.handler=48,queue=3,port=16020" #91 daemon prio=5
>>>>> os_prio=0 tid=0x00007ff4ce458000 nid=0xca0a waiting on condition
>>>>> [0x00007ff47a607000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>> - parking to wait for  <0x0000000708ef59c0> (a
>>>>> java.util.concurrent.CountDownLatch$Sync)
>>>>> at
>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>>>>> at
>>>>> java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037)
>>>>> at
>>>>> java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328)
>>>>> at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277)
>>>>> at
>>>>> org.apache.hadoop.hbase.regionserver.HRegion.getRowLockInternal(HRegion.java:5047)
>>>>> at
>>>>> org.apache.hadoop.hbase.regionserver.HRegion.getRowLock(HRegion.java:5013)
>>>>> at
>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2397)
>>>>> at
>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2365)
>>>>> at
>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.getTable(MetaDataEndpointImpl.java:440)
>>>>> at
>>>>> org.apache.phoenix.coprocessor.generated.MetaDataProtos$MetaDataService.callMethod(MetaDataProtos.java:11609)
>>>>> at
>>>>> org.apache.hadoop.hbase.regionserver.HRegion.execService(HRegion.java:7435)
>>>>> at
>>>>> org.apache.hadoop.hbase.regionserver.RSRpcServices.execServiceOnRegion(RSRpcServices.java:1875)
>>>>> at
>>>>> org.apache.hadoop.hbase.regionserver.RSRpcServices.execService(RSRpcServices.java:1857)
>>>>> at
>>>>> org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32209)
>>>>> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2114)
>>>>> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:101)
>>>>> at
>>>>> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:130)
>>>>> at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:107)
>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>
>>>>> [2]: rs handler stats rpc stack
>>>>> "B.defaultRpcServer.handler=19,queue=4,port=16020" #62 daemon prio=5
>>>>> os_prio=0 tid=0x00007ff4ce420000 nid=0xc9ea in Object.wait()
>>>>> [0x00007ff47c323000]
>>>>>    java.lang.Thread.State: TIMED_WAITING (on object monitor)
>>>>> at java.lang.Object.wait(Native Method)
>>>>> at java.lang.Object.wait(Object.java:460)
>>>>> at java.util.concurrent.TimeUnit.timedWait(TimeUnit.java:348)
>>>>> at
>>>>> org.apache.hadoop.hbase.client.ResultBoundedCompletionService.poll(ResultBoundedCompletionService.java:155)
>>>>> - locked <0x00000007a3f5e030> (a
>>>>> [Lorg.apache.hadoop.hbase.client.ResultBoundedCompletionService$QueueingFuture;)
>>>>> at
>>>>> org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:168)
>>>>> at
>>>>> org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:59)
>>>>> at
>>>>> org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:200)
>>>>> at
>>>>> org.apache.hadoop.hbase.client.ClientScanner.call(ClientScanner.java:320)
>>>>> at
>>>>> org.apache.hadoop.hbase.client.ClientScanner.loadCache(ClientScanner.java:403)
>>>>> at
>>>>> org.apache.hadoop.hbase.client.ClientScanner.next(ClientScanner.java:364)
>>>>> at
>>>>> org.apache.phoenix.schema.stats.StatisticsUtil.readStatistics(StatisticsUtil.java:99)
>>>>> at
>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.getTable(MetaDataEndpointImpl.java:836)
>>>>> at
>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.buildTable(MetaDataEndpointImpl.java:472)
>>>>> at
>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2418)
>>>>> at
>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2365)
>>>>> at
>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.getTable(MetaDataEndpointImpl.java:440)
>>>>> at
>>>>> org.apache.phoenix.coprocessor.generated.MetaDataProtos$MetaDataService.callMethod(MetaDataProtos.java:11609)
>>>>> at
>>>>> org.apache.hadoop.hbase.regionserver.HRegion.execService(HRegion.java:7435)
>>>>> at
>>>>> org.apache.hadoop.hbase.regionserver.RSRpcServices.execServiceOnRegion(RSRpcServices.java:1875)
>>>>> at
>>>>> org.apache.hadoop.hbase.regionserver.RSRpcServices.execService(RSRpcServices.java:1857)
>>>>> at
>>>>> org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32209)
>>>>> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2114)
>>>>> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:101)
>>>>> at
>>>>> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:130)
>>>>> at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:107)
>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>
>>>>> ------------------------------
>>>>> This message and any attachments are intended only for the use of the
>>>>> addressee and may contain information that is privileged and confidential.
>>>>> If the reader of the message is not the intended recipient or an authorized
>>>>> representative of the intended recipient, you are hereby notified that any
>>>>> dissemination of this communication is strictly prohibited. If you have
>>>>> received this communication in error, notify the sender immediately by
>>>>> return email and delete the message and any attachments from your system.
>>>>>
>>>>
>>>>
>>>
>>>
>>> --
>>> Best regards,
>>>
>>>    - Andy
>>>
>>> Problems worthy of attack prove their worth by hitting back. - Piet Hein
>>> (via Tom White)
>>>
>>
>>
>

Re: Write path blocked by MetaDataEndpoint acquiring region lock

Posted by Nick Dimiduk <nd...@gmail.com>.
Hi James,

I'm not sure that tweaking UPDATE_CACHE_FREQUENCY is the right solution.
Certainly it's not a complete solution. After spending some time reading
code, I've opened a couple JIRAs related to this thread.

https://issues.apache.org/jira/browse/PHOENIX-2939
https://issues.apache.org/jira/browse/PHOENIX-2940
https://issues.apache.org/jira/browse/PHOENIX-2941

I think I've happened into a couple of unrelated problems that interacted
badly amongst themselves.

Thanks a lot,
Nick

On Mon, May 9, 2016 at 3:05 PM, James Taylor <ja...@apache.org> wrote:

> bq. But, using UPDATE_CACHE_FREQUENCY restricts the usage of a
> changing-schema table in production.
>
> You can still alter tables in production. It's just that a client on a
> different JVM won't see the schema changes until their cache expires. If
> only schema additions are being performed, we can do even better
> (see PHOENIX-2885).
>
> Does this meet your needs, Arun & Nick?
>
> Thanks,
> James
>
> On Mon, May 9, 2016 at 1:02 PM, Thangamani, Arun <Ar...@cdk.com>
> wrote:
>
>> Sorry, I haven’t had the chance to test the UPDATE_CACHE_FREQUENCY
>> parameter yet, we are behind on phoenix versions through our vendor.
>> So, I am a little restricted in testing this out. But, will find a way to
>> test soon.
>>
>> Overall though, I do believe supporting client controlled timestamp
>> upserts with random timestamps (going forward or backward in time) would be
>> a fundamental use case.
>>
>> Right now, theoretically, the only way we can support this - without
>> running into the row lock issue — is by defining UPDATE_CACHE_FREQUENCY.
>> But, using UPDATE_CACHE_FREQUENCY restricts the usage of a
>> changing-schema table in production.
>>
>> In my case, the table didn’t change much, so I might be ok for now like
>> James pointed out, but we can use a new JIRA to address this issue in the
>> long run.
>>
>> Thanks
>> Arun
>>
>> From: James Taylor <ja...@apache.org>
>> Reply-To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>> Date: Monday, May 9, 2016 at 10:18 AM
>> To: user <us...@phoenix.apache.org>
>> Subject: Re: Write path blocked by MetaDataEndpoint acquiring region lock
>>
>> On Mon, May 9, 2016 at 9:52 AM, Nick Dimiduk <nd...@apache.org> wrote:
>>
>>> On Mon, May 9, 2016 at 12:06 AM, James Taylor <ja...@apache.org>
>>> wrote:
>>>
>>>> Have you tried using the UPDATE_CACHE_FREQUENCY property [1] I
>>>> mentioned before?
>>>>
>>>
>>> I am still running 4.6, so this flag is not available to me at the
>>> moment. It is unacceptable to disable updates to this cache; I do
>>> infrequently add columns to my tables. This does not change my position on
>>> the cache update happening under rowlock.
>>>
>>
>> I'm hoping that the UPDATE_CACHE_FREQUENCY property will provide a quick,
>> practical solution to the problem. We're, of course, always open to patches
>> and contributions.
>>
>>
>>>
>>> Would be good to file a JIRA if you haven't already, and continue the
>>>> discussion there.
>>>>
>>>
>>> Arun already filed PHOENIX-2607. Do you believe his issue is different
>>> from what I'm experiencing?
>>>
>>
>> Based on Arun's comment[1], it sounds like phoenix.stats.useCurrentTime
>> and UPDATE_CACHE_FREQUENCY solved his immediate issue. If there are bigger,
>> architectural changes you'd like to discuss/contribute, let's open a new
>> JIRA.
>>
>> Thanks,
>> James
>>
>> [1]
>> https://issues.apache.org/jira/browse/PHOENIX-2607?focusedCommentId=15146319&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15146319
>> <https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_PHOENIX-2D2607-3FfocusedCommentId-3D15146319-26page-3Dcom.atlassian.jira.plugin.system.issuetabpanels-3Acomment-2Dtabpanel-23comment-2D15146319&d=DQMFaQ&c=N13-TaG7c-EYAiUNohBk74oLRjUiBTwVm-KSnr4bPSc&r=8g2kPpY-h3f5UtWTI1wWrWsWv9dLqY7DoaD5gi4GbNk&m=VcOngVZBlhLdrPlPtPTAKbgJG8JkbuqBBHznBeGalSY&s=-kBM8n6j3QOqV-t8yKNfLiR-2IvkO9EO1YNNyFnpDgk&e=>
>>
>> ------------------------------
>> This message and any attachments are intended only for the use of the
>> addressee and may contain information that is privileged and confidential.
>> If the reader of the message is not the intended recipient or an authorized
>> representative of the intended recipient, you are hereby notified that any
>> dissemination of this communication is strictly prohibited. If you have
>> received this communication in error, notify the sender immediately by
>> return email and delete the message and any attachments from your system.
>>
>
>

Re: Write path blocked by MetaDataEndpoint acquiring region lock

Posted by James Taylor <ja...@apache.org>.
bq. But, using UPDATE_CACHE_FREQUENCY restricts the usage of a
changing-schema table in production.

You can still alter tables in production. It's just that a client on a
different JVM won't see the schema changes until their cache expires. If
only schema additions are being performed, we can do even better
(see PHOENIX-2885).

Does this meet your needs, Arun & Nick?

Thanks,
James

On Mon, May 9, 2016 at 1:02 PM, Thangamani, Arun <Ar...@cdk.com>
wrote:

> Sorry, I haven’t had the chance to test the UPDATE_CACHE_FREQUENCY
> parameter yet, we are behind on phoenix versions through our vendor.
> So, I am a little restricted in testing this out. But, will find a way to
> test soon.
>
> Overall though, I do believe supporting client controlled timestamp
> upserts with random timestamps (going forward or backward in time) would be
> a fundamental use case.
>
> Right now, theoretically, the only way we can support this - without
> running into the row lock issue — is by defining UPDATE_CACHE_FREQUENCY.
> But, using UPDATE_CACHE_FREQUENCY restricts the usage of a changing-schema
> table in production.
>
> In my case, the table didn’t change much, so I might be ok for now like
> James pointed out, but we can use a new JIRA to address this issue in the
> long run.
>
> Thanks
> Arun
>
> From: James Taylor <ja...@apache.org>
> Reply-To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
> Date: Monday, May 9, 2016 at 10:18 AM
> To: user <us...@phoenix.apache.org>
> Subject: Re: Write path blocked by MetaDataEndpoint acquiring region lock
>
> On Mon, May 9, 2016 at 9:52 AM, Nick Dimiduk <nd...@apache.org> wrote:
>
>> On Mon, May 9, 2016 at 12:06 AM, James Taylor <ja...@apache.org>
>> wrote:
>>
>>> Have you tried using the UPDATE_CACHE_FREQUENCY property [1] I mentioned
>>> before?
>>>
>>
>> I am still running 4.6, so this flag is not available to me at the
>> moment. It is unacceptable to disable updates to this cache; I do
>> infrequently add columns to my tables. This does not change my position on
>> the cache update happening under rowlock.
>>
>
> I'm hoping that the UPDATE_CACHE_FREQUENCY property will provide a quick,
> practical solution to the problem. We're, of course, always open to patches
> and contributions.
>
>
>>
>> Would be good to file a JIRA if you haven't already, and continue the
>>> discussion there.
>>>
>>
>> Arun already filed PHOENIX-2607. Do you believe his issue is different
>> from what I'm experiencing?
>>
>
> Based on Arun's comment[1], it sounds like phoenix.stats.useCurrentTime
> and UPDATE_CACHE_FREQUENCY solved his immediate issue. If there are bigger,
> architectural changes you'd like to discuss/contribute, let's open a new
> JIRA.
>
> Thanks,
> James
>
> [1]
> https://issues.apache.org/jira/browse/PHOENIX-2607?focusedCommentId=15146319&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15146319
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_PHOENIX-2D2607-3FfocusedCommentId-3D15146319-26page-3Dcom.atlassian.jira.plugin.system.issuetabpanels-3Acomment-2Dtabpanel-23comment-2D15146319&d=DQMFaQ&c=N13-TaG7c-EYAiUNohBk74oLRjUiBTwVm-KSnr4bPSc&r=8g2kPpY-h3f5UtWTI1wWrWsWv9dLqY7DoaD5gi4GbNk&m=VcOngVZBlhLdrPlPtPTAKbgJG8JkbuqBBHznBeGalSY&s=-kBM8n6j3QOqV-t8yKNfLiR-2IvkO9EO1YNNyFnpDgk&e=>
>
> ------------------------------
> This message and any attachments are intended only for the use of the
> addressee and may contain information that is privileged and confidential.
> If the reader of the message is not the intended recipient or an authorized
> representative of the intended recipient, you are hereby notified that any
> dissemination of this communication is strictly prohibited. If you have
> received this communication in error, notify the sender immediately by
> return email and delete the message and any attachments from your system.
>

Re: Write path blocked by MetaDataEndpoint acquiring region lock

Posted by "Thangamani, Arun" <Ar...@cdk.com>.
Sorry, I haven’t had the chance to test the UPDATE_CACHE_FREQUENCY parameter yet, we are behind on phoenix versions through our vendor.
So, I am a little restricted in testing this out. But, will find a way to test soon.

Overall though, I do believe supporting client controlled timestamp upserts with random timestamps (going forward or backward in time) would be a fundamental use case.

Right now, theoretically, the only way we can support this - without running into the row lock issue — is by defining UPDATE_CACHE_FREQUENCY.
But, using UPDATE_CACHE_FREQUENCY restricts the usage of a changing-schema table in production.

In my case, the table didn’t change much, so I might be ok for now like James pointed out, but we can use a new JIRA to address this issue in the long run.

Thanks
Arun

From: James Taylor <ja...@apache.org>>
Reply-To: "user@phoenix.apache.org<ma...@phoenix.apache.org>" <us...@phoenix.apache.org>>
Date: Monday, May 9, 2016 at 10:18 AM
To: user <us...@phoenix.apache.org>>
Subject: Re: Write path blocked by MetaDataEndpoint acquiring region lock

On Mon, May 9, 2016 at 9:52 AM, Nick Dimiduk <nd...@apache.org>> wrote:
On Mon, May 9, 2016 at 12:06 AM, James Taylor <ja...@apache.org>> wrote:
Have you tried using the UPDATE_CACHE_FREQUENCY property [1] I mentioned before?

I am still running 4.6, so this flag is not available to me at the moment. It is unacceptable to disable updates to this cache; I do infrequently add columns to my tables. This does not change my position on the cache update happening under rowlock.

I'm hoping that the UPDATE_CACHE_FREQUENCY property will provide a quick, practical solution to the problem. We're, of course, always open to patches and contributions.


Would be good to file a JIRA if you haven't already, and continue the discussion there.

Arun already filed PHOENIX-2607. Do you believe his issue is different from what I'm experiencing?

Based on Arun's comment[1], it sounds like phoenix.stats.useCurrentTime and UPDATE_CACHE_FREQUENCY solved his immediate issue. If there are bigger, architectural changes you'd like to discuss/contribute, let's open a new JIRA.

Thanks,
James

[1] https://issues.apache.org/jira/browse/PHOENIX-2607?focusedCommentId=15146319&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15146319<https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_PHOENIX-2D2607-3FfocusedCommentId-3D15146319-26page-3Dcom.atlassian.jira.plugin.system.issuetabpanels-3Acomment-2Dtabpanel-23comment-2D15146319&d=DQMFaQ&c=N13-TaG7c-EYAiUNohBk74oLRjUiBTwVm-KSnr4bPSc&r=8g2kPpY-h3f5UtWTI1wWrWsWv9dLqY7DoaD5gi4GbNk&m=VcOngVZBlhLdrPlPtPTAKbgJG8JkbuqBBHznBeGalSY&s=-kBM8n6j3QOqV-t8yKNfLiR-2IvkO9EO1YNNyFnpDgk&e=>

----------------------------------------------------------------------
This message and any attachments are intended only for the use of the addressee and may contain information that is privileged and confidential. If the reader of the message is not the intended recipient or an authorized representative of the intended recipient, you are hereby notified that any dissemination of this communication is strictly prohibited. If you have received this communication in error, notify the sender immediately by return email and delete the message and any attachments from your system.

Re: Write path blocked by MetaDataEndpoint acquiring region lock

Posted by James Taylor <ja...@apache.org>.
On Mon, May 9, 2016 at 9:52 AM, Nick Dimiduk <nd...@apache.org> wrote:

> On Mon, May 9, 2016 at 12:06 AM, James Taylor <ja...@apache.org>
> wrote:
>
>> Have you tried using the UPDATE_CACHE_FREQUENCY property [1] I mentioned
>> before?
>>
>
> I am still running 4.6, so this flag is not available to me at the moment.
> It is unacceptable to disable updates to this cache; I do infrequently add
> columns to my tables. This does not change my position on the cache update
> happening under rowlock.
>

I'm hoping that the UPDATE_CACHE_FREQUENCY property will provide a quick,
practical solution to the problem. We're, of course, always open to patches
and contributions.


>
> Would be good to file a JIRA if you haven't already, and continue the
>> discussion there.
>>
>
> Arun already filed PHOENIX-2607. Do you believe his issue is different
> from what I'm experiencing?
>

Based on Arun's comment[1], it sounds like phoenix.stats.useCurrentTime and
UPDATE_CACHE_FREQUENCY solved his immediate issue. If there are bigger,
architectural changes you'd like to discuss/contribute, let's open a new
JIRA.

Thanks,
James

[1]
https://issues.apache.org/jira/browse/PHOENIX-2607?focusedCommentId=15146319&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15146319

Re: Write path blocked by MetaDataEndpoint acquiring region lock

Posted by Nick Dimiduk <nd...@apache.org>.
On Mon, May 9, 2016 at 12:06 AM, James Taylor <ja...@apache.org>
wrote:

> Have you tried using the UPDATE_CACHE_FREQUENCY property [1] I mentioned
> before?
>

I am still running 4.6, so this flag is not available to me at the moment.
It is unacceptable to disable updates to this cache; I do infrequently add
columns to my tables. This does not change my position on the cache update
happening under rowlock.

Would be good to file a JIRA if you haven't already, and continue the
> discussion there.
>

Arun already filed PHOENIX-2607. Do you believe his issue is different from
what I'm experiencing?

On Sun, May 8, 2016 at 8:39 PM, Nick Dimiduk <nd...@apache.org> wrote:
>
>> Switching from user to dev, user to BCC.
>>
>> Hi fellas,
>>
>> I suffered an ingest outage in production this weekend with the symptoms
>> discussed here; I need to revive this thread.
>>
>> After speaking a bit offline with Arun, I believe my use of the
>> ROW_TIMESTAMP feature in combination with a modest total data volume is the
>> culprit. Prod was unblocked by performing the following actions, though I
>> must admit I don't know if one of these alone would have been enough:
>>
>> 1. increasing phoenix.stats.guidepost.width to 500mb
>> 2. setting phoenix.stats.useCurrentTime to false
>> 3. dropping all data from SYSTEM.STATS for my user tables
>>
>> Just as my previous resolution of increasing
>> phoenix.coprocessor.maxMetaDataCacheSize, I fear these steps are merely
>> kicking the can down the road. As our user load increases, so too will our
>> data size and thus more regions and an increasing size of the metadata
>> cache.
>>
>> If this is indeed related to user-managed timestamps, as appears to be
>> the case for both Arun and myself, it means it will also bite users using
>> the new transactions feature in 4.7. Given the popularity of this new
>> feature, I believe it's critical we identify a resolution.
>>
>> I think, as a minimum, we should move the critical section of refreshing
>> SYSTEM.CATALOG and SYSTEM.STATS out of the regionserver coprocessor
>> rowlock. Frankly, it's unacceptable to be making RPC calls under such
>> circumstances. A background thread that refreshes these caches on some
>> interval is more appropriate. If we require writes be interrupted in order
>> to accept online schema changes, I propose a ZK watcher/notification as an
>> alternative design choice.
>>
>> I'm happy to provide more context, details, logs, jstacks, &c as needed.
>> Looking forward to a timely resolution.
>>
>> Thanks,
>> Nick
>>
>> On Wed, Mar 23, 2016 at 9:20 AM, Thangamani, Arun <
>> Arun.Thangamani@cdk.com> wrote:
>>
>>> Hey Nick, at least as far as PHOENIX-2607 is concerned, traveling back
>>> in time to insert data is the fundamental cause of the issue; that is, even
>>> after we insert the correct data in cache, we ignore whats in the cache
>>> next time around, and start rebuilding every time. This is by design and
>>> implementation of timestamps.
>>> I haven’t had the chance to completely check how UPDATE_CACHE_FREQUENCY
>>> works yet (James Suggestion) , I am hoping to check that in the next few
>>> weeks and close out PHOENIX-2607
>>>
>>> But in your situation, why are we rebuilding often or not finding meta
>>> data in cache?, there are few suspects I can think off (from the phoenix
>>> source code)
>>> 1) multi tenancy
>>> 2) guava library version, maybe accidentally a older version is getting
>>> pulled in at runtime
>>> 3) client/stat/table timestamp – whatever is getting build for metadata
>>> cache has timestamp that are different than what we are expecting?
>>> 4) the cache implementation by itself has a bug getting triggered by
>>> your use case
>>>
>>> I used the same table definition as my prod table, created a local
>>> instance of phoenix and attached a debugger to see why we needed the
>>> constant rebuild of meta data.
>>>
>>> Sorry, I wish I could help more, but if you can share your table
>>> definition, I can keep an eye in the next few weeks when I play with
>>> PHOENIX-2607.
>>>
>>> Thanks
>>> -Arun
>>>
>>> From: Nick Dimiduk <nd...@gmail.com>
>>> Date: Friday, March 18, 2016 at 11:32 AM
>>> To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>> Cc: James Taylor <ja...@apache.org>, Lars Hofhansl <
>>> larsh@apache.org>, "Thangamani, Arun" <Ar...@cdk.com>
>>>
>>> Subject: Re: Write path blocked by MetaDataEndpoint acquiring region
>>> lock
>>>
>>> Spinning back around here, it seems my configuration change helped, but
>>> hasn't solved the problem. Jobs are no longer dying from RPC timeouts but I
>>> still see significant RPC latency spikes associated with SYSTEM.CATALOG.
>>> Hopefully I can make time to investigate further next week.
>>>
>>> @Arun did you gain any more insight into these symptoms on your side?
>>>
>>> On Mon, Feb 29, 2016 at 5:03 PM, Nick Dimiduk <nd...@gmail.com>
>>> wrote:
>>>
>>>> Is 1000 a good default?
>>>>>
>>>>
>>>> I'm sure it depends a lot on one's workload.
>>>>
>>>> I added some debug logging around the metaDataCache and and acquisition
>>>> of the rowlock. Checking into the one host with excessive RPC call time, I
>>>> do indeed see MetaDataEndpointImpl logging cache evictions happening
>>>> frequently. Looks like the estimatedSize of the stats for one of my tables
>>>> is pushing 11mb and another table is not far behind. I bumped the value
>>>> of phoenix.coprocessor.maxMetaDataCacheSize to 100mb, will let that soak
>>>> for a couple days.
>>>>
>>>> Let's get in some extra debug logging folks can enable to see what's
>>>> going on in there; there's currently no visibility (stats or logging)
>>>> around this cache. Maybe stats would be better? Better still would be a
>>>> cache that can dynamically resize to accommodate increasing table (stats)
>>>> sizes and/or increasing number of tables. I also wonder if it's worth
>>>> pinning SYSTEM.CATALOG and SYSTEM.STATS to the same host, and
>>>> short-circuiting the RPC call between them. At the very least let's cache
>>>> the HBase Connection object so we're not recreating it with each stats
>>>> table lookup.
>>>>
>>>> Thanks,
>>>> Nick
>>>>
>>>> On Wed, Feb 17, 2016 at 9:42 AM, Nick Dimiduk <nd...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> Thanks for the context Arun.
>>>>>>
>>>>>> For what it's worth, I greatly increased the batch size (from default
>>>>>> 1,000 to 500,000), which i believe reduced contention on the lock and
>>>>>> allowed ingest to catch up.
>>>>>>
>>>>>> On Tue, Feb 16, 2016 at 9:14 PM, Thangamani, Arun <
>>>>>> Arun.Thangamani@cdk.com> wrote:
>>>>>>
>>>>>>> Sorry I had pressed Control + Enter a little earlier than I wanted
>>>>>>> to, corrections inline. Thanks
>>>>>>>
>>>>>>> From: "Thangamani, Arun" <Ar...@cdk.com>
>>>>>>> Reply-To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>>>>>> Date: Tuesday, February 16, 2016 at 8:38 PM
>>>>>>> To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>>>>>> Cc: James Taylor <ja...@apache.org>, Lars Hofhansl <
>>>>>>> larsh@apache.org>
>>>>>>> Subject: Re: Write path blocked by MetaDataEndpoint acquiring
>>>>>>> region lock
>>>>>>>
>>>>>>> Hey Nick,
>>>>>>>
>>>>>>> Looks like you are failing to find your table in meta data cache, if
>>>>>>> you don’t find it in the meta data cache, we end up rebuilding the metadata
>>>>>>> from both the SYSTEM.CATALOG and SYSTEM.STATS tables.
>>>>>>> The rebuilding process for the meta data is a scan on both the
>>>>>>> tables.
>>>>>>>
>>>>>>> So, we will end up going to zookeeper to find the region and execute
>>>>>>> the scan on the region, it is an expensive operation, that explains the
>>>>>>> calls to a specific region server and zookeeper
>>>>>>>
>>>>>>> Similar thing happens in PHOENIX-2607, but that is specifiically
>>>>>>> related to timestamps, if the client timestamp is less than or equal to the
>>>>>>> stats timestamp of the table, we will end up doing the above rebuilding
>>>>>>> process repeatedly for every batch of insert from PhoenixMapReduceUtil
>>>>>>> (Batch size default is 1000 rows). I don’t believe you have a timestamp
>>>>>>> issue, but looks like you have the same repeated lookups and related scans
>>>>>>> for rebuilding that happen in PHOENIX-2607
>>>>>>>
>>>>>>> James has suggested a workaround for the meta data cache refresh
>>>>>>> using UPDATE_CACHE_FREQUENCY variable while defining the table, and
>>>>>>> it will probably help  (we are trying it out for timestamp issue)
>>>>>>>
>>>>>>> Hope this helps.
>>>>>>>
>>>>>>> Thanks
>>>>>>> Arun
>>>>>>>
>>>>>>> Please look at the following lines in MetadataCacheImpl
>>>>>>> private PTable doGetTable(byte[] key, long clientTimeStamp, RowLock
>>>>>>> rowLock) throws IOException, SQLException {
>>>>>>>
>>>>>>>     ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
>>>>>>>     Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
>>>>>>>             GlobalCache.getInstance(this.env).getMetaDataCache();
>>>>>>>     PTable table = (PTable)metaDataCache.getIfPresent(cacheKey);
>>>>>>>     // We only cache the latest, so we'll end up building the table with every call if the
>>>>>>>     // client connection has specified an SCN.
>>>>>>>     // TODO: If we indicate to the client that we're returning an older version, but there's a
>>>>>>>     // newer version available, the client
>>>>>>>     // can safely not call this, since we only allow modifications to the latest.
>>>>>>>     if (table != null && table.getTimeStamp() < clientTimeStamp) {
>>>>>>>         // Table on client is up-to-date with table on server, so just return
>>>>>>>         if (isTableDeleted(table)) {
>>>>>>>             return null;
>>>>>>>         }
>>>>>>>         return table;
>>>>>>>     }
>>>>>>>     // Ask Lars about the expense of this call - if we don't take the lock, we still won't get
>>>>>>>     // partial results
>>>>>>>     // get the co-processor environment
>>>>>>>     // TODO: check that key is within region.getStartKey() and region.getEndKey()
>>>>>>>     // and return special code to force client to lookup region from meta.
>>>>>>>     Region region = env.getRegion();
>>>>>>>     /*
>>>>>>>      * Lock directly on key, though it may be an index table. This will just prevent a table
>>>>>>>      * from getting rebuilt too often.
>>>>>>>      */
>>>>>>>     final boolean wasLocked = (rowLock != null);
>>>>>>>     if (!wasLocked) {
>>>>>>>         rowLock = region.getRowLock(key, true);
>>>>>>>         if (rowLock == null) {
>>>>>>>             throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
>>>>>>>         }
>>>>>>>     }
>>>>>>>     try {
>>>>>>>         // Try cache again in case we were waiting on a lock
>>>>>>>         table = (PTable)metaDataCache.getIfPresent(cacheKey);
>>>>>>>         // We only cache the latest, so we'll end up building the table with every call if the
>>>>>>>         // client connection has specified an SCN.
>>>>>>>         // TODO: If we indicate to the client that we're returning an older version, but there's
>>>>>>>         // a newer version available, the client
>>>>>>>         // can safely not call this, since we only allow modifications to the latest.
>>>>>>>         if (table != null && table.getTimeStamp() < clientTimeStamp) {
>>>>>>>             // Table on client is up-to-date with table on server, so just return
>>>>>>>             if (isTableDeleted(table)) {
>>>>>>>                 return null;
>>>>>>>             }
>>>>>>>             return table;
>>>>>>>         }
>>>>>>>         // Query for the latest table first, since it's not cached
>>>>>>>         table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP);
>>>>>>>         if (table != null && table.getTimeStamp() < clientTimeStamp) {
>>>>>>>             return table;
>>>>>>>         }
>>>>>>>         // Otherwise, query for an older version of the table - it won't be cached
>>>>>>>         return buildTable(key, cacheKey, region, clientTimeStamp);
>>>>>>>     } finally {
>>>>>>>         if (!wasLocked) rowLock.release();
>>>>>>>     }
>>>>>>> }
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> From: Nick Dimiduk <nd...@gmail.com>
>>>>>>> Reply-To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>>>>>> Date: Tuesday, February 16, 2016 at 5:58 PM
>>>>>>> To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>>>>>> Cc: James Taylor <ja...@apache.org>, Lars Hofhansl <
>>>>>>> larsh@apache.org>
>>>>>>> Subject: Write path blocked by MetaDataEndpoint acquiring region
>>>>>>> lock
>>>>>>>
>>>>>>> Hello,
>>>>>>>
>>>>>>> I have a high throughput ingest pipeline that's seised up. My ingest
>>>>>>> application ultimately crashes, contains the following stack trace [0].
>>>>>>>
>>>>>>> Independently, I noticed that the RPC call time of one of the
>>>>>>> machines was significantly higher than others (95pct at multiple seconds vs
>>>>>>> 10's of ms). I grabbed the RS log and a couple jstacks from the process. In
>>>>>>> the logs I see handler threads creating ZK connections excessively (~50
>>>>>>> INFO lines per second). The jstacks show handler threads parked while
>>>>>>> taking region row locks, calling HRegion.getRowLockInternal() via
>>>>>>> MetaDataEndpointImpl.doGetTable() [1]. The one handler thread I see that's
>>>>>>> in the same MetaDataEndpointImpl area but not under lock appears to be
>>>>>>> making an RPC to read the statistics table [2].
>>>>>>>
>>>>>>> I believe these two occurrences are related.
>>>>>>>
>>>>>>> My working theory is that the metaDataCache object is performing
>>>>>>> poorly for some reason. This results in excessive meta data lookups, some
>>>>>>> of which appear to require making an RPC call while under lock.
>>>>>>>
>>>>>>> What can I do to relive pressure on this rowlock? Looking at the
>>>>>>> code around the lock, this looks like it's populating to a connection-level
>>>>>>> cache of schema metadata. The host with the high RPC call time is hosting
>>>>>>> the SYSTEM.CATALOG table region. I see some configuration settings that may
>>>>>>> be related (ie, phoenix.coprocessor.maxMetaDataCacheSize), but I see no way
>>>>>>> to get debug information about cache size or evictions from this Guava
>>>>>>> cache instance.
>>>>>>>
>>>>>>> I'll be digging into this further, but I appreciate any pointers you
>>>>>>> may have.
>>>>>>>
>>>>>>> BTW, this is Phoenix 4.6.0 + HBase 1.1.2.
>>>>>>>
>>>>>>> Thanks a lot,
>>>>>>> -n
>>>>>>>
>>>>>>> [0]: client-side stack
>>>>>>> Caused by: org.apache.phoenix.exception.PhoenixIOException:
>>>>>>> Interrupted calling coprocessor service
>>>>>>> org.apache.phoenix.coprocessor.generated.MetaDataProtos$MetaDataService for
>>>>>>> row \x00<schema>\x00<user_table>
>>>>>>>         at
>>>>>>> org.apache.phoenix.util.ServerUtil.parseServerException(ServerUtil.java:108)
>>>>>>>         at
>>>>>>> org.apache.phoenix.query.ConnectionQueryServicesImpl.metaDataCoprocessorExec(ConnectionQueryServicesImpl.java:1053)
>>>>>>>         at
>>>>>>> org.apache.phoenix.query.ConnectionQueryServicesImpl.metaDataCoprocessorExec(ConnectionQueryServicesImpl.java:1016)
>>>>>>>         at
>>>>>>> org.apache.phoenix.query.ConnectionQueryServicesImpl.getTable(ConnectionQueryServicesImpl.java:1289)
>>>>>>>         at
>>>>>>> org.apache.phoenix.schema.MetaDataClient.updateCache(MetaDataClient.java:446)
>>>>>>>         at
>>>>>>> org.apache.phoenix.schema.MetaDataClient.updateCache(MetaDataClient.java:389)
>>>>>>>         at
>>>>>>> org.apache.phoenix.schema.MetaDataClient.updateCache(MetaDataClient.java:385)
>>>>>>>         at
>>>>>>> org.apache.phoenix.execute.MutationState.validate(MutationState.java:369)
>>>>>>>         at
>>>>>>> org.apache.phoenix.execute.MutationState.commit(MutationState.java:417)
>>>>>>>         at
>>>>>>> org.apache.phoenix.jdbc.PhoenixConnection$3.call(PhoenixConnection.java:482)
>>>>>>>         at
>>>>>>> org.apache.phoenix.jdbc.PhoenixConnection$3.call(PhoenixConnection.java:479)
>>>>>>>         at org.apache.phoenix.call.CallRunner.run(CallRunner.java:53)
>>>>>>>         at
>>>>>>> org.apache.phoenix.jdbc.PhoenixConnection.commit(PhoenixConnection.java:479)
>>>>>>>         at
>>>>>>> org.apache.phoenix.mapreduce.PhoenixRecordWriter.write(PhoenixRecordWriter.java:84)
>>>>>>>
>>>>>>> [1] rs handlers blocked stack
>>>>>>> "B.defaultRpcServer.handler=48,queue=3,port=16020" #91 daemon prio=5
>>>>>>> os_prio=0 tid=0x00007ff4ce458000 nid=0xca0a waiting on condition
>>>>>>> [0x00007ff47a607000]
>>>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>>>> - parking to wait for  <0x0000000708ef59c0> (a
>>>>>>> java.util.concurrent.CountDownLatch$Sync)
>>>>>>> at
>>>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>>>>>>> at
>>>>>>> java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037)
>>>>>>> at
>>>>>>> java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328)
>>>>>>> at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.getRowLockInternal(HRegion.java:5047)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.getRowLock(HRegion.java:5013)
>>>>>>> at
>>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2397)
>>>>>>> at
>>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2365)
>>>>>>> at
>>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.getTable(MetaDataEndpointImpl.java:440)
>>>>>>> at
>>>>>>> org.apache.phoenix.coprocessor.generated.MetaDataProtos$MetaDataService.callMethod(MetaDataProtos.java:11609)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.execService(HRegion.java:7435)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.regionserver.RSRpcServices.execServiceOnRegion(RSRpcServices.java:1875)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.regionserver.RSRpcServices.execService(RSRpcServices.java:1857)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32209)
>>>>>>> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2114)
>>>>>>> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:101)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:130)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:107)
>>>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>>>
>>>>>>> [2]: rs handler stats rpc stack
>>>>>>> "B.defaultRpcServer.handler=19,queue=4,port=16020" #62 daemon prio=5
>>>>>>> os_prio=0 tid=0x00007ff4ce420000 nid=0xc9ea in Object.wait()
>>>>>>> [0x00007ff47c323000]
>>>>>>>    java.lang.Thread.State: TIMED_WAITING (on object monitor)
>>>>>>> at java.lang.Object.wait(Native Method)
>>>>>>> at java.lang.Object.wait(Object.java:460)
>>>>>>> at java.util.concurrent.TimeUnit.timedWait(TimeUnit.java:348)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.client.ResultBoundedCompletionService.poll(ResultBoundedCompletionService.java:155)
>>>>>>> - locked <0x00000007a3f5e030> (a
>>>>>>> [Lorg.apache.hadoop.hbase.client.ResultBoundedCompletionService$QueueingFuture;)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:168)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:59)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:200)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.client.ClientScanner.call(ClientScanner.java:320)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.client.ClientScanner.loadCache(ClientScanner.java:403)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.client.ClientScanner.next(ClientScanner.java:364)
>>>>>>> at
>>>>>>> org.apache.phoenix.schema.stats.StatisticsUtil.readStatistics(StatisticsUtil.java:99)
>>>>>>> at
>>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.getTable(MetaDataEndpointImpl.java:836)
>>>>>>> at
>>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.buildTable(MetaDataEndpointImpl.java:472)
>>>>>>> at
>>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2418)
>>>>>>> at
>>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2365)
>>>>>>> at
>>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.getTable(MetaDataEndpointImpl.java:440)
>>>>>>> at
>>>>>>> org.apache.phoenix.coprocessor.generated.MetaDataProtos$MetaDataService.callMethod(MetaDataProtos.java:11609)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.execService(HRegion.java:7435)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.regionserver.RSRpcServices.execServiceOnRegion(RSRpcServices.java:1875)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.regionserver.RSRpcServices.execService(RSRpcServices.java:1857)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32209)
>>>>>>> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2114)
>>>>>>> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:101)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:130)
>>>>>>> at
>>>>>>> org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:107)
>>>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>>>
>>>>>>> ------------------------------
>>>>>>> This message and any attachments are intended only for the use of
>>>>>>> the addressee and may contain information that is privileged and
>>>>>>> confidential. If the reader of the message is not the intended recipient or
>>>>>>> an authorized representative of the intended recipient, you are hereby
>>>>>>> notified that any dissemination of this communication is strictly
>>>>>>> prohibited. If you have received this communication in error, notify the
>>>>>>> sender immediately by return email and delete the message and any
>>>>>>> attachments from your system.
>>>>>>>
>>>>>>
>>>>>>
>>>>>
>>>>>
>>>>> --
>>>>> Best regards,
>>>>>
>>>>>    - Andy
>>>>>
>>>>> Problems worthy of attack prove their worth by hitting back. - Piet
>>>>> Hein (via Tom White)
>>>>>
>>>>
>>>>
>>>
>>
>

Re: Write path blocked by MetaDataEndpoint acquiring region lock

Posted by James Taylor <ja...@apache.org>.
Nick,
Have you tried using the UPDATE_CACHE_FREQUENCY property [1] I mentioned
before?

Would be good to file a JIRA if you haven't already, and continue the
discussion there.

Thanks,
James

[1] https://phoenix.apache.org/#Altering

On Sun, May 8, 2016 at 8:39 PM, Nick Dimiduk <nd...@apache.org> wrote:

> Switching from user to dev, user to BCC.
>
> Hi fellas,
>
> I suffered an ingest outage in production this weekend with the symptoms
> discussed here; I need to revive this thread.
>
> After speaking a bit offline with Arun, I believe my use of the
> ROW_TIMESTAMP feature in combination with a modest total data volume is the
> culprit. Prod was unblocked by performing the following actions, though I
> must admit I don't know if one of these alone would have been enough:
>
> 1. increasing phoenix.stats.guidepost.width to 500mb
> 2. setting phoenix.stats.useCurrentTime to false
> 3. dropping all data from SYSTEM.STATS for my user tables
>
> Just as my previous resolution of increasing
> phoenix.coprocessor.maxMetaDataCacheSize, I fear these steps are merely
> kicking the can down the road. As our user load increases, so too will our
> data size and thus more regions and an increasing size of the metadata
> cache.
>
> If this is indeed related to user-managed timestamps, as appears to be the
> case for both Arun and myself, it means it will also bite users using the
> new transactions feature in 4.7. Given the popularity of this new feature,
> I believe it's critical we identify a resolution.
>
> I think, as a minimum, we should move the critical section of refreshing
> SYSTEM.CATALOG and SYSTEM.STATS out of the regionserver coprocessor
> rowlock. Frankly, it's unacceptable to be making RPC calls under such
> circumstances. A background thread that refreshes these caches on some
> interval is more appropriate. If we require writes be interrupted in order
> to accept online schema changes, I propose a ZK watcher/notification as an
> alternative design choice.
>
> I'm happy to provide more context, details, logs, jstacks, &c as needed.
> Looking forward to a timely resolution.
>
> Thanks,
> Nick
>
> On Wed, Mar 23, 2016 at 9:20 AM, Thangamani, Arun <Arun.Thangamani@cdk.com
> > wrote:
>
>> Hey Nick, at least as far as PHOENIX-2607 is concerned, traveling back
>> in time to insert data is the fundamental cause of the issue; that is, even
>> after we insert the correct data in cache, we ignore whats in the cache
>> next time around, and start rebuilding every time. This is by design and
>> implementation of timestamps.
>> I haven’t had the chance to completely check how UPDATE_CACHE_FREQUENCY
>> works yet (James Suggestion) , I am hoping to check that in the next few
>> weeks and close out PHOENIX-2607
>>
>> But in your situation, why are we rebuilding often or not finding meta
>> data in cache?, there are few suspects I can think off (from the phoenix
>> source code)
>> 1) multi tenancy
>> 2) guava library version, maybe accidentally a older version is getting
>> pulled in at runtime
>> 3) client/stat/table timestamp – whatever is getting build for metadata
>> cache has timestamp that are different than what we are expecting?
>> 4) the cache implementation by itself has a bug getting triggered by your
>> use case
>>
>> I used the same table definition as my prod table, created a local
>> instance of phoenix and attached a debugger to see why we needed the
>> constant rebuild of meta data.
>>
>> Sorry, I wish I could help more, but if you can share your table
>> definition, I can keep an eye in the next few weeks when I play with
>> PHOENIX-2607.
>>
>> Thanks
>> -Arun
>>
>> From: Nick Dimiduk <nd...@gmail.com>
>> Date: Friday, March 18, 2016 at 11:32 AM
>> To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>> Cc: James Taylor <ja...@apache.org>, Lars Hofhansl <
>> larsh@apache.org>, "Thangamani, Arun" <Ar...@cdk.com>
>>
>> Subject: Re: Write path blocked by MetaDataEndpoint acquiring region lock
>>
>> Spinning back around here, it seems my configuration change helped, but
>> hasn't solved the problem. Jobs are no longer dying from RPC timeouts but I
>> still see significant RPC latency spikes associated with SYSTEM.CATALOG.
>> Hopefully I can make time to investigate further next week.
>>
>> @Arun did you gain any more insight into these symptoms on your side?
>>
>> On Mon, Feb 29, 2016 at 5:03 PM, Nick Dimiduk <nd...@gmail.com> wrote:
>>
>>> Is 1000 a good default?
>>>>
>>>
>>> I'm sure it depends a lot on one's workload.
>>>
>>> I added some debug logging around the metaDataCache and and acquisition
>>> of the rowlock. Checking into the one host with excessive RPC call time, I
>>> do indeed see MetaDataEndpointImpl logging cache evictions happening
>>> frequently. Looks like the estimatedSize of the stats for one of my tables
>>> is pushing 11mb and another table is not far behind. I bumped the value
>>> of phoenix.coprocessor.maxMetaDataCacheSize to 100mb, will let that soak
>>> for a couple days.
>>>
>>> Let's get in some extra debug logging folks can enable to see what's
>>> going on in there; there's currently no visibility (stats or logging)
>>> around this cache. Maybe stats would be better? Better still would be a
>>> cache that can dynamically resize to accommodate increasing table (stats)
>>> sizes and/or increasing number of tables. I also wonder if it's worth
>>> pinning SYSTEM.CATALOG and SYSTEM.STATS to the same host, and
>>> short-circuiting the RPC call between them. At the very least let's cache
>>> the HBase Connection object so we're not recreating it with each stats
>>> table lookup.
>>>
>>> Thanks,
>>> Nick
>>>
>>> On Wed, Feb 17, 2016 at 9:42 AM, Nick Dimiduk <nd...@gmail.com>
>>>> wrote:
>>>>
>>>>> Thanks for the context Arun.
>>>>>
>>>>> For what it's worth, I greatly increased the batch size (from default
>>>>> 1,000 to 500,000), which i believe reduced contention on the lock and
>>>>> allowed ingest to catch up.
>>>>>
>>>>> On Tue, Feb 16, 2016 at 9:14 PM, Thangamani, Arun <
>>>>> Arun.Thangamani@cdk.com> wrote:
>>>>>
>>>>>> Sorry I had pressed Control + Enter a little earlier than I wanted
>>>>>> to, corrections inline. Thanks
>>>>>>
>>>>>> From: "Thangamani, Arun" <Ar...@cdk.com>
>>>>>> Reply-To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>>>>> Date: Tuesday, February 16, 2016 at 8:38 PM
>>>>>> To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>>>>> Cc: James Taylor <ja...@apache.org>, Lars Hofhansl <
>>>>>> larsh@apache.org>
>>>>>> Subject: Re: Write path blocked by MetaDataEndpoint acquiring region
>>>>>> lock
>>>>>>
>>>>>> Hey Nick,
>>>>>>
>>>>>> Looks like you are failing to find your table in meta data cache, if
>>>>>> you don’t find it in the meta data cache, we end up rebuilding the metadata
>>>>>> from both the SYSTEM.CATALOG and SYSTEM.STATS tables.
>>>>>> The rebuilding process for the meta data is a scan on both the
>>>>>> tables.
>>>>>>
>>>>>> So, we will end up going to zookeeper to find the region and execute
>>>>>> the scan on the region, it is an expensive operation, that explains the
>>>>>> calls to a specific region server and zookeeper
>>>>>>
>>>>>> Similar thing happens in PHOENIX-2607, but that is specifiically
>>>>>> related to timestamps, if the client timestamp is less than or equal to the
>>>>>> stats timestamp of the table, we will end up doing the above rebuilding
>>>>>> process repeatedly for every batch of insert from PhoenixMapReduceUtil
>>>>>> (Batch size default is 1000 rows). I don’t believe you have a timestamp
>>>>>> issue, but looks like you have the same repeated lookups and related scans
>>>>>> for rebuilding that happen in PHOENIX-2607
>>>>>>
>>>>>> James has suggested a workaround for the meta data cache refresh
>>>>>> using UPDATE_CACHE_FREQUENCY variable while defining the table, and
>>>>>> it will probably help  (we are trying it out for timestamp issue)
>>>>>>
>>>>>> Hope this helps.
>>>>>>
>>>>>> Thanks
>>>>>> Arun
>>>>>>
>>>>>> Please look at the following lines in MetadataCacheImpl
>>>>>> private PTable doGetTable(byte[] key, long clientTimeStamp, RowLock
>>>>>> rowLock) throws IOException, SQLException {
>>>>>>
>>>>>>     ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
>>>>>>     Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
>>>>>>             GlobalCache.getInstance(this.env).getMetaDataCache();
>>>>>>     PTable table = (PTable)metaDataCache.getIfPresent(cacheKey);
>>>>>>     // We only cache the latest, so we'll end up building the table with every call if the
>>>>>>     // client connection has specified an SCN.
>>>>>>     // TODO: If we indicate to the client that we're returning an older version, but there's a
>>>>>>     // newer version available, the client
>>>>>>     // can safely not call this, since we only allow modifications to the latest.
>>>>>>     if (table != null && table.getTimeStamp() < clientTimeStamp) {
>>>>>>         // Table on client is up-to-date with table on server, so just return
>>>>>>         if (isTableDeleted(table)) {
>>>>>>             return null;
>>>>>>         }
>>>>>>         return table;
>>>>>>     }
>>>>>>     // Ask Lars about the expense of this call - if we don't take the lock, we still won't get
>>>>>>     // partial results
>>>>>>     // get the co-processor environment
>>>>>>     // TODO: check that key is within region.getStartKey() and region.getEndKey()
>>>>>>     // and return special code to force client to lookup region from meta.
>>>>>>     Region region = env.getRegion();
>>>>>>     /*
>>>>>>      * Lock directly on key, though it may be an index table. This will just prevent a table
>>>>>>      * from getting rebuilt too often.
>>>>>>      */
>>>>>>     final boolean wasLocked = (rowLock != null);
>>>>>>     if (!wasLocked) {
>>>>>>         rowLock = region.getRowLock(key, true);
>>>>>>         if (rowLock == null) {
>>>>>>             throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
>>>>>>         }
>>>>>>     }
>>>>>>     try {
>>>>>>         // Try cache again in case we were waiting on a lock
>>>>>>         table = (PTable)metaDataCache.getIfPresent(cacheKey);
>>>>>>         // We only cache the latest, so we'll end up building the table with every call if the
>>>>>>         // client connection has specified an SCN.
>>>>>>         // TODO: If we indicate to the client that we're returning an older version, but there's
>>>>>>         // a newer version available, the client
>>>>>>         // can safely not call this, since we only allow modifications to the latest.
>>>>>>         if (table != null && table.getTimeStamp() < clientTimeStamp) {
>>>>>>             // Table on client is up-to-date with table on server, so just return
>>>>>>             if (isTableDeleted(table)) {
>>>>>>                 return null;
>>>>>>             }
>>>>>>             return table;
>>>>>>         }
>>>>>>         // Query for the latest table first, since it's not cached
>>>>>>         table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP);
>>>>>>         if (table != null && table.getTimeStamp() < clientTimeStamp) {
>>>>>>             return table;
>>>>>>         }
>>>>>>         // Otherwise, query for an older version of the table - it won't be cached
>>>>>>         return buildTable(key, cacheKey, region, clientTimeStamp);
>>>>>>     } finally {
>>>>>>         if (!wasLocked) rowLock.release();
>>>>>>     }
>>>>>> }
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> From: Nick Dimiduk <nd...@gmail.com>
>>>>>> Reply-To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>>>>> Date: Tuesday, February 16, 2016 at 5:58 PM
>>>>>> To: "user@phoenix.apache.org" <us...@phoenix.apache.org>
>>>>>> Cc: James Taylor <ja...@apache.org>, Lars Hofhansl <
>>>>>> larsh@apache.org>
>>>>>> Subject: Write path blocked by MetaDataEndpoint acquiring region lock
>>>>>>
>>>>>> Hello,
>>>>>>
>>>>>> I have a high throughput ingest pipeline that's seised up. My ingest
>>>>>> application ultimately crashes, contains the following stack trace [0].
>>>>>>
>>>>>> Independently, I noticed that the RPC call time of one of the
>>>>>> machines was significantly higher than others (95pct at multiple seconds vs
>>>>>> 10's of ms). I grabbed the RS log and a couple jstacks from the process. In
>>>>>> the logs I see handler threads creating ZK connections excessively (~50
>>>>>> INFO lines per second). The jstacks show handler threads parked while
>>>>>> taking region row locks, calling HRegion.getRowLockInternal() via
>>>>>> MetaDataEndpointImpl.doGetTable() [1]. The one handler thread I see that's
>>>>>> in the same MetaDataEndpointImpl area but not under lock appears to be
>>>>>> making an RPC to read the statistics table [2].
>>>>>>
>>>>>> I believe these two occurrences are related.
>>>>>>
>>>>>> My working theory is that the metaDataCache object is performing
>>>>>> poorly for some reason. This results in excessive meta data lookups, some
>>>>>> of which appear to require making an RPC call while under lock.
>>>>>>
>>>>>> What can I do to relive pressure on this rowlock? Looking at the code
>>>>>> around the lock, this looks like it's populating to a connection-level
>>>>>> cache of schema metadata. The host with the high RPC call time is hosting
>>>>>> the SYSTEM.CATALOG table region. I see some configuration settings that may
>>>>>> be related (ie, phoenix.coprocessor.maxMetaDataCacheSize), but I see no way
>>>>>> to get debug information about cache size or evictions from this Guava
>>>>>> cache instance.
>>>>>>
>>>>>> I'll be digging into this further, but I appreciate any pointers you
>>>>>> may have.
>>>>>>
>>>>>> BTW, this is Phoenix 4.6.0 + HBase 1.1.2.
>>>>>>
>>>>>> Thanks a lot,
>>>>>> -n
>>>>>>
>>>>>> [0]: client-side stack
>>>>>> Caused by: org.apache.phoenix.exception.PhoenixIOException:
>>>>>> Interrupted calling coprocessor service
>>>>>> org.apache.phoenix.coprocessor.generated.MetaDataProtos$MetaDataService for
>>>>>> row \x00<schema>\x00<user_table>
>>>>>>         at
>>>>>> org.apache.phoenix.util.ServerUtil.parseServerException(ServerUtil.java:108)
>>>>>>         at
>>>>>> org.apache.phoenix.query.ConnectionQueryServicesImpl.metaDataCoprocessorExec(ConnectionQueryServicesImpl.java:1053)
>>>>>>         at
>>>>>> org.apache.phoenix.query.ConnectionQueryServicesImpl.metaDataCoprocessorExec(ConnectionQueryServicesImpl.java:1016)
>>>>>>         at
>>>>>> org.apache.phoenix.query.ConnectionQueryServicesImpl.getTable(ConnectionQueryServicesImpl.java:1289)
>>>>>>         at
>>>>>> org.apache.phoenix.schema.MetaDataClient.updateCache(MetaDataClient.java:446)
>>>>>>         at
>>>>>> org.apache.phoenix.schema.MetaDataClient.updateCache(MetaDataClient.java:389)
>>>>>>         at
>>>>>> org.apache.phoenix.schema.MetaDataClient.updateCache(MetaDataClient.java:385)
>>>>>>         at
>>>>>> org.apache.phoenix.execute.MutationState.validate(MutationState.java:369)
>>>>>>         at
>>>>>> org.apache.phoenix.execute.MutationState.commit(MutationState.java:417)
>>>>>>         at
>>>>>> org.apache.phoenix.jdbc.PhoenixConnection$3.call(PhoenixConnection.java:482)
>>>>>>         at
>>>>>> org.apache.phoenix.jdbc.PhoenixConnection$3.call(PhoenixConnection.java:479)
>>>>>>         at org.apache.phoenix.call.CallRunner.run(CallRunner.java:53)
>>>>>>         at
>>>>>> org.apache.phoenix.jdbc.PhoenixConnection.commit(PhoenixConnection.java:479)
>>>>>>         at
>>>>>> org.apache.phoenix.mapreduce.PhoenixRecordWriter.write(PhoenixRecordWriter.java:84)
>>>>>>
>>>>>> [1] rs handlers blocked stack
>>>>>> "B.defaultRpcServer.handler=48,queue=3,port=16020" #91 daemon prio=5
>>>>>> os_prio=0 tid=0x00007ff4ce458000 nid=0xca0a waiting on condition
>>>>>> [0x00007ff47a607000]
>>>>>>    java.lang.Thread.State: TIMED_WAITING (parking)
>>>>>> at sun.misc.Unsafe.park(Native Method)
>>>>>> - parking to wait for  <0x0000000708ef59c0> (a
>>>>>> java.util.concurrent.CountDownLatch$Sync)
>>>>>> at
>>>>>> java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
>>>>>> at
>>>>>> java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037)
>>>>>> at
>>>>>> java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328)
>>>>>> at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.getRowLockInternal(HRegion.java:5047)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.getRowLock(HRegion.java:5013)
>>>>>> at
>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2397)
>>>>>> at
>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2365)
>>>>>> at
>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.getTable(MetaDataEndpointImpl.java:440)
>>>>>> at
>>>>>> org.apache.phoenix.coprocessor.generated.MetaDataProtos$MetaDataService.callMethod(MetaDataProtos.java:11609)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.execService(HRegion.java:7435)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.regionserver.RSRpcServices.execServiceOnRegion(RSRpcServices.java:1875)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.regionserver.RSRpcServices.execService(RSRpcServices.java:1857)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32209)
>>>>>> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2114)
>>>>>> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:101)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:130)
>>>>>> at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:107)
>>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>>
>>>>>> [2]: rs handler stats rpc stack
>>>>>> "B.defaultRpcServer.handler=19,queue=4,port=16020" #62 daemon prio=5
>>>>>> os_prio=0 tid=0x00007ff4ce420000 nid=0xc9ea in Object.wait()
>>>>>> [0x00007ff47c323000]
>>>>>>    java.lang.Thread.State: TIMED_WAITING (on object monitor)
>>>>>> at java.lang.Object.wait(Native Method)
>>>>>> at java.lang.Object.wait(Object.java:460)
>>>>>> at java.util.concurrent.TimeUnit.timedWait(TimeUnit.java:348)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.client.ResultBoundedCompletionService.poll(ResultBoundedCompletionService.java:155)
>>>>>> - locked <0x00000007a3f5e030> (a
>>>>>> [Lorg.apache.hadoop.hbase.client.ResultBoundedCompletionService$QueueingFuture;)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:168)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.client.ScannerCallableWithReplicas.call(ScannerCallableWithReplicas.java:59)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(RpcRetryingCaller.java:200)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.client.ClientScanner.call(ClientScanner.java:320)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.client.ClientScanner.loadCache(ClientScanner.java:403)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.client.ClientScanner.next(ClientScanner.java:364)
>>>>>> at
>>>>>> org.apache.phoenix.schema.stats.StatisticsUtil.readStatistics(StatisticsUtil.java:99)
>>>>>> at
>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.getTable(MetaDataEndpointImpl.java:836)
>>>>>> at
>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.buildTable(MetaDataEndpointImpl.java:472)
>>>>>> at
>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2418)
>>>>>> at
>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.doGetTable(MetaDataEndpointImpl.java:2365)
>>>>>> at
>>>>>> org.apache.phoenix.coprocessor.MetaDataEndpointImpl.getTable(MetaDataEndpointImpl.java:440)
>>>>>> at
>>>>>> org.apache.phoenix.coprocessor.generated.MetaDataProtos$MetaDataService.callMethod(MetaDataProtos.java:11609)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.regionserver.HRegion.execService(HRegion.java:7435)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.regionserver.RSRpcServices.execServiceOnRegion(RSRpcServices.java:1875)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.regionserver.RSRpcServices.execService(RSRpcServices.java:1857)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$2.callBlockingMethod(ClientProtos.java:32209)
>>>>>> at org.apache.hadoop.hbase.ipc.RpcServer.call(RpcServer.java:2114)
>>>>>> at org.apache.hadoop.hbase.ipc.CallRunner.run(CallRunner.java:101)
>>>>>> at
>>>>>> org.apache.hadoop.hbase.ipc.RpcExecutor.consumerLoop(RpcExecutor.java:130)
>>>>>> at org.apache.hadoop.hbase.ipc.RpcExecutor$1.run(RpcExecutor.java:107)
>>>>>> at java.lang.Thread.run(Thread.java:745)
>>>>>>
>>>>>> ------------------------------
>>>>>> This message and any attachments are intended only for the use of the
>>>>>> addressee and may contain information that is privileged and confidential.
>>>>>> If the reader of the message is not the intended recipient or an authorized
>>>>>> representative of the intended recipient, you are hereby notified that any
>>>>>> dissemination of this communication is strictly prohibited. If you have
>>>>>> received this communication in error, notify the sender immediately by
>>>>>> return email and delete the message and any attachments from your system.
>>>>>>
>>>>>
>>>>>
>>>>
>>>>
>>>> --
>>>> Best regards,
>>>>
>>>>    - Andy
>>>>
>>>> Problems worthy of attack prove their worth by hitting back. - Piet
>>>> Hein (via Tom White)
>>>>
>>>
>>>
>>
>