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 <[email protected]> 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 <[email protected] > > 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 <[email protected]> >> Date: Friday, March 18, 2016 at 11:32 AM >> To: "[email protected]" <[email protected]> >> Cc: James Taylor <[email protected]>, Lars Hofhansl < >> [email protected]>, "Thangamani, Arun" <[email protected]> >> >> 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 <[email protected]> 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 <[email protected]> >>>> 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 < >>>>> [email protected]> wrote: >>>>> >>>>>> Sorry I had pressed Control + Enter a little earlier than I wanted >>>>>> to, corrections inline. Thanks >>>>>> >>>>>> From: "Thangamani, Arun" <[email protected]> >>>>>> Reply-To: "[email protected]" <[email protected]> >>>>>> Date: Tuesday, February 16, 2016 at 8:38 PM >>>>>> To: "[email protected]" <[email protected]> >>>>>> Cc: James Taylor <[email protected]>, Lars Hofhansl < >>>>>> [email protected]> >>>>>> 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 <[email protected]> >>>>>> Reply-To: "[email protected]" <[email protected]> >>>>>> Date: Tuesday, February 16, 2016 at 5:58 PM >>>>>> To: "[email protected]" <[email protected]> >>>>>> Cc: James Taylor <[email protected]>, Lars Hofhansl < >>>>>> [email protected]> >>>>>> 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) >>>> >>> >>> >> >
