[ https://issues.apache.org/jira/browse/PHOENIX-3838?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Geoffrey Jacoby updated PHOENIX-3838: ------------------------------------- Description: >From [~mujtabachohan]: Meanwhile with HBase 1.3.1 if I try to split a table while data load is in progress, the table state remains in SPLITTING_NEW and index writer blocked. Table splits fine if there is no active writes happening to the table when split is requested. {code} Thread 163 (RpcServer.FifoWFPBQ.priority.handler=19,queue=1,port=48109): State: WAITING Blocked count: 100 Waited count: 463 Waiting on com.google.common.util.concurrent.AbstractFuture$Sync@16703eda Stack: sun.misc.Unsafe.park(Native Method) java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997) java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304) com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:275) com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:111) org.apache.phoenix.hbase.index.parallel.BaseTaskRunner.submit(BaseTaskRunner.java:66) org.apache.phoenix.hbase.index.parallel.BaseTaskRunner.submitUninterruptible(BaseTaskRunner.java:99) org.apache.phoenix.hbase.index.write.ParallelWriterIndexCommitter.write(ParallelWriterIndexCommitter.java:197) org.apache.phoenix.hbase.index.write.IndexWriter.write(IndexWriter.java:185) org.apache.phoenix.hbase.index.write.IndexWriter.writeAndKillYourselfOnFailure(IndexWriter.java:146) org.apache.phoenix.hbase.index.write.IndexWriter.writeAndKillYourselfOnFailure(IndexWriter.java:135) org.apache.phoenix.hbase.index.Indexer.doPostWithExceptions(Indexer.java:474) org.apache.phoenix.hbase.index.Indexer.doPost(Indexer.java:407) org.apache.phoenix.hbase.index.Indexer.postPut(Indexer.java:375) org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$32.call(RegionCoprocessorHost.java:956) org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$RegionOperation.call(RegionCoprocessorHost.java:1673) org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1749) org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1705) {code} Following schema was used with batch size of 1000 inserting data in background: {code} CREATE TABLE IF NOT EXISTS T (PKA CHAR(15) NOT NULL, PKF CHAR(3) NOT NULL, PKP CHAR(15) NOT NULL, CRD DATE NOT NULL, EHI CHAR(15) NOT NULL, FID CHAR(15), CREATED_BY_ID VARCHAR, FH VARCHAR, DT VARCHAR, OS VARCHAR, NS VARCHAR, OFN VARCHAR CONSTRAINT PK PRIMARY KEY ( PKA, PKF, PKP, CRD DESC, EHI )) VERSIONS=1,MULTI_TENANT=true,IMMUTABLE_ROWS=true; CREATE LOCAL INDEX IF NOT EXISTS TIDX ON T (PKF, CRD, PKP, EHI) INCLUDE (FID, CREATED_BY_ID, FH, DT, OS, NS, OFN); {code} was: >From [~mujtabachohan]: Meanwhile with HBase 1.3.1 if I try to split a table while data load is in progress, the table state remains in SPLITTING_NEW and index writer blocked. Table splits fine if there is no active writes happening to the table when split is requested. ``` Thread 163 (RpcServer.FifoWFPBQ.priority.handler=19,queue=1,port=48109): State: WAITING Blocked count: 100 Waited count: 463 Waiting on com.google.common.util.concurrent.AbstractFuture$Sync@16703eda Stack: sun.misc.Unsafe.park(Native Method) java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997) java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304) com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:275) com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:111) org.apache.phoenix.hbase.index.parallel.BaseTaskRunner.submit(BaseTaskRunner.java:66) org.apache.phoenix.hbase.index.parallel.BaseTaskRunner.submitUninterruptible(BaseTaskRunner.java:99) org.apache.phoenix.hbase.index.write.ParallelWriterIndexCommitter.write(ParallelWriterIndexCommitter.java:197) org.apache.phoenix.hbase.index.write.IndexWriter.write(IndexWriter.java:185) org.apache.phoenix.hbase.index.write.IndexWriter.writeAndKillYourselfOnFailure(IndexWriter.java:146) org.apache.phoenix.hbase.index.write.IndexWriter.writeAndKillYourselfOnFailure(IndexWriter.java:135) org.apache.phoenix.hbase.index.Indexer.doPostWithExceptions(Indexer.java:474) org.apache.phoenix.hbase.index.Indexer.doPost(Indexer.java:407) org.apache.phoenix.hbase.index.Indexer.postPut(Indexer.java:375) org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$32.call(RegionCoprocessorHost.java:956) org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$RegionOperation.call(RegionCoprocessorHost.java:1673) org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1749) org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1705) ``` Following schema was used with batch size of 1000 inserting data in background: ``` CREATE TABLE IF NOT EXISTS T (PKA CHAR(15) NOT NULL, PKF CHAR(3) NOT NULL, PKP CHAR(15) NOT NULL, CRD DATE NOT NULL, EHI CHAR(15) NOT NULL, FID CHAR(15), CREATED_BY_ID VARCHAR, FH VARCHAR, DT VARCHAR, OS VARCHAR, NS VARCHAR, OFN VARCHAR CONSTRAINT PK PRIMARY KEY ( PKA, PKF, PKP, CRD DESC, EHI )) VERSIONS=1,MULTI_TENANT=true,IMMUTABLE_ROWS=true; CREATE LOCAL INDEX IF NOT EXISTS TIDX ON T (PKF, CRD, PKP, EHI) INCLUDE (FID, CREATED_BY_ID, FH, DT, OS, NS, OFN); ``` > Region splits can hang on SPLITTING_NEW when local index is present > ------------------------------------------------------------------- > > Key: PHOENIX-3838 > URL: https://issues.apache.org/jira/browse/PHOENIX-3838 > Project: Phoenix > Issue Type: Bug > Affects Versions: 4.10.0 > Reporter: Geoffrey Jacoby > Assignee: Geoffrey Jacoby > Priority: Blocker > > From [~mujtabachohan]: > Meanwhile with HBase 1.3.1 if I try to split a table while data load is in > progress, the table state remains in SPLITTING_NEW and index writer blocked. > Table splits fine if there is no active writes happening to the table when > split is requested. > {code} > Thread 163 (RpcServer.FifoWFPBQ.priority.handler=19,queue=1,port=48109): > State: WAITING > Blocked count: 100 > Waited count: 463 > Waiting on com.google.common.util.concurrent.AbstractFuture$Sync@16703eda > Stack: > sun.misc.Unsafe.park(Native Method) > java.util.concurrent.locks.LockSupport.park(LockSupport.java:175) > > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836) > > java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997) > > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304) > > com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:275) > > com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:111) > > org.apache.phoenix.hbase.index.parallel.BaseTaskRunner.submit(BaseTaskRunner.java:66) > > org.apache.phoenix.hbase.index.parallel.BaseTaskRunner.submitUninterruptible(BaseTaskRunner.java:99) > > org.apache.phoenix.hbase.index.write.ParallelWriterIndexCommitter.write(ParallelWriterIndexCommitter.java:197) > > org.apache.phoenix.hbase.index.write.IndexWriter.write(IndexWriter.java:185) > > org.apache.phoenix.hbase.index.write.IndexWriter.writeAndKillYourselfOnFailure(IndexWriter.java:146) > > org.apache.phoenix.hbase.index.write.IndexWriter.writeAndKillYourselfOnFailure(IndexWriter.java:135) > > org.apache.phoenix.hbase.index.Indexer.doPostWithExceptions(Indexer.java:474) > org.apache.phoenix.hbase.index.Indexer.doPost(Indexer.java:407) > org.apache.phoenix.hbase.index.Indexer.postPut(Indexer.java:375) > > org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$32.call(RegionCoprocessorHost.java:956) > > org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost$RegionOperation.call(RegionCoprocessorHost.java:1673) > > org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1749) > > org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost.execOperation(RegionCoprocessorHost.java:1705) > {code} > Following schema was used with batch size of 1000 inserting data in > background: > {code} > CREATE TABLE IF NOT EXISTS T (PKA CHAR(15) NOT NULL, PKF CHAR(3) NOT NULL, > PKP CHAR(15) NOT NULL, CRD DATE NOT NULL, EHI CHAR(15) NOT NULL, FID > CHAR(15), CREATED_BY_ID VARCHAR, > FH VARCHAR, DT VARCHAR, OS VARCHAR, NS VARCHAR, OFN VARCHAR CONSTRAINT PK > PRIMARY KEY ( PKA, PKF, PKP, CRD DESC, EHI )) > VERSIONS=1,MULTI_TENANT=true,IMMUTABLE_ROWS=true; > CREATE LOCAL INDEX IF NOT EXISTS TIDX ON T (PKF, CRD, PKP, EHI) > INCLUDE (FID, CREATED_BY_ID, FH, DT, OS, NS, OFN); > {code} -- This message was sent by Atlassian JIRA (v6.3.15#6346)