You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by "Duo Zhang (JIRA)" <ji...@apache.org> on 2018/01/13 11:59:00 UTC

[jira] [Commented] (HBASE-19792) TestReplicationSmallTests.testDisableEnable fails

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

Duo Zhang commented on HBASE-19792:
-----------------------------------

Copy one of the failed output here. Seems there are missing WAL files.
{noformat}
Error Message
Waited too much time for put replication
Stacktrace
java.lang.AssertionError: Waited too much time for put replication
	at org.apache.hadoop.hbase.replication.TestReplicationSmallTests.testDisableEnable(TestReplicationSmallTests.java:365)
Standard Output
2018-01-13 07:00:13,855 INFO  [main] hbase.HBaseZKTestingUtility(86): Created new mini-cluster data directory: /home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39, deleteOnExit=true
2018-01-13 07:00:14,004 ERROR [main] server.ZooKeeperServer(472): ZKShutdownHandler is not registered, so ZooKeeper server won't take any action on ERROR or SHUTDOWN server state changes
2018-01-13 07:00:14,022 INFO  [main] zookeeper.MiniZooKeeperCluster(281): Started MiniZooKeeperCluster and ran successful 'stat' on client port=65105
2018-01-13 07:00:14,050 INFO  [main] zookeeper.RecoverableZooKeeper(106): Process identifier=cluster1 connecting to ZooKeeper ensemble=localhost:65105
2018-01-13 07:00:14,129 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): cluster10x0, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=None, state=SyncConnected, path=null
2018-01-13 07:00:14,130 DEBUG [main-EventThread] zookeeper.ZKWatcher(523): cluster1-0x160ee519be80000 connected
2018-01-13 07:00:14,251 WARN  [main] util.NativeCodeLoader(62): Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
2018-01-13 07:00:14,615 INFO  [main] zookeeper.ReadOnlyZKClient(130): Start read only zookeeper connection 0x128d2484 to localhost:65105, session timeout 90000 ms, retries 1, retry interval 10 ms, keep alive 60000 ms
2018-01-13 07:00:14,642 WARN  [main] client.ConnectionImplementation(528): Retrieve cluster id failed
java.util.concurrent.ExecutionException: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /1/hbaseid
	at java.util.concurrent.CompletableFuture.reportGet(CompletableFuture.java:357)
	at java.util.concurrent.CompletableFuture.get(CompletableFuture.java:1895)
	at org.apache.hadoop.hbase.client.ConnectionImplementation.retrieveClusterId(ConnectionImplementation.java:526)
	at org.apache.hadoop.hbase.client.ConnectionImplementation.<init>(ConnectionImplementation.java:286)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
	at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
	at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
	at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
	at org.apache.hadoop.hbase.client.ConnectionFactory.createConnection(ConnectionFactory.java:219)
	at org.apache.hadoop.hbase.client.ConnectionFactory.createConnection(ConnectionFactory.java:114)
	at org.apache.hadoop.hbase.client.replication.ReplicationAdmin.<init>(ReplicationAdmin.java:96)
	at org.apache.hadoop.hbase.replication.TestReplicationBase.setUpBeforeClass(TestReplicationBase.java:132)
	at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
	at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
	at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
	at java.lang.reflect.Method.invoke(Method.java:498)
	at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50)
	at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12)
	at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47)
	at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:24)
	at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27)
	at org.junit.runners.ParentRunner.run(ParentRunner.java:363)
	at org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:369)
	at org.apache.maven.surefire.junit4.JUnit4Provider.executeWithRerun(JUnit4Provider.java:275)
	at org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:239)
	at org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:160)
	at org.apache.maven.surefire.booter.ForkedBooter.invokeProviderInSameClassLoader(ForkedBooter.java:373)
	at org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:334)
	at org.apache.maven.surefire.booter.ForkedBooter.execute(ForkedBooter.java:119)
	at org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:407)
Caused by: org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /1/hbaseid
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
	at org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient$ZKTask$1.exec(ReadOnlyZKClient.java:164)
	at org.apache.hadoop.hbase.zookeeper.ReadOnlyZKClient.run(ReadOnlyZKClient.java:321)
	at java.lang.Thread.run(Thread.java:748)
2018-01-13 07:00:14,647 DEBUG [main] client.ConnectionImplementation(532): clusterid came back null, using default default-cluster
2018-01-13 07:00:14,713 DEBUG [main] util.ClassSize(229): Using Unsafe to estimate memory layout
2018-01-13 07:00:14,718 DEBUG [main] ipc.AbstractRpcClient(200): Codec=org.apache.hadoop.hbase.codec.KeyValueCodec@10aa41f2, compressor=null, tcpKeepAlive=true, tcpNoDelay=true, connectTO=10000, readTO=20000, writeTO=60000, minIdleTimeBeforeClose=120000, maxRetries=0, fallbackAllowed=true, bind address=null
2018-01-13 07:00:14,762 INFO  [main] replication.TestReplicationBase(133): Setup first Zk
2018-01-13 07:00:14,788 INFO  [main] zookeeper.RecoverableZooKeeper(106): Process identifier=cluster2 connecting to ZooKeeper ensemble=localhost:65105
2018-01-13 07:00:14,802 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): cluster20x0, quorum=localhost:65105, baseZNode=/2 Received ZooKeeper Event, type=None, state=SyncConnected, path=null
2018-01-13 07:00:14,802 DEBUG [main-EventThread] zookeeper.ZKWatcher(523): cluster2-0x160ee519be80002 connected
2018-01-13 07:00:14,869 INFO  [main] replication.TestReplicationBase(144): Setup second Zk
2018-01-13 07:00:14,891 INFO  [main] hbase.HBaseTestingUtility(945): Starting up minicluster with 1 master(s) and 2 regionserver(s) and 2 datanode(s)
2018-01-13 07:00:14,891 INFO  [main] hbase.HBaseTestingUtility(960): STARTING DFS
2018-01-13 07:00:14,892 INFO  [main] hbase.HBaseTestingUtility(737): Setting test.cache.data to /home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cache_data in system properties and HBase conf
2018-01-13 07:00:14,892 INFO  [main] hbase.HBaseTestingUtility(737): Setting hadoop.tmp.dir to /home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/hadoop_tmp in system properties and HBase conf
2018-01-13 07:00:14,893 INFO  [main] hbase.HBaseTestingUtility(737): Setting hadoop.log.dir to /home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/hadoop_logs in system properties and HBase conf
2018-01-13 07:00:14,893 INFO  [main] hbase.HBaseTestingUtility(737): Setting mapreduce.cluster.local.dir to /home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/mapred_local in system properties and HBase conf
2018-01-13 07:00:14,894 INFO  [main] hbase.HBaseTestingUtility(737): Setting mapreduce.cluster.temp.dir to /home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/mapred_temp in system properties and HBase conf
2018-01-13 07:00:14,894 INFO  [main] hbase.HBaseTestingUtility(728): read short circuit is OFF
2018-01-13 07:00:15,143 DEBUG [main] fs.HFileSystem(318): The file system is not a DistributedFileSystem. Skipping on block location reordering
Formatting using clusterid: testClusterID
2018-01-13 07:00:16,499 WARN  [main] impl.MetricsConfig(125): Cannot locate configuration: tried hadoop-metrics2-namenode.properties,hadoop-metrics2.properties
2018-01-13 07:00:16,679 INFO  [main] log.Slf4jLog(67): Logging to org.slf4j.impl.Log4jLoggerAdapter(org.mortbay.log) via org.mortbay.log.Slf4jLog
2018-01-13 07:00:16,744 INFO  [main] log.Slf4jLog(67): jetty-6.1.26
2018-01-13 07:00:16,782 INFO  [main] log.Slf4jLog(67): Extract jar:file:/home/jenkins/yetus-m2/hbase-flaky-tests/org/apache/hadoop/hadoop-hdfs/2.7.4/hadoop-hdfs-2.7.4-tests.jar!/webapps/hdfs to /tmp/Jetty_localhost_34782_hdfs____.mru38m/webapp
2018-01-13 07:00:16,976 INFO  [main] log.Slf4jLog(67): Started HttpServer2$SelectChannelConnectorWithSafeStartup@localhost:34782
2018-01-13 07:00:17,913 INFO  [main] log.Slf4jLog(67): jetty-6.1.26
2018-01-13 07:00:17,920 INFO  [main] log.Slf4jLog(67): Extract jar:file:/home/jenkins/yetus-m2/hbase-flaky-tests/org/apache/hadoop/hadoop-hdfs/2.7.4/hadoop-hdfs-2.7.4-tests.jar!/webapps/datanode to /tmp/Jetty_localhost_52048_datanode____hehfmm/webapp
2018-01-13 07:00:18,076 INFO  [main] log.Slf4jLog(67): Started HttpServer2$SelectChannelConnectorWithSafeStartup@localhost:52048
2018-01-13 07:00:18,742 INFO  [main] log.Slf4jLog(67): jetty-6.1.26
2018-01-13 07:00:18,749 INFO  [main] log.Slf4jLog(67): Extract jar:file:/home/jenkins/yetus-m2/hbase-flaky-tests/org/apache/hadoop/hadoop-hdfs/2.7.4/hadoop-hdfs-2.7.4-tests.jar!/webapps/datanode to /tmp/Jetty_localhost_58057_datanode____y9j54m/webapp
2018-01-13 07:00:18,951 INFO  [main] log.Slf4jLog(67): Started HttpServer2$SelectChannelConnectorWithSafeStartup@localhost:58057
2018-01-13 07:00:19,848 ERROR [DataNode: [[[DISK]file:/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39/dfs/data/data3/, [DISK]file:/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39/dfs/data/data4/]]  heartbeating to localhost/127.0.0.1:44892] datanode.DirectoryScanner(430): dfs.datanode.directoryscan.throttle.limit.ms.per.sec set to value below 1 ms/sec. Assuming default value of 1000
2018-01-13 07:00:19,848 ERROR [DataNode: [[[DISK]file:/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39/dfs/data/data1/, [DISK]file:/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39/dfs/data/data2/]]  heartbeating to localhost/127.0.0.1:44892] datanode.DirectoryScanner(430): dfs.datanode.directoryscan.throttle.limit.ms.per.sec set to value below 1 ms/sec. Assuming default value of 1000
2018-01-13 07:00:19,931 INFO  [Block report processor] blockmanagement.BlockManager(1933): BLOCK* processReport 0x136f4605b74d3f: from storage DS-2d1c80f9-c237-446d-b413-0d899e38665b node DatanodeRegistration(127.0.0.1:34940, datanodeUuid=76f1583d-69ee-4f07-a1e5-7cf1c5409e62, infoPort=60029, infoSecurePort=0, ipcPort=34172, storageInfo=lv=-56;cid=testClusterID;nsid=2101341033;c=0), blocks: 0, hasStaleStorage: true, processing time: 3 msecs
2018-01-13 07:00:19,932 INFO  [Block report processor] blockmanagement.BlockManager(1933): BLOCK* processReport 0x136f4605b759a5: from storage DS-51167812-25c5-42dd-9759-7b939eb0ef2b node DatanodeRegistration(127.0.0.1:36231, datanodeUuid=fa37f8ae-4063-4419-b02f-823a76a4c0da, infoPort=42408, infoSecurePort=0, ipcPort=54082, storageInfo=lv=-56;cid=testClusterID;nsid=2101341033;c=0), blocks: 0, hasStaleStorage: true, processing time: 1 msecs
2018-01-13 07:00:19,932 INFO  [Block report processor] blockmanagement.BlockManager(1933): BLOCK* processReport 0x136f4605b74d3f: from storage DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e node DatanodeRegistration(127.0.0.1:34940, datanodeUuid=76f1583d-69ee-4f07-a1e5-7cf1c5409e62, infoPort=60029, infoSecurePort=0, ipcPort=34172, storageInfo=lv=-56;cid=testClusterID;nsid=2101341033;c=0), blocks: 0, hasStaleStorage: false, processing time: 0 msecs
2018-01-13 07:00:19,932 INFO  [Block report processor] blockmanagement.BlockManager(1933): BLOCK* processReport 0x136f4605b759a5: from storage DS-f5839f39-099d-419b-8c6f-32dec519a9b9 node DatanodeRegistration(127.0.0.1:36231, datanodeUuid=fa37f8ae-4063-4419-b02f-823a76a4c0da, infoPort=42408, infoSecurePort=0, ipcPort=54082, storageInfo=lv=-56;cid=testClusterID;nsid=2101341033;c=0), blocks: 0, hasStaleStorage: false, processing time: 0 msecs
2018-01-13 07:00:21,063 INFO  [main] fs.HFileSystem(349): Added intercepting call to namenode#getBlockLocations so can do block reordering using class org.apache.hadoop.hbase.fs.HFileSystem$ReorderWALBlocks
2018-01-13 07:00:21,066 INFO  [main] fs.HFileSystem(349): Added intercepting call to namenode#getBlockLocations so can do block reordering using class org.apache.hadoop.hbase.fs.HFileSystem$ReorderWALBlocks
2018-01-13 07:00:21,459 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:34940 is added to blk_1073741825_1001{UCState=COMMITTED, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e:NORMAL:127.0.0.1:34940|RBW], ReplicaUC[[DISK]DS-51167812-25c5-42dd-9759-7b939eb0ef2b:NORMAL:127.0.0.1:36231|RBW]]} size 7
2018-01-13 07:00:21,460 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:36231 is added to blk_1073741825_1001 size 7
2018-01-13 07:00:21,879 INFO  [main] util.FSUtils(513): Created version file at hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00 with version=8
2018-01-13 07:00:21,879 INFO  [main] hbase.HBaseTestingUtility(1204): Setting hbase.fs.tmp.dir to hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/hbase-staging
2018-01-13 07:00:22,021 INFO  [main] metrics.MetricRegistriesLoader(66): Loaded MetricRegistries class org.apache.hadoop.hbase.metrics.impl.MetricRegistriesImpl
2018-01-13 07:00:22,277 INFO  [main] client.ConnectionUtils(122): master/asf911.gq1.ygridcore.net/67.195.81.155:0 server-side Connection retries=45
2018-01-13 07:00:22,300 INFO  [main] ipc.RpcExecutor(148): RpcExecutor default.FPBQ using fifo as call queue; numCallQueues=1; maxQueueLength=50; handlerCount=5
2018-01-13 07:00:22,300 INFO  [main] ipc.RpcExecutor(148): RpcExecutor priority.FPBQ using fifo as call queue; numCallQueues=1; maxQueueLength=50; handlerCount=6
2018-01-13 07:00:22,300 INFO  [main] ipc.RpcExecutor(148): RpcExecutor replication.FPBQ using fifo as call queue; numCallQueues=1; maxQueueLength=50; handlerCount=3
2018-01-13 07:00:22,453 INFO  [main] ipc.RpcServerFactory(65): Creating org.apache.hadoop.hbase.ipc.NettyRpcServer hosting hbase.pb.MasterService, hbase.pb.RegionServerStatusService, hbase.pb.LockService, hbase.pb.ClientService, hbase.pb.AdminService
2018-01-13 07:00:23,002 INFO  [main] ipc.NettyRpcServer(109): NettyRpcServer bind to address=/67.195.81.155:53894
2018-01-13 07:00:23,006 INFO  [main] hfile.CacheConfig(553): Allocating On heap LruBlockCache size=995.60 MB, blockSize=64 KB
2018-01-13 07:00:23,015 INFO  [main] hfile.CacheConfig(262): Created cacheConfig: blockCache=LruBlockCache{blockCount=0, currentSize=765640, freeSize=1043196664, maxSize=1043962304, heapSize=765640, minSize=991764160, minFactor=0.95, multiSize=495882080, multiFactor=0.5, singleSize=247941040, singleFactor=0.25}, cacheDataOnRead=true, cacheDataOnWrite=false, cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false, cacheDataCompressed=false, prefetchOnOpen=false
2018-01-13 07:00:23,015 INFO  [main] hfile.CacheConfig(262): Created cacheConfig: blockCache=LruBlockCache{blockCount=0, currentSize=765640, freeSize=1043196664, maxSize=1043962304, heapSize=765640, minSize=991764160, minFactor=0.95, multiSize=495882080, multiFactor=0.5, singleSize=247941040, singleFactor=0.25}, cacheDataOnRead=true, cacheDataOnWrite=false, cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false, cacheDataCompressed=false, prefetchOnOpen=false
2018-01-13 07:00:23,019 DEBUG [main] mob.MobFileCache(123): MobFileCache enabled with cacheSize=1000, evictPeriods=3600sec, evictRemainRatio=0.5
2018-01-13 07:00:23,021 INFO  [main] fs.HFileSystem(349): Added intercepting call to namenode#getBlockLocations so can do block reordering using class org.apache.hadoop.hbase.fs.HFileSystem$ReorderWALBlocks
2018-01-13 07:00:23,025 INFO  [main] fs.HFileSystem(349): Added intercepting call to namenode#getBlockLocations so can do block reordering using class org.apache.hadoop.hbase.fs.HFileSystem$ReorderWALBlocks
2018-01-13 07:00:23,061 INFO  [main] zookeeper.RecoverableZooKeeper(106): Process identifier=master:53894 connecting to ZooKeeper ensemble=localhost:65105
2018-01-13 07:00:23,077 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): master:538940x0, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=None, state=SyncConnected, path=null
2018-01-13 07:00:23,077 DEBUG [main-EventThread] zookeeper.ZKWatcher(523): master:53894-0x160ee519be80003 connected
2018-01-13 07:00:23,085 DEBUG [main] zookeeper.RecoverableZooKeeper(571): Node /1 already exists
2018-01-13 07:00:23,103 DEBUG [main] zookeeper.ZKUtil(357): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Set watcher on znode that does not yet exist, /1/master
2018-01-13 07:00:23,104 DEBUG [main] zookeeper.ZKUtil(357): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Set watcher on znode that does not yet exist, /1/running
2018-01-13 07:00:23,128 DEBUG [main] ipc.RpcExecutor(254): Started 0 default.FPBQ.Fifo handlers, qsize=1 on port=53894
2018-01-13 07:00:23,145 DEBUG [main] ipc.RpcExecutor(254): Started 0 priority.FPBQ.Fifo handlers, qsize=1 on port=53894
2018-01-13 07:00:23,146 DEBUG [main] ipc.RpcExecutor(254): Started 0 replication.FPBQ.Fifo handlers, qsize=1 on port=53894
2018-01-13 07:00:23,154 INFO  [main] master.HMaster(480): hbase.rootdir=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00, hbase.cluster.distributed=false
2018-01-13 07:00:23,257 INFO  [main] client.ConnectionUtils(122): regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0 server-side Connection retries=45
2018-01-13 07:00:23,258 INFO  [main] ipc.RpcExecutor(148): RpcExecutor default.FPBQ using fifo as call queue; numCallQueues=1; maxQueueLength=50; handlerCount=5
2018-01-13 07:00:23,258 INFO  [main] ipc.RpcExecutor(148): RpcExecutor priority.FPBQ using fifo as call queue; numCallQueues=1; maxQueueLength=50; handlerCount=6
2018-01-13 07:00:23,258 INFO  [main] ipc.RpcExecutor(148): RpcExecutor replication.FPBQ using fifo as call queue; numCallQueues=1; maxQueueLength=50; handlerCount=3
2018-01-13 07:00:23,263 INFO  [main] ipc.RpcServerFactory(65): Creating org.apache.hadoop.hbase.ipc.NettyRpcServer hosting hbase.pb.ClientService, hbase.pb.AdminService
2018-01-13 07:00:23,264 INFO  [main] io.ByteBufferPool(83): Created ByteBufferPool with bufferSize : 65536 and maxPoolSize : 320
2018-01-13 07:00:23,274 INFO  [main] ipc.NettyRpcServer(109): NettyRpcServer bind to address=/67.195.81.155:60093
2018-01-13 07:00:23,275 INFO  [main] hfile.CacheConfig(262): Created cacheConfig: blockCache=LruBlockCache{blockCount=0, currentSize=765640, freeSize=1043196664, maxSize=1043962304, heapSize=765640, minSize=991764160, minFactor=0.95, multiSize=495882080, multiFactor=0.5, singleSize=247941040, singleFactor=0.25}, cacheDataOnRead=true, cacheDataOnWrite=false, cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false, cacheDataCompressed=false, prefetchOnOpen=false
2018-01-13 07:00:23,276 INFO  [main] hfile.CacheConfig(262): Created cacheConfig: blockCache=LruBlockCache{blockCount=0, currentSize=765640, freeSize=1043196664, maxSize=1043962304, heapSize=765640, minSize=991764160, minFactor=0.95, multiSize=495882080, multiFactor=0.5, singleSize=247941040, singleFactor=0.25}, cacheDataOnRead=true, cacheDataOnWrite=false, cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false, cacheDataCompressed=false, prefetchOnOpen=false
2018-01-13 07:00:23,278 INFO  [main] fs.HFileSystem(349): Added intercepting call to namenode#getBlockLocations so can do block reordering using class org.apache.hadoop.hbase.fs.HFileSystem$ReorderWALBlocks
2018-01-13 07:00:23,303 INFO  [main] fs.HFileSystem(349): Added intercepting call to namenode#getBlockLocations so can do block reordering using class org.apache.hadoop.hbase.fs.HFileSystem$ReorderWALBlocks
2018-01-13 07:00:23,307 INFO  [main] zookeeper.RecoverableZooKeeper(106): Process identifier=regionserver:60093 connecting to ZooKeeper ensemble=localhost:65105
2018-01-13 07:00:23,324 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): regionserver:600930x0, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=None, state=SyncConnected, path=null
2018-01-13 07:00:23,326 DEBUG [main-EventThread] zookeeper.ZKWatcher(523): regionserver:60093-0x160ee519be80004 connected
2018-01-13 07:00:23,326 DEBUG [main] zookeeper.ZKUtil(357): regionserver:60093-0x160ee519be80004, quorum=localhost:65105, baseZNode=/1 Set watcher on znode that does not yet exist, /1/master
2018-01-13 07:00:23,328 DEBUG [main] zookeeper.ZKUtil(357): regionserver:60093-0x160ee519be80004, quorum=localhost:65105, baseZNode=/1 Set watcher on znode that does not yet exist, /1/running
2018-01-13 07:00:23,328 DEBUG [main] ipc.RpcExecutor(254): Started 0 default.FPBQ.Fifo handlers, qsize=1 on port=60093
2018-01-13 07:00:23,337 DEBUG [main] ipc.RpcExecutor(254): Started 0 priority.FPBQ.Fifo handlers, qsize=1 on port=60093
2018-01-13 07:00:23,345 DEBUG [main] ipc.RpcExecutor(254): Started 0 replication.FPBQ.Fifo handlers, qsize=1 on port=60093
2018-01-13 07:00:23,379 INFO  [main] client.ConnectionUtils(122): regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0 server-side Connection retries=45
2018-01-13 07:00:23,380 INFO  [main] ipc.RpcExecutor(148): RpcExecutor default.FPBQ using fifo as call queue; numCallQueues=1; maxQueueLength=50; handlerCount=5
2018-01-13 07:00:23,380 INFO  [main] ipc.RpcExecutor(148): RpcExecutor priority.FPBQ using fifo as call queue; numCallQueues=1; maxQueueLength=50; handlerCount=6
2018-01-13 07:00:23,381 INFO  [main] ipc.RpcExecutor(148): RpcExecutor replication.FPBQ using fifo as call queue; numCallQueues=1; maxQueueLength=50; handlerCount=3
2018-01-13 07:00:23,381 INFO  [main] ipc.RpcServerFactory(65): Creating org.apache.hadoop.hbase.ipc.NettyRpcServer hosting hbase.pb.ClientService, hbase.pb.AdminService
2018-01-13 07:00:23,381 INFO  [main] io.ByteBufferPool(83): Created ByteBufferPool with bufferSize : 65536 and maxPoolSize : 320
2018-01-13 07:00:23,387 INFO  [main] ipc.NettyRpcServer(109): NettyRpcServer bind to address=/67.195.81.155:34432
2018-01-13 07:00:23,388 INFO  [main] hfile.CacheConfig(262): Created cacheConfig: blockCache=LruBlockCache{blockCount=0, currentSize=765640, freeSize=1043196664, maxSize=1043962304, heapSize=765640, minSize=991764160, minFactor=0.95, multiSize=495882080, multiFactor=0.5, singleSize=247941040, singleFactor=0.25}, cacheDataOnRead=true, cacheDataOnWrite=false, cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false, cacheDataCompressed=false, prefetchOnOpen=false
2018-01-13 07:00:23,388 INFO  [main] hfile.CacheConfig(262): Created cacheConfig: blockCache=LruBlockCache{blockCount=0, currentSize=765640, freeSize=1043196664, maxSize=1043962304, heapSize=765640, minSize=991764160, minFactor=0.95, multiSize=495882080, multiFactor=0.5, singleSize=247941040, singleFactor=0.25}, cacheDataOnRead=true, cacheDataOnWrite=false, cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false, cacheDataCompressed=false, prefetchOnOpen=false
2018-01-13 07:00:23,391 INFO  [main] fs.HFileSystem(349): Added intercepting call to namenode#getBlockLocations so can do block reordering using class org.apache.hadoop.hbase.fs.HFileSystem$ReorderWALBlocks
2018-01-13 07:00:23,395 INFO  [main] fs.HFileSystem(349): Added intercepting call to namenode#getBlockLocations so can do block reordering using class org.apache.hadoop.hbase.fs.HFileSystem$ReorderWALBlocks
2018-01-13 07:00:23,398 INFO  [main] zookeeper.RecoverableZooKeeper(106): Process identifier=regionserver:34432 connecting to ZooKeeper ensemble=localhost:65105
2018-01-13 07:00:23,419 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): regionserver:344320x0, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=None, state=SyncConnected, path=null
2018-01-13 07:00:23,420 DEBUG [main-EventThread] zookeeper.ZKWatcher(523): regionserver:34432-0x160ee519be80005 connected
2018-01-13 07:00:23,421 DEBUG [main] zookeeper.ZKUtil(357): regionserver:34432-0x160ee519be80005, quorum=localhost:65105, baseZNode=/1 Set watcher on znode that does not yet exist, /1/master
2018-01-13 07:00:23,422 DEBUG [main] zookeeper.ZKUtil(357): regionserver:34432-0x160ee519be80005, quorum=localhost:65105, baseZNode=/1 Set watcher on znode that does not yet exist, /1/running
2018-01-13 07:00:23,422 DEBUG [main] ipc.RpcExecutor(254): Started 0 default.FPBQ.Fifo handlers, qsize=1 on port=34432
2018-01-13 07:00:23,437 DEBUG [main] ipc.RpcExecutor(254): Started 0 priority.FPBQ.Fifo handlers, qsize=1 on port=34432
2018-01-13 07:00:23,451 DEBUG [main] ipc.RpcExecutor(254): Started 0 replication.FPBQ.Fifo handlers, qsize=1 on port=34432
2018-01-13 07:00:23,464 INFO  [M:0;asf911:53894] master.HMaster(1998): Adding backup master ZNode /1/backup-masters/asf911.gq1.ygridcore.net,53894,1515826821951
2018-01-13 07:00:23,494 DEBUG [M:0;asf911:53894] zookeeper.ZKUtil(355): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Set watcher on existing znode=/1/backup-masters/asf911.gq1.ygridcore.net,53894,1515826821951
2018-01-13 07:00:23,550 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeCreated, state=SyncConnected, path=/1/master
2018-01-13 07:00:23,550 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): regionserver:34432-0x160ee519be80005, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeCreated, state=SyncConnected, path=/1/master
2018-01-13 07:00:23,550 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): regionserver:60093-0x160ee519be80004, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeCreated, state=SyncConnected, path=/1/master
2018-01-13 07:00:23,554 DEBUG [M:0;asf911:53894] zookeeper.ZKUtil(355): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Set watcher on existing znode=/1/master
2018-01-13 07:00:23,556 DEBUG [main-EventThread] zookeeper.ZKUtil(355): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Set watcher on existing znode=/1/master
2018-01-13 07:00:23,556 DEBUG [main-EventThread] master.ActiveMasterManager(128): A master is now available
2018-01-13 07:00:23,557 INFO  [M:0;asf911:53894] master.ActiveMasterManager(172): Deleting ZNode for /1/backup-masters/asf911.gq1.ygridcore.net,53894,1515826821951 from backup master directory
2018-01-13 07:00:23,568 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeDeleted, state=SyncConnected, path=/1/backup-masters/asf911.gq1.ygridcore.net,53894,1515826821951
2018-01-13 07:00:23,570 WARN  [M:0;asf911:53894] hbase.ZNodeClearer(62): Environment variable HBASE_ZNODE_FILE not set; znodes will not be cleared on crash by start scripts (Longer MTTR!)
2018-01-13 07:00:23,570 INFO  [M:0;asf911:53894] master.ActiveMasterManager(181): Registered Active Master=asf911.gq1.ygridcore.net,53894,1515826821951
2018-01-13 07:00:23,574 INFO  [M:0;asf911:53894] regionserver.ChunkCreator(408): Allocating MemStoreChunkPool with chunk size 2 MB, max count 497, initial count 0
2018-01-13 07:00:23,701 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:34940 is added to blk_1073741826_1002{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-f5839f39-099d-419b-8c6f-32dec519a9b9:NORMAL:127.0.0.1:36231|RBW], ReplicaUC[[DISK]DS-2d1c80f9-c237-446d-b413-0d899e38665b:NORMAL:127.0.0.1:34940|RBW]]} size 0
2018-01-13 07:00:23,702 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:36231 is added to blk_1073741826_1002{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-2d1c80f9-c237-446d-b413-0d899e38665b:NORMAL:127.0.0.1:34940|RBW], ReplicaUC[[DISK]DS-51167812-25c5-42dd-9759-7b939eb0ef2b:NORMAL:127.0.0.1:36231|FINALIZED]]} size 0
2018-01-13 07:00:23,706 DEBUG [M:0;asf911:53894] util.FSUtils(665): Created cluster ID file at hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/hbase.id with ID: c57babe9-d8c4-4f5d-8c8d-7884be49786a
2018-01-13 07:00:23,767 INFO  [M:0;asf911:53894] master.MasterFileSystem(388): BOOTSTRAP: creating hbase:meta region
2018-01-13 07:00:23,776 INFO  [M:0;asf911:53894] regionserver.HRegion(6772): creating HRegion hbase:meta HTD == 'hbase:meta', {TABLE_ATTRIBUTES => {IS_META => 'true', coprocessor$1 => '|org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint|536870911|'}, {NAME => 'info', VERSIONS => '3', EVICT_BLOCKS_ON_CLOSE => 'false', NEW_VERSION_BEHAVIOR => 'false', KEEP_DELETED_CELLS => 'FALSE', CACHE_DATA_ON_WRITE => 'false', DATA_BLOCK_ENCODING => 'NONE', TTL => 'FOREVER', MIN_VERSIONS => '0', REPLICATION_SCOPE => '0', BLOOMFILTER => 'NONE', CACHE_INDEX_ON_WRITE => 'false', IN_MEMORY => 'false', CACHE_BLOOMS_ON_WRITE => 'false', PREFETCH_BLOCKS_ON_OPEN => 'false', COMPRESSION => 'NONE', BLOCKCACHE => 'false', BLOCKSIZE => '8192'}, {NAME => 'rep_barrier', VERSIONS => '3', EVICT_BLOCKS_ON_CLOSE => 'false', NEW_VERSION_BEHAVIOR => 'false', KEEP_DELETED_CELLS => 'FALSE', CACHE_DATA_ON_WRITE => 'false', DATA_BLOCK_ENCODING => 'NONE', TTL => 'FOREVER', MIN_VERSIONS => '0', REPLICATION_SCOPE => '0', BLOOMFILTER => 'NONE', CACHE_INDEX_ON_WRITE => 'false', IN_MEMORY => 'true', CACHE_BLOOMS_ON_WRITE => 'false', PREFETCH_BLOCKS_ON_OPEN => 'false', COMPRESSION => 'NONE', BLOCKCACHE => 'true', BLOCKSIZE => '8192'}, {NAME => 'rep_meta', VERSIONS => '3', EVICT_BLOCKS_ON_CLOSE => 'false', NEW_VERSION_BEHAVIOR => 'false', KEEP_DELETED_CELLS => 'FALSE', CACHE_DATA_ON_WRITE => 'false', DATA_BLOCK_ENCODING => 'NONE', TTL => 'FOREVER', MIN_VERSIONS => '0', REPLICATION_SCOPE => '0', BLOOMFILTER => 'NONE', CACHE_INDEX_ON_WRITE => 'false', IN_MEMORY => 'true', CACHE_BLOOMS_ON_WRITE => 'false', PREFETCH_BLOCKS_ON_OPEN => 'false', COMPRESSION => 'NONE', BLOCKCACHE => 'true', BLOCKSIZE => '8192'}, {NAME => 'rep_position', VERSIONS => '3', EVICT_BLOCKS_ON_CLOSE => 'false', NEW_VERSION_BEHAVIOR => 'false', KEEP_DELETED_CELLS => 'FALSE', CACHE_DATA_ON_WRITE => 'false', DATA_BLOCK_ENCODING => 'NONE', TTL => 'FOREVER', MIN_VERSIONS => '0', REPLICATION_SCOPE => '0', BLOOMFILTER => 'NONE', CACHE_INDEX_ON_WRITE => 'false', IN_MEMORY => 'true', CACHE_BLOOMS_ON_WRITE => 'false', PREFETCH_BLOCKS_ON_OPEN => 'false', COMPRESSION => 'NONE', BLOCKCACHE => 'true', BLOCKSIZE => '8192'}, {NAME => 'table', VERSIONS => '10', EVICT_BLOCKS_ON_CLOSE => 'false', NEW_VERSION_BEHAVIOR => 'false', KEEP_DELETED_CELLS => 'FALSE', CACHE_DATA_ON_WRITE => 'false', DATA_BLOCK_ENCODING => 'NONE', TTL => 'FOREVER', MIN_VERSIONS => '0', REPLICATION_SCOPE => '0', BLOOMFILTER => 'NONE', CACHE_INDEX_ON_WRITE => 'false', IN_MEMORY => 'true', CACHE_BLOOMS_ON_WRITE => 'false', PREFETCH_BLOCKS_ON_OPEN => 'false', COMPRESSION => 'NONE', BLOCKCACHE => 'true', BLOCKSIZE => '8192'} RootDir = hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00 Table name == hbase:meta
2018-01-13 07:00:23,832 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:34940 is added to blk_1073741827_1003{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-51167812-25c5-42dd-9759-7b939eb0ef2b:NORMAL:127.0.0.1:36231|RBW], ReplicaUC[[DISK]DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e:NORMAL:127.0.0.1:34940|FINALIZED]]} size 0
2018-01-13 07:00:23,832 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:36231 is added to blk_1073741827_1003{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e:NORMAL:127.0.0.1:34940|FINALIZED], ReplicaUC[[DISK]DS-f5839f39-099d-419b-8c6f-32dec519a9b9:NORMAL:127.0.0.1:36231|FINALIZED]]} size 0
2018-01-13 07:00:23,840 DEBUG [M:0;asf911:53894] regionserver.HRegion(807): Instantiated hbase:meta,,1.1588230740
2018-01-13 07:00:23,887 DEBUG [StoreOpener-1588230740-1] util.CommonFSUtils(559): Set storagePolicy=HOT for path=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/hbase/meta/1588230740/info
2018-01-13 07:00:23,905 INFO  [StoreOpener-1588230740-1] regionserver.HStore(293): Memstore class name is org.apache.hadoop.hbase.regionserver.DefaultMemStore
2018-01-13 07:00:23,909 INFO  [StoreOpener-1588230740-1] hfile.CacheConfig(239): Created cacheConfig for info: blockCache=LruBlockCache{blockCount=0, currentSize=765640, freeSize=1043196664, maxSize=1043962304, heapSize=765640, minSize=991764160, minFactor=0.95, multiSize=495882080, multiFactor=0.5, singleSize=247941040, singleFactor=0.25}, cacheDataOnRead=false, cacheDataOnWrite=false, cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false, cacheDataCompressed=false, prefetchOnOpen=false
2018-01-13 07:00:23,925 INFO  [StoreOpener-1588230740-1] compactions.CompactionConfiguration(146): size [134217728, 9223372036854775807, 9223372036854775807); files [3, 10); ratio 1.200000; off-peak ratio 5.000000; throttle point 2684354560; major period 604800000, major jitter 0.500000, min locality to compact 0.000000; tiered compaction: max_age 9223372036854775807, incoming window min 6, compaction policy for tiered window org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy, single output for minor true, compaction window factory org.apache.hadoop.hbase.regionserver.compactions.ExponentialCompactionWindowFactory
2018-01-13 07:00:23,949 DEBUG [StoreOpener-1588230740-1] util.CommonFSUtils(559): Set storagePolicy=HOT for path=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/hbase/meta/1588230740/rep_barrier
2018-01-13 07:00:23,949 INFO  [StoreOpener-1588230740-1] regionserver.HStore(293): Memstore class name is org.apache.hadoop.hbase.regionserver.DefaultMemStore
2018-01-13 07:00:23,950 INFO  [StoreOpener-1588230740-1] hfile.CacheConfig(239): Created cacheConfig for rep_barrier: blockCache=LruBlockCache{blockCount=0, currentSize=765640, freeSize=1043196664, maxSize=1043962304, heapSize=765640, minSize=991764160, minFactor=0.95, multiSize=495882080, multiFactor=0.5, singleSize=247941040, singleFactor=0.25}, cacheDataOnRead=true, cacheDataOnWrite=false, cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false, cacheDataCompressed=false, prefetchOnOpen=false
2018-01-13 07:00:23,951 INFO  [StoreOpener-1588230740-1] compactions.CompactionConfiguration(146): size [134217728, 9223372036854775807, 9223372036854775807); files [3, 10); ratio 1.200000; off-peak ratio 5.000000; throttle point 2684354560; major period 604800000, major jitter 0.500000, min locality to compact 0.000000; tiered compaction: max_age 9223372036854775807, incoming window min 6, compaction policy for tiered window org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy, single output for minor true, compaction window factory org.apache.hadoop.hbase.regionserver.compactions.ExponentialCompactionWindowFactory
2018-01-13 07:00:23,958 DEBUG [StoreOpener-1588230740-1] util.CommonFSUtils(559): Set storagePolicy=HOT for path=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/hbase/meta/1588230740/rep_meta
2018-01-13 07:00:23,959 INFO  [StoreOpener-1588230740-1] regionserver.HStore(293): Memstore class name is org.apache.hadoop.hbase.regionserver.DefaultMemStore
2018-01-13 07:00:23,959 INFO  [StoreOpener-1588230740-1] hfile.CacheConfig(239): Created cacheConfig for rep_meta: blockCache=LruBlockCache{blockCount=0, currentSize=765640, freeSize=1043196664, maxSize=1043962304, heapSize=765640, minSize=991764160, minFactor=0.95, multiSize=495882080, multiFactor=0.5, singleSize=247941040, singleFactor=0.25}, cacheDataOnRead=true, cacheDataOnWrite=false, cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false, cacheDataCompressed=false, prefetchOnOpen=false
2018-01-13 07:00:23,960 INFO  [StoreOpener-1588230740-1] compactions.CompactionConfiguration(146): size [134217728, 9223372036854775807, 9223372036854775807); files [3, 10); ratio 1.200000; off-peak ratio 5.000000; throttle point 2684354560; major period 604800000, major jitter 0.500000, min locality to compact 0.000000; tiered compaction: max_age 9223372036854775807, incoming window min 6, compaction policy for tiered window org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy, single output for minor true, compaction window factory org.apache.hadoop.hbase.regionserver.compactions.ExponentialCompactionWindowFactory
2018-01-13 07:00:23,967 DEBUG [StoreOpener-1588230740-1] util.CommonFSUtils(559): Set storagePolicy=HOT for path=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/hbase/meta/1588230740/rep_position
2018-01-13 07:00:23,968 INFO  [StoreOpener-1588230740-1] regionserver.HStore(293): Memstore class name is org.apache.hadoop.hbase.regionserver.DefaultMemStore
2018-01-13 07:00:23,968 INFO  [StoreOpener-1588230740-1] hfile.CacheConfig(239): Created cacheConfig for rep_position: blockCache=LruBlockCache{blockCount=0, currentSize=765640, freeSize=1043196664, maxSize=1043962304, heapSize=765640, minSize=991764160, minFactor=0.95, multiSize=495882080, multiFactor=0.5, singleSize=247941040, singleFactor=0.25}, cacheDataOnRead=true, cacheDataOnWrite=false, cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false, cacheDataCompressed=false, prefetchOnOpen=false
2018-01-13 07:00:23,969 INFO  [StoreOpener-1588230740-1] compactions.CompactionConfiguration(146): size [134217728, 9223372036854775807, 9223372036854775807); files [3, 10); ratio 1.200000; off-peak ratio 5.000000; throttle point 2684354560; major period 604800000, major jitter 0.500000, min locality to compact 0.000000; tiered compaction: max_age 9223372036854775807, incoming window min 6, compaction policy for tiered window org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy, single output for minor true, compaction window factory org.apache.hadoop.hbase.regionserver.compactions.ExponentialCompactionWindowFactory
2018-01-13 07:00:23,976 DEBUG [StoreOpener-1588230740-1] util.CommonFSUtils(559): Set storagePolicy=HOT for path=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/hbase/meta/1588230740/table
2018-01-13 07:00:23,977 INFO  [StoreOpener-1588230740-1] regionserver.HStore(293): Memstore class name is org.apache.hadoop.hbase.regionserver.DefaultMemStore
2018-01-13 07:00:23,977 INFO  [StoreOpener-1588230740-1] hfile.CacheConfig(239): Created cacheConfig for table: blockCache=LruBlockCache{blockCount=0, currentSize=765640, freeSize=1043196664, maxSize=1043962304, heapSize=765640, minSize=991764160, minFactor=0.95, multiSize=495882080, multiFactor=0.5, singleSize=247941040, singleFactor=0.25}, cacheDataOnRead=true, cacheDataOnWrite=false, cacheIndexesOnWrite=false, cacheBloomsOnWrite=false, cacheEvictOnClose=false, cacheDataCompressed=false, prefetchOnOpen=false
2018-01-13 07:00:23,979 INFO  [StoreOpener-1588230740-1] compactions.CompactionConfiguration(146): size [134217728, 9223372036854775807, 9223372036854775807); files [3, 10); ratio 1.200000; off-peak ratio 5.000000; throttle point 2684354560; major period 604800000, major jitter 0.500000, min locality to compact 0.000000; tiered compaction: max_age 9223372036854775807, incoming window min 6, compaction policy for tiered window org.apache.hadoop.hbase.regionserver.compactions.ExploringCompactionPolicy, single output for minor true, compaction window factory org.apache.hadoop.hbase.regionserver.compactions.ExponentialCompactionWindowFactory
2018-01-13 07:00:23,995 DEBUG [M:0;asf911:53894] regionserver.HRegion(4350): Found 0 recovered edits file(s) under hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/hbase/meta/1588230740
2018-01-13 07:00:24,019 DEBUG [M:0;asf911:53894] regionserver.FlushLargeStoresPolicy(60): No hbase.hregion.percolumnfamilyflush.size.lower.bound set in table hbase:meta descriptor;using region.getMemStoreFlushSize/# of families (25.6M)) instead.
2018-01-13 07:00:24,028 DEBUG [M:0;asf911:53894] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/hbase/meta/1588230740/recovered.edits/2.seqid, newSeqId=2, maxSeqId=0
2018-01-13 07:00:24,028 INFO  [M:0;asf911:53894] regionserver.HRegion(960): Onlined 1588230740; next sequenceid=2
2018-01-13 07:00:24,029 DEBUG [M:0;asf911:53894] regionserver.HRegion(1481): Closing hbase:meta,,1.1588230740: disabling compactions & flushes
2018-01-13 07:00:24,029 DEBUG [M:0;asf911:53894] regionserver.HRegion(1520): Updates disabled for region hbase:meta,,1.1588230740
2018-01-13 07:00:24,031 INFO  [StoreCloserThread-hbase:meta,,1.1588230740-1] regionserver.HStore(930): Closed info
2018-01-13 07:00:24,033 INFO  [StoreCloserThread-hbase:meta,,1.1588230740-1] regionserver.HStore(930): Closed rep_barrier
2018-01-13 07:00:24,033 INFO  [StoreCloserThread-hbase:meta,,1.1588230740-1] regionserver.HStore(930): Closed rep_meta
2018-01-13 07:00:24,033 INFO  [StoreCloserThread-hbase:meta,,1.1588230740-1] regionserver.HStore(930): Closed rep_position
2018-01-13 07:00:24,033 INFO  [StoreCloserThread-hbase:meta,,1.1588230740-1] regionserver.HStore(930): Closed table
2018-01-13 07:00:24,034 INFO  [M:0;asf911:53894] regionserver.HRegion(1633): Closed hbase:meta,,1.1588230740
2018-01-13 07:00:24,077 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:34940 is added to blk_1073741828_1004{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-f5839f39-099d-419b-8c6f-32dec519a9b9:NORMAL:127.0.0.1:36231|RBW], ReplicaUC[[DISK]DS-2d1c80f9-c237-446d-b413-0d899e38665b:NORMAL:127.0.0.1:34940|RBW]]} size 0
2018-01-13 07:00:24,079 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:36231 is added to blk_1073741828_1004{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-2d1c80f9-c237-446d-b413-0d899e38665b:NORMAL:127.0.0.1:34940|RBW], ReplicaUC[[DISK]DS-51167812-25c5-42dd-9759-7b939eb0ef2b:NORMAL:127.0.0.1:36231|FINALIZED]]} size 0
2018-01-13 07:00:24,085 DEBUG [M:0;asf911:53894] util.FSTableDescriptors(716): Wrote descriptor into: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/hbase/meta/.tabledesc/.tableinfo.0000000001
2018-01-13 07:00:24,127 INFO  [M:0;asf911:53894] fs.HFileSystem(349): Added intercepting call to namenode#getBlockLocations so can do block reordering using class org.apache.hadoop.hbase.fs.HFileSystem$ReorderWALBlocks
2018-01-13 07:00:24,149 INFO  [M:0;asf911:53894] coordination.ZKSplitLogManagerCoordination(492): Found 0 orphan tasks and 0 rescan nodes
2018-01-13 07:00:24,151 DEBUG [M:0;asf911:53894] util.FSTableDescriptors(292): Fetching table descriptors from the filesystem.
2018-01-13 07:00:24,172 INFO  [M:0;asf911:53894] zookeeper.ReadOnlyZKClient(130): Start read only zookeeper connection 0x39045593 to localhost:65105, session timeout 90000 ms, retries 1, retry interval 10 ms, keep alive 60000 ms
2018-01-13 07:00:24,189 DEBUG [M:0;asf911:53894] ipc.AbstractRpcClient(200): Codec=org.apache.hadoop.hbase.codec.KeyValueCodec@53258214, compressor=null, tcpKeepAlive=true, tcpNoDelay=true, connectTO=10000, readTO=20000, writeTO=60000, minIdleTimeBeforeClose=120000, maxRetries=0, fallbackAllowed=true, bind address=null
2018-01-13 07:00:24,233 INFO  [M:0;asf911:53894] balancer.BaseLoadBalancer(1037): slop=0.001, tablesOnMaster=false, systemTablesOnMaster=false
2018-01-13 07:00:24,243 INFO  [M:0;asf911:53894] balancer.StochasticLoadBalancer(214): Loaded config; maxSteps=1000000, stepsPerRegion=800, maxRunningTime=30000, isByTable=false, etc.
2018-01-13 07:00:24,247 DEBUG [M:0;asf911:53894] zookeeper.ZKUtil(357): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Set watcher on znode that does not yet exist, /1/balancer
2018-01-13 07:00:24,249 DEBUG [M:0;asf911:53894] zookeeper.ZKUtil(357): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Set watcher on znode that does not yet exist, /1/normalizer
2018-01-13 07:00:24,261 DEBUG [M:0;asf911:53894] zookeeper.ZKUtil(357): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Set watcher on znode that does not yet exist, /1/switch/split
2018-01-13 07:00:24,262 DEBUG [M:0;asf911:53894] zookeeper.ZKUtil(357): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Set watcher on znode that does not yet exist, /1/switch/merge
2018-01-13 07:00:24,291 INFO  [M:0;asf911:53894] assignment.AssignmentManager(219): Starting assignment manager
2018-01-13 07:00:24,323 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): regionserver:60093-0x160ee519be80004, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeCreated, state=SyncConnected, path=/1/running
2018-01-13 07:00:24,324 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeCreated, state=SyncConnected, path=/1/running
2018-01-13 07:00:24,324 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): regionserver:34432-0x160ee519be80005, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeCreated, state=SyncConnected, path=/1/running
2018-01-13 07:00:24,324 INFO  [M:0;asf911:53894] master.HMaster(746): Server active/primary master=asf911.gq1.ygridcore.net,53894,1515826821951, sessionid=0x160ee519be80003, setting cluster-up flag (Was=false)
2018-01-13 07:00:24,393 INFO  [M:0;asf911:53894] procedure.ZKProcedureUtil(271): Clearing all procedure znodes: /1/flush-table-proc/acquired /1/flush-table-proc/reached /1/flush-table-proc/abort
2018-01-13 07:00:24,395 DEBUG [M:0;asf911:53894] procedure.ZKProcedureCoordinator(248): Starting the controller for procedure member:asf911.gq1.ygridcore.net,53894,1515826821951
2018-01-13 07:00:24,443 INFO  [M:0;asf911:53894] procedure.ZKProcedureUtil(271): Clearing all procedure znodes: /1/online-snapshot/acquired /1/online-snapshot/reached /1/online-snapshot/abort
2018-01-13 07:00:24,445 DEBUG [M:0;asf911:53894] procedure.ZKProcedureCoordinator(248): Starting the controller for procedure member:asf911.gq1.ygridcore.net,53894,1515826821951
2018-01-13 07:00:24,471 INFO  [RS:1;asf911:34432] zookeeper.ReadOnlyZKClient(130): Start read only zookeeper connection 0x100d71db to localhost:65105, session timeout 90000 ms, retries 1, retry interval 10 ms, keep alive 60000 ms
2018-01-13 07:00:24,471 INFO  [RS:0;asf911:60093] zookeeper.ReadOnlyZKClient(130): Start read only zookeeper connection 0x38966aaa to localhost:65105, session timeout 90000 ms, retries 1, retry interval 10 ms, keep alive 60000 ms
2018-01-13 07:00:24,496 DEBUG [RS:1;asf911:34432] ipc.AbstractRpcClient(200): Codec=org.apache.hadoop.hbase.codec.KeyValueCodec@1eca8daa, compressor=null, tcpKeepAlive=true, tcpNoDelay=true, connectTO=10000, readTO=20000, writeTO=60000, minIdleTimeBeforeClose=120000, maxRetries=0, fallbackAllowed=true, bind address=null
2018-01-13 07:00:24,498 INFO  [RS:1;asf911:34432] regionserver.HRegionServer(866): ClusterId : c57babe9-d8c4-4f5d-8c8d-7884be49786a
2018-01-13 07:00:24,498 DEBUG [RS:0;asf911:60093] ipc.AbstractRpcClient(200): Codec=org.apache.hadoop.hbase.codec.KeyValueCodec@365183c8, compressor=null, tcpKeepAlive=true, tcpNoDelay=true, connectTO=10000, readTO=20000, writeTO=60000, minIdleTimeBeforeClose=120000, maxRetries=0, fallbackAllowed=true, bind address=null
2018-01-13 07:00:24,500 INFO  [RS:0;asf911:60093] regionserver.HRegionServer(866): ClusterId : c57babe9-d8c4-4f5d-8c8d-7884be49786a
2018-01-13 07:00:24,505 DEBUG [RS:0;asf911:60093] procedure.RegionServerProcedureManagerHost(44): Procedure flush-table-proc is initializing
2018-01-13 07:00:24,506 DEBUG [RS:1;asf911:34432] procedure.RegionServerProcedureManagerHost(44): Procedure flush-table-proc is initializing
2018-01-13 07:00:24,518 DEBUG [RS:0;asf911:60093] zookeeper.RecoverableZooKeeper(571): Node /1/flush-table-proc/acquired already exists
2018-01-13 07:00:24,526 DEBUG [RS:1;asf911:34432] zookeeper.RecoverableZooKeeper(571): Node /1/flush-table-proc/acquired already exists
2018-01-13 07:00:24,529 DEBUG [RS:0;asf911:60093] procedure.RegionServerProcedureManagerHost(46): Procedure flush-table-proc is initialized
2018-01-13 07:00:24,529 DEBUG [RS:0;asf911:60093] procedure.RegionServerProcedureManagerHost(44): Procedure online-snapshot is initializing
2018-01-13 07:00:24,529 DEBUG [RS:1;asf911:34432] procedure.RegionServ
...[truncated 5156697 bytes]...
[StoreCloserThread-testVerifyRepJobWithRawOptions,ddd,1515826887170.37059cf08e4cfedff27dac1daba95e05.-1] regionserver.HStore(930): Closed fam_raw
2018-01-13 07:03:28,384 DEBUG [RS_CLOSE_REGION-asf911:60093-2] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/testVerifyRepJobWithRawOptions/37059cf08e4cfedff27dac1daba95e05/recovered.edits/4.seqid, newSeqId=4, maxSeqId=2
2018-01-13 07:03:28,385 INFO  [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1633): Closed testVerifyRepJobWithRawOptions,ddd,1515826887170.37059cf08e4cfedff27dac1daba95e05.
2018-01-13 07:03:28,385 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(125): Closed testVerifyRepJobWithRawOptions,ddd,1515826887170.37059cf08e4cfedff27dac1daba95e05.
2018-01-13 07:03:28,385 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(92): Processing close of test,jjj,1515826840321.1ba12e11d723fd327c25186a09c46178.
2018-01-13 07:03:28,385 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1481): Closing test,jjj,1515826840321.1ba12e11d723fd327c25186a09c46178.: disabling compactions & flushes
2018-01-13 07:03:28,385 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1520): Updates disabled for region test,jjj,1515826840321.1ba12e11d723fd327c25186a09c46178.
2018-01-13 07:03:28,385 INFO  [StoreCloserThread-test,jjj,1515826840321.1ba12e11d723fd327c25186a09c46178.-1] regionserver.HStore(930): Closed f
2018-01-13 07:03:28,386 INFO  [StoreCloserThread-test,jjj,1515826840321.1ba12e11d723fd327c25186a09c46178.-1] regionserver.HStore(930): Closed norep
2018-01-13 07:03:28,388 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:36231 is added to blk_1073742075_1251{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-2d1c80f9-c237-446d-b413-0d899e38665b:NORMAL:127.0.0.1:34940|RBW], ReplicaUC[[DISK]DS-f5839f39-099d-419b-8c6f-32dec519a9b9:NORMAL:127.0.0.1:36231|FINALIZED]]} size 0
2018-01-13 07:03:28,388 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:34940 is added to blk_1073742075_1251{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-f5839f39-099d-419b-8c6f-32dec519a9b9:NORMAL:127.0.0.1:36231|FINALIZED], ReplicaUC[[DISK]DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e:NORMAL:127.0.0.1:34940|FINALIZED]]} size 0
2018-01-13 07:03:28,389 INFO  [RS_CLOSE_REGION-asf911:60093-0] regionserver.DefaultStoreFlusher(87): Flushed, sequenceid=25, memsize=28, hasBloomFilter=true, into tmp file hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/83b0e412e19c9a662a6efe90cdb69ea3/.tmp/norep/ffbb6eac964546cb9f20b25aad9ab8a2
2018-01-13 07:03:28,391 DEBUG [RS_CLOSE_REGION-asf911:60093-2] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/1ba12e11d723fd327c25186a09c46178/recovered.edits/6.seqid, newSeqId=6, maxSeqId=2
2018-01-13 07:03:28,392 INFO  [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1633): Closed test,jjj,1515826840321.1ba12e11d723fd327c25186a09c46178.
2018-01-13 07:03:28,392 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(125): Closed test,jjj,1515826840321.1ba12e11d723fd327c25186a09c46178.
2018-01-13 07:03:28,392 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(92): Processing close of test,rrr,1515826840321.51ee82cf1a69d9bb3ad4d0da45bef87e.
2018-01-13 07:03:28,392 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1481): Closing test,rrr,1515826840321.51ee82cf1a69d9bb3ad4d0da45bef87e.: disabling compactions & flushes
2018-01-13 07:03:28,392 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1520): Updates disabled for region test,rrr,1515826840321.51ee82cf1a69d9bb3ad4d0da45bef87e.
2018-01-13 07:03:28,393 INFO  [StoreCloserThread-test,rrr,1515826840321.51ee82cf1a69d9bb3ad4d0da45bef87e.-1] regionserver.HStore(930): Closed f
2018-01-13 07:03:28,394 INFO  [StoreCloserThread-test,rrr,1515826840321.51ee82cf1a69d9bb3ad4d0da45bef87e.-1] regionserver.HStore(930): Closed norep
2018-01-13 07:03:28,394 INFO  [RS_CLOSE_REGION-asf911:60093-0] regionserver.StoreFileReader(496): Loaded Delete Family Bloom (CompoundBloomFilter) metadata for ffbb6eac964546cb9f20b25aad9ab8a2
2018-01-13 07:03:28,395 DEBUG [RS_CLOSE_REGION-asf911:60093-0] regionserver.HRegionFileSystem(463): Committing store file hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/83b0e412e19c9a662a6efe90cdb69ea3/.tmp/f/63c186c9ff2f49789c0d09b45eedd979 as hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/83b0e412e19c9a662a6efe90cdb69ea3/f/63c186c9ff2f49789c0d09b45eedd979
2018-01-13 07:03:28,398 DEBUG [RS_CLOSE_REGION-asf911:60093-2] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/51ee82cf1a69d9bb3ad4d0da45bef87e/recovered.edits/109.seqid, newSeqId=109, maxSeqId=2
2018-01-13 07:03:28,399 INFO  [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1633): Closed test,rrr,1515826840321.51ee82cf1a69d9bb3ad4d0da45bef87e.
2018-01-13 07:03:28,399 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(125): Closed test,rrr,1515826840321.51ee82cf1a69d9bb3ad4d0da45bef87e.
2018-01-13 07:03:28,400 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(92): Processing close of test,eee,1515826840321.af71e5d81998d4d16221e67a556c8800.
2018-01-13 07:03:28,400 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1481): Closing test,eee,1515826840321.af71e5d81998d4d16221e67a556c8800.: disabling compactions & flushes
2018-01-13 07:03:28,400 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1520): Updates disabled for region test,eee,1515826840321.af71e5d81998d4d16221e67a556c8800.
2018-01-13 07:03:28,400 INFO  [StoreCloserThread-test,eee,1515826840321.af71e5d81998d4d16221e67a556c8800.-1] regionserver.HStore(930): Closed f
2018-01-13 07:03:28,400 INFO  [StoreCloserThread-test,eee,1515826840321.af71e5d81998d4d16221e67a556c8800.-1] regionserver.HStore(930): Closed norep
2018-01-13 07:03:28,400 INFO  [RS_CLOSE_REGION-asf911:60093-0] regionserver.StoreFileReader(496): Loaded Delete Family Bloom (CompoundBloomFilter) metadata for 63c186c9ff2f49789c0d09b45eedd979
2018-01-13 07:03:28,401 INFO  [RS_CLOSE_REGION-asf911:60093-0] regionserver.HStore(1028): Added hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/83b0e412e19c9a662a6efe90cdb69ea3/f/63c186c9ff2f49789c0d09b45eedd979, entries=1, sequenceid=25, filesize=4.8 K
2018-01-13 07:03:28,402 DEBUG [RS_CLOSE_REGION-asf911:60093-0] regionserver.HRegionFileSystem(463): Committing store file hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/83b0e412e19c9a662a6efe90cdb69ea3/.tmp/norep/ffbb6eac964546cb9f20b25aad9ab8a2 as hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/83b0e412e19c9a662a6efe90cdb69ea3/norep/ffbb6eac964546cb9f20b25aad9ab8a2
2018-01-13 07:03:28,405 DEBUG [RS_CLOSE_REGION-asf911:60093-2] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/af71e5d81998d4d16221e67a556c8800/recovered.edits/6.seqid, newSeqId=6, maxSeqId=2
2018-01-13 07:03:28,406 INFO  [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1633): Closed test,eee,1515826840321.af71e5d81998d4d16221e67a556c8800.
2018-01-13 07:03:28,406 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(125): Closed test,eee,1515826840321.af71e5d81998d4d16221e67a556c8800.
2018-01-13 07:03:28,406 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(92): Processing close of testVerifyRepJobWithRawOptions,ppp,1515826887170.f20f1b57571e61f7e8006013ec44bb57.
2018-01-13 07:03:28,406 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1481): Closing testVerifyRepJobWithRawOptions,ppp,1515826887170.f20f1b57571e61f7e8006013ec44bb57.: disabling compactions & flushes
2018-01-13 07:03:28,406 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1520): Updates disabled for region testVerifyRepJobWithRawOptions,ppp,1515826887170.f20f1b57571e61f7e8006013ec44bb57.
2018-01-13 07:03:28,406 INFO  [StoreCloserThread-testVerifyRepJobWithRawOptions,ppp,1515826887170.f20f1b57571e61f7e8006013ec44bb57.-1] regionserver.HStore(930): Closed fam_raw
2018-01-13 07:03:28,408 INFO  [RS_CLOSE_REGION-asf911:60093-0] regionserver.StoreFileReader(496): Loaded Delete Family Bloom (CompoundBloomFilter) metadata for ffbb6eac964546cb9f20b25aad9ab8a2
2018-01-13 07:03:28,408 INFO  [RS_CLOSE_REGION-asf911:60093-0] regionserver.HStore(1028): Added hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/83b0e412e19c9a662a6efe90cdb69ea3/norep/ffbb6eac964546cb9f20b25aad9ab8a2, entries=1, sequenceid=25, filesize=4.8 K
2018-01-13 07:03:28,409 INFO  [RS_CLOSE_REGION-asf911:60093-0] regionserver.HRegion(2712): Finished memstore flush of ~82 B/82, currentsize=0 B/0 for region test,qqq,1515826840321.83b0e412e19c9a662a6efe90cdb69ea3. in 82ms, sequenceid=25, compaction requested=false
2018-01-13 07:03:28,413 INFO  [StoreCloserThread-test,qqq,1515826840321.83b0e412e19c9a662a6efe90cdb69ea3.-1] regionserver.HStore(930): Closed f
2018-01-13 07:03:28,416 INFO  [StoreCloserThread-test,qqq,1515826840321.83b0e412e19c9a662a6efe90cdb69ea3.-1] regionserver.HStore(930): Closed norep
2018-01-13 07:03:28,416 DEBUG [RS_CLOSE_REGION-asf911:60093-2] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/testVerifyRepJobWithRawOptions/f20f1b57571e61f7e8006013ec44bb57/recovered.edits/4.seqid, newSeqId=4, maxSeqId=2
2018-01-13 07:03:28,417 INFO  [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1633): Closed testVerifyRepJobWithRawOptions,ppp,1515826887170.f20f1b57571e61f7e8006013ec44bb57.
2018-01-13 07:03:28,418 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(125): Closed testVerifyRepJobWithRawOptions,ppp,1515826887170.f20f1b57571e61f7e8006013ec44bb57.
2018-01-13 07:03:28,418 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(92): Processing close of test,hhh,1515826840321.3b70165581b1cf4a21062f627377c0bc.
2018-01-13 07:03:28,418 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1481): Closing test,hhh,1515826840321.3b70165581b1cf4a21062f627377c0bc.: disabling compactions & flushes
2018-01-13 07:03:28,418 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1520): Updates disabled for region test,hhh,1515826840321.3b70165581b1cf4a21062f627377c0bc.
2018-01-13 07:03:28,419 INFO  [StoreCloserThread-test,hhh,1515826840321.3b70165581b1cf4a21062f627377c0bc.-1] regionserver.HStore(930): Closed f
2018-01-13 07:03:28,419 INFO  [StoreCloserThread-test,hhh,1515826840321.3b70165581b1cf4a21062f627377c0bc.-1] regionserver.HStore(930): Closed norep
2018-01-13 07:03:28,421 DEBUG [RS_CLOSE_REGION-asf911:60093-0] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/83b0e412e19c9a662a6efe90cdb69ea3/recovered.edits/28.seqid, newSeqId=28, maxSeqId=2
2018-01-13 07:03:28,422 INFO  [RS_CLOSE_REGION-asf911:60093-0] regionserver.HRegion(1633): Closed test,qqq,1515826840321.83b0e412e19c9a662a6efe90cdb69ea3.
2018-01-13 07:03:28,422 DEBUG [RS_CLOSE_REGION-asf911:60093-0] handler.CloseRegionHandler(125): Closed test,qqq,1515826840321.83b0e412e19c9a662a6efe90cdb69ea3.
2018-01-13 07:03:28,422 DEBUG [RS_CLOSE_REGION-asf911:60093-0] handler.CloseRegionHandler(92): Processing close of testVerifyRepJobWithRawOptions,rrr,1515826887170.402010593b22a88e5ab442e7469d445e.
2018-01-13 07:03:28,422 DEBUG [RS_CLOSE_REGION-asf911:60093-0] regionserver.HRegion(1481): Closing testVerifyRepJobWithRawOptions,rrr,1515826887170.402010593b22a88e5ab442e7469d445e.: disabling compactions & flushes
2018-01-13 07:03:28,422 DEBUG [RS_CLOSE_REGION-asf911:60093-0] regionserver.HRegion(1520): Updates disabled for region testVerifyRepJobWithRawOptions,rrr,1515826887170.402010593b22a88e5ab442e7469d445e.
2018-01-13 07:03:28,423 INFO  [StoreCloserThread-testVerifyRepJobWithRawOptions,rrr,1515826887170.402010593b22a88e5ab442e7469d445e.-1] regionserver.HStore(930): Closed fam_raw
2018-01-13 07:03:28,423 DEBUG [RS_CLOSE_REGION-asf911:60093-2] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/3b70165581b1cf4a21062f627377c0bc/recovered.edits/6.seqid, newSeqId=6, maxSeqId=2
2018-01-13 07:03:28,424 INFO  [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1633): Closed test,hhh,1515826840321.3b70165581b1cf4a21062f627377c0bc.
2018-01-13 07:03:28,424 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(125): Closed test,hhh,1515826840321.3b70165581b1cf4a21062f627377c0bc.
2018-01-13 07:03:28,424 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(92): Processing close of test,mmm,1515826840321.c8230ae310db3c741e23f471feabd42a.
2018-01-13 07:03:28,424 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1481): Closing test,mmm,1515826840321.c8230ae310db3c741e23f471feabd42a.: disabling compactions & flushes
2018-01-13 07:03:28,424 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1520): Updates disabled for region test,mmm,1515826840321.c8230ae310db3c741e23f471feabd42a.
2018-01-13 07:03:28,425 INFO  [StoreCloserThread-test,mmm,1515826840321.c8230ae310db3c741e23f471feabd42a.-1] regionserver.HStore(930): Closed f
2018-01-13 07:03:28,425 INFO  [StoreCloserThread-test,mmm,1515826840321.c8230ae310db3c741e23f471feabd42a.-1] regionserver.HStore(930): Closed norep
2018-01-13 07:03:28,429 DEBUG [RS_CLOSE_REGION-asf911:60093-0] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/testVerifyRepJobWithRawOptions/402010593b22a88e5ab442e7469d445e/recovered.edits/4.seqid, newSeqId=4, maxSeqId=2
2018-01-13 07:03:28,430 INFO  [RS_CLOSE_REGION-asf911:60093-0] regionserver.HRegion(1633): Closed testVerifyRepJobWithRawOptions,rrr,1515826887170.402010593b22a88e5ab442e7469d445e.
2018-01-13 07:03:28,430 DEBUG [RS_CLOSE_REGION-asf911:60093-0] handler.CloseRegionHandler(125): Closed testVerifyRepJobWithRawOptions,rrr,1515826887170.402010593b22a88e5ab442e7469d445e.
2018-01-13 07:03:28,430 DEBUG [RS_CLOSE_REGION-asf911:60093-0] handler.CloseRegionHandler(92): Processing close of testVerifyRepJobWithRawOptions,kkk,1515826887170.e00c78671344bc3f3ded24b19d7bde5e.
2018-01-13 07:03:28,430 DEBUG [RS_CLOSE_REGION-asf911:60093-0] regionserver.HRegion(1481): Closing testVerifyRepJobWithRawOptions,kkk,1515826887170.e00c78671344bc3f3ded24b19d7bde5e.: disabling compactions & flushes
2018-01-13 07:03:28,430 DEBUG [RS_CLOSE_REGION-asf911:60093-0] regionserver.HRegion(1520): Updates disabled for region testVerifyRepJobWithRawOptions,kkk,1515826887170.e00c78671344bc3f3ded24b19d7bde5e.
2018-01-13 07:03:28,431 INFO  [StoreCloserThread-testVerifyRepJobWithRawOptions,kkk,1515826887170.e00c78671344bc3f3ded24b19d7bde5e.-1] regionserver.HStore(930): Closed fam_raw
2018-01-13 07:03:28,432 DEBUG [RS_CLOSE_REGION-asf911:60093-2] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/test/c8230ae310db3c741e23f471feabd42a/recovered.edits/6.seqid, newSeqId=6, maxSeqId=2
2018-01-13 07:03:28,433 INFO  [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1633): Closed test,mmm,1515826840321.c8230ae310db3c741e23f471feabd42a.
2018-01-13 07:03:28,433 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(125): Closed test,mmm,1515826840321.c8230ae310db3c741e23f471feabd42a.
2018-01-13 07:03:28,433 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(92): Processing close of testVerifyRepJobWithRawOptions,zzz,1515826887170.8c4d6f4f764dd104ac151c211e1a22f2.
2018-01-13 07:03:28,433 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1481): Closing testVerifyRepJobWithRawOptions,zzz,1515826887170.8c4d6f4f764dd104ac151c211e1a22f2.: disabling compactions & flushes
2018-01-13 07:03:28,433 DEBUG [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1520): Updates disabled for region testVerifyRepJobWithRawOptions,zzz,1515826887170.8c4d6f4f764dd104ac151c211e1a22f2.
2018-01-13 07:03:28,433 INFO  [StoreCloserThread-testVerifyRepJobWithRawOptions,zzz,1515826887170.8c4d6f4f764dd104ac151c211e1a22f2.-1] regionserver.HStore(930): Closed fam_raw
2018-01-13 07:03:28,434 INFO  [RS:1;asf911:34432] regionserver.HRegionServer(1109): stopping server asf911.gq1.ygridcore.net,34432,1515826823377; all regions closed.
2018-01-13 07:03:28,437 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:34940 is added to blk_1073742068_1244{UCState=COMMITTED, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-2d1c80f9-c237-446d-b413-0d899e38665b:NORMAL:127.0.0.1:34940|RBW], ReplicaUC[[DISK]DS-51167812-25c5-42dd-9759-7b939eb0ef2b:NORMAL:127.0.0.1:36231|RBW]]} size 924
2018-01-13 07:03:28,437 WARN  [Close-WAL-Writer-0] asyncfs.FanOutOneBlockAsyncDFSOutputHelper(841): complete file /user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.meta.1515827004191.meta not finished, retry = 0
2018-01-13 07:03:28,437 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:36231 is added to blk_1073742068_1244 size 924
2018-01-13 07:03:28,438 DEBUG [RS_CLOSE_REGION-asf911:60093-0] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/testVerifyRepJobWithRawOptions/e00c78671344bc3f3ded24b19d7bde5e/recovered.edits/4.seqid, newSeqId=4, maxSeqId=2
2018-01-13 07:03:28,442 DEBUG [RS_CLOSE_REGION-asf911:60093-2] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/testVerifyRepJobWithRawOptions/8c4d6f4f764dd104ac151c211e1a22f2/recovered.edits/4.seqid, newSeqId=4, maxSeqId=2
2018-01-13 07:03:28,442 INFO  [RS_CLOSE_REGION-asf911:60093-0] regionserver.HRegion(1633): Closed testVerifyRepJobWithRawOptions,kkk,1515826887170.e00c78671344bc3f3ded24b19d7bde5e.
2018-01-13 07:03:28,443 DEBUG [RS_CLOSE_REGION-asf911:60093-0] handler.CloseRegionHandler(125): Closed testVerifyRepJobWithRawOptions,kkk,1515826887170.e00c78671344bc3f3ded24b19d7bde5e.
2018-01-13 07:03:28,443 INFO  [RS_CLOSE_REGION-asf911:60093-2] regionserver.HRegion(1633): Closed testVerifyRepJobWithRawOptions,zzz,1515826887170.8c4d6f4f764dd104ac151c211e1a22f2.
2018-01-13 07:03:28,443 DEBUG [RS_CLOSE_REGION-asf911:60093-2] handler.CloseRegionHandler(125): Closed testVerifyRepJobWithRawOptions,zzz,1515826887170.8c4d6f4f764dd104ac151c211e1a22f2.
2018-01-13 07:03:28,544 DEBUG [RS:1;asf911:34432] wal.AbstractFSWAL(861): Moved 4 WAL file(s) to /user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs
2018-01-13 07:03:28,544 INFO  [RS:1;asf911:34432] wal.AbstractFSWAL(864): Closed WAL: AsyncFSWAL asf911.gq1.ygridcore.net%2C34432%2C1515826823377.meta:.meta(num 1515827004191)
2018-01-13 07:03:28,546 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:36231 is added to blk_1073742067_1243{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e:NORMAL:127.0.0.1:34940|RBW], ReplicaUC[[DISK]DS-f5839f39-099d-419b-8c6f-32dec519a9b9:NORMAL:127.0.0.1:36231|RBW]]} size 0
2018-01-13 07:03:28,547 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:34940 is added to blk_1073742067_1243{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e:NORMAL:127.0.0.1:34940|RBW], ReplicaUC[[DISK]DS-f5839f39-099d-419b-8c6f-32dec519a9b9:NORMAL:127.0.0.1:36231|RBW]]} size 0
2018-01-13 07:03:28,551 DEBUG [RS:1;asf911:34432] wal.AbstractFSWAL(861): Moved 1 WAL file(s) to /user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs
2018-01-13 07:03:28,552 INFO  [RS:1;asf911:34432] wal.AbstractFSWAL(864): Closed WAL: AsyncFSWAL asf911.gq1.ygridcore.net%2C34432%2C1515826823377:(num 1515827004160)
2018-01-13 07:03:28,552 DEBUG [RS:1;asf911:34432] ipc.AbstractRpcClient(483): Stopping rpc client
2018-01-13 07:03:28,552 INFO  [RS:1;asf911:34432] regionserver.Leases(147): RS:1;asf911:34432 closing leases
2018-01-13 07:03:28,552 INFO  [RS:1;asf911:34432] regionserver.Leases(150): RS:1;asf911:34432 closed leases
2018-01-13 07:03:28,552 INFO  [RS:1;asf911:34432] hbase.ChoreService(327): Chore service for: asf911.gq1.ygridcore.net,34432,1515826823377 had [[ScheduledChore: Name: MovedRegionsCleaner for region asf911.gq1.ygridcore.net,34432,1515826823377 Period: 120000 Unit: MILLISECONDS], [ScheduledChore: Name: CompactionThroughputTuner Period: 60000 Unit: MILLISECONDS], [ScheduledChore: Name: CompactedHFilesCleaner Period: 120000 Unit: MILLISECONDS]] on shutdown
2018-01-13 07:03:28,553 INFO  [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] regionserver.LogRoller(195): LogRoller exiting.
2018-01-13 07:03:28,557 INFO  [RS:1;asf911:34432] regionserver.ReplicationSource(470): Closing source 2 because: Region server is closing
2018-01-13 07:03:28,557 WARN  [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] util.Threads(150): sleep interrupted
java.lang.InterruptedException: sleep interrupted
	at java.lang.Thread.sleep(Native Method)
	at org.apache.hadoop.hbase.util.Threads.sleep(Threads.java:148)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:159)
2018-01-13 07:03:28,577 INFO  [RS:1;asf911:34432] zookeeper.ReadOnlyZKClient(342): Close zookeeper connection 0x2de51bc2 to localhost:65105
2018-01-13 07:03:28,577 DEBUG [RS:1;asf911:34432] ipc.AbstractRpcClient(483): Stopping rpc client
2018-01-13 07:03:28,578 INFO  [RS:1;asf911:34432] regionserver.ReplicationSource(495): ReplicationSourceWorker RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2 terminated
2018-01-13 07:03:28,578 INFO  [RS:1;asf911:34432] ipc.NettyRpcServer(143): Stopping server on 0
2018-01-13 07:03:28,595 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): regionserver:60093-0x160ee519be80004, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeDeleted, state=SyncConnected, path=/1/rs/asf911.gq1.ygridcore.net,34432,1515826823377
2018-01-13 07:03:28,595 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): regionserver:34432-0x160ee519be80005, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeDeleted, state=SyncConnected, path=/1/rs/asf911.gq1.ygridcore.net,34432,1515826823377
2018-01-13 07:03:28,595 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeDeleted, state=SyncConnected, path=/1/rs/asf911.gq1.ygridcore.net,34432,1515826823377
2018-01-13 07:03:28,595 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): regionserver:60093-0x160ee519be80004, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeChildrenChanged, state=SyncConnected, path=/1/rs
2018-01-13 07:03:28,596 INFO  [main-EventThread] master.RegionServerTracker(122): RegionServer ephemeral node deleted, processing expiration [asf911.gq1.ygridcore.net,34432,1515826823377]
2018-01-13 07:03:28,596 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): regionserver:34432-0x160ee519be80005, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeChildrenChanged, state=SyncConnected, path=/1/rs
2018-01-13 07:03:28,602 INFO  [main-EventThread] master.ServerManager(593): Cluster shutdown set; asf911.gq1.ygridcore.net,34432,1515826823377 expired; onlineServers=1
2018-01-13 07:03:28,602 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeChildrenChanged, state=SyncConnected, path=/1/rs
2018-01-13 07:03:28,603 INFO  [RS:1;asf911:34432] regionserver.HRegionServer(1155): stopping server asf911.gq1.ygridcore.net,34432,1515826823377; zookeeper connection closed.
2018-01-13 07:03:28,603 INFO  [RS:1;asf911:34432] regionserver.HRegionServer(1157): RS:1;asf911:34432 exiting
2018-01-13 07:03:28,604 INFO  [Shutdown of org.apache.hadoop.hbase.fs.HFileSystem@20408833] hbase.MiniHBaseCluster$SingleFileSystemShutdownThread(209): Hook closing fs=org.apache.hadoop.hbase.fs.HFileSystem@20408833
2018-01-13 07:03:28,604 DEBUG [main-EventThread] zookeeper.ZKUtil(355): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Set watcher on existing znode=/1/rs/asf911.gq1.ygridcore.net,60093,1515826823254
2018-01-13 07:03:28,670 INFO  [RS_CLOSE_REGION-asf911:60093-1] regionserver.DefaultStoreFlusher(87): Flushed, sequenceid=6, memsize=82, hasBloomFilter=true, into tmp file hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/testVerifyRepJobWithRawOptions/e209997496b8c4b9175598ad3b9bfc86/.tmp/fam_raw/6178cf7b4b174482b4158239036f2667
2018-01-13 07:03:28,680 INFO  [RS_CLOSE_REGION-asf911:60093-1] regionserver.StoreFileReader(496): Loaded Delete Family Bloom (CompoundBloomFilter) metadata for 6178cf7b4b174482b4158239036f2667
2018-01-13 07:03:28,681 DEBUG [RS_CLOSE_REGION-asf911:60093-1] regionserver.HRegionFileSystem(463): Committing store file hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/testVerifyRepJobWithRawOptions/e209997496b8c4b9175598ad3b9bfc86/.tmp/fam_raw/6178cf7b4b174482b4158239036f2667 as hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/testVerifyRepJobWithRawOptions/e209997496b8c4b9175598ad3b9bfc86/fam_raw/6178cf7b4b174482b4158239036f2667
2018-01-13 07:03:28,687 INFO  [RS_CLOSE_REGION-asf911:60093-1] regionserver.StoreFileReader(496): Loaded Delete Family Bloom (CompoundBloomFilter) metadata for 6178cf7b4b174482b4158239036f2667
2018-01-13 07:03:28,687 INFO  [RS_CLOSE_REGION-asf911:60093-1] regionserver.HStore(1028): Added hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/testVerifyRepJobWithRawOptions/e209997496b8c4b9175598ad3b9bfc86/fam_raw/6178cf7b4b174482b4158239036f2667, entries=1, sequenceid=6, filesize=4.9 K
2018-01-13 07:03:28,689 INFO  [RS_CLOSE_REGION-asf911:60093-1] regionserver.HRegion(2712): Finished memstore flush of ~82 B/82, currentsize=0 B/0 for region testVerifyRepJobWithRawOptions,qqq,1515826887170.e209997496b8c4b9175598ad3b9bfc86. in 435ms, sequenceid=6, compaction requested=false
2018-01-13 07:03:28,693 INFO  [StoreCloserThread-testVerifyRepJobWithRawOptions,qqq,1515826887170.e209997496b8c4b9175598ad3b9bfc86.-1] regionserver.HStore(930): Closed fam_raw
2018-01-13 07:03:28,698 DEBUG [RS_CLOSE_REGION-asf911:60093-1] wal.WALSplitter(675): Wrote file=hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/data/default/testVerifyRepJobWithRawOptions/e209997496b8c4b9175598ad3b9bfc86/recovered.edits/9.seqid, newSeqId=9, maxSeqId=2
2018-01-13 07:03:28,699 INFO  [RS_CLOSE_REGION-asf911:60093-1] regionserver.HRegion(1633): Closed testVerifyRepJobWithRawOptions,qqq,1515826887170.e209997496b8c4b9175598ad3b9bfc86.
2018-01-13 07:03:28,699 DEBUG [RS_CLOSE_REGION-asf911:60093-1] handler.CloseRegionHandler(125): Closed testVerifyRepJobWithRawOptions,qqq,1515826887170.e209997496b8c4b9175598ad3b9bfc86.
2018-01-13 07:03:28,824 INFO  [RS:0;asf911:60093] regionserver.HRegionServer(1109): stopping server asf911.gq1.ygridcore.net,60093,1515826823254; all regions closed.
2018-01-13 07:03:28,826 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:34940 is added to blk_1073742066_1242{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-51167812-25c5-42dd-9759-7b939eb0ef2b:NORMAL:127.0.0.1:36231|RBW], ReplicaUC[[DISK]DS-2d1c80f9-c237-446d-b413-0d899e38665b:NORMAL:127.0.0.1:34940|RBW]]} size 0
2018-01-13 07:03:28,830 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:36231 is added to blk_1073742066_1242{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-51167812-25c5-42dd-9759-7b939eb0ef2b:NORMAL:127.0.0.1:36231|RBW], ReplicaUC[[DISK]DS-2d1c80f9-c237-446d-b413-0d899e38665b:NORMAL:127.0.0.1:34940|RBW]]} size 0
2018-01-13 07:03:28,839 DEBUG [RS:0;asf911:60093] wal.AbstractFSWAL(861): Moved 4 WAL file(s) to /user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs
2018-01-13 07:03:28,839 INFO  [RS:0;asf911:60093] wal.AbstractFSWAL(864): Closed WAL: AsyncFSWAL asf911.gq1.ygridcore.net%2C60093%2C1515826823254:(num 1515827004159)
2018-01-13 07:03:28,840 DEBUG [RS:0;asf911:60093] ipc.AbstractRpcClient(483): Stopping rpc client
2018-01-13 07:03:28,840 INFO  [RS:0;asf911:60093] regionserver.Leases(147): RS:0;asf911:60093 closing leases
2018-01-13 07:03:28,840 INFO  [RS:0;asf911:60093] regionserver.Leases(150): RS:0;asf911:60093 closed leases
2018-01-13 07:03:28,840 INFO  [RS:0;asf911:60093] hbase.ChoreService(327): Chore service for: asf911.gq1.ygridcore.net,60093,1515826823254 had [[ScheduledChore: Name: MovedRegionsCleaner for region asf911.gq1.ygridcore.net,60093,1515826823254 Period: 120000 Unit: MILLISECONDS], [ScheduledChore: Name: CompactionThroughputTuner Period: 60000 Unit: MILLISECONDS], [ScheduledChore: Name: CompactedHFilesCleaner Period: 120000 Unit: MILLISECONDS]] on shutdown
2018-01-13 07:03:28,840 INFO  [RS:0;asf911:60093] regionserver.CompactSplit(394): Waiting for Split Thread to finish...
2018-01-13 07:03:28,840 INFO  [RS:0;asf911:60093] regionserver.CompactSplit(394): Waiting for Large Compaction Thread to finish...
2018-01-13 07:03:28,840 INFO  [RS:0;asf911:60093] regionserver.CompactSplit(394): Waiting for Small Compaction Thread to finish...
2018-01-13 07:03:28,841 INFO  [RS:0;asf911:60093] regionserver.ReplicationSource(470): Closing source 2 because: Region server is closing
2018-01-13 07:03:28,845 INFO  [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] regionserver.LogRoller(195): LogRoller exiting.
2018-01-13 07:03:28,853 INFO  [RS:0;asf911:60093] zookeeper.ReadOnlyZKClient(342): Close zookeeper connection 0x3f30dc52 to localhost:65105
2018-01-13 07:03:28,853 DEBUG [RS:0;asf911:60093] ipc.AbstractRpcClient(483): Stopping rpc client
2018-01-13 07:03:28,854 INFO  [RS:0;asf911:60093] regionserver.ReplicationSource(495): ReplicationSourceWorker RS_REFRESH_PEER-asf911:60093-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C60093%2C1515826823254,2 terminated
2018-01-13 07:03:28,854 INFO  [RS:0;asf911:60093] ipc.NettyRpcServer(143): Stopping server on 0
2018-01-13 07:03:28,870 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): regionserver:60093-0x160ee519be80004, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeDeleted, state=SyncConnected, path=/1/rs/asf911.gq1.ygridcore.net,60093,1515826823254
2018-01-13 07:03:28,870 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeDeleted, state=SyncConnected, path=/1/rs/asf911.gq1.ygridcore.net,60093,1515826823254
2018-01-13 07:03:28,870 INFO  [main-EventThread] master.RegionServerTracker(122): RegionServer ephemeral node deleted, processing expiration [asf911.gq1.ygridcore.net,60093,1515826823254]
2018-01-13 07:03:28,870 INFO  [main-EventThread] master.ServerManager(593): Cluster shutdown set; asf911.gq1.ygridcore.net,60093,1515826823254 expired; onlineServers=0
2018-01-13 07:03:28,870 INFO  [main-EventThread] regionserver.HRegionServer(2158): ***** STOPPING region server 'asf911.gq1.ygridcore.net,53894,1515826821951' *****
2018-01-13 07:03:28,870 INFO  [main-EventThread] regionserver.HRegionServer(2172): STOPPED: Cluster shutdown set; onlineServer=0
2018-01-13 07:03:28,870 DEBUG [M:0;asf911:53894] ipc.AbstractRpcClient(200): Codec=org.apache.hadoop.hbase.codec.KeyValueCodec@328dc77d, compressor=null, tcpKeepAlive=true, tcpNoDelay=true, connectTO=10000, readTO=20000, writeTO=60000, minIdleTimeBeforeClose=120000, maxRetries=0, fallbackAllowed=true, bind address=asf911.gq1.ygridcore.net/67.195.81.155:0
2018-01-13 07:03:28,871 INFO  [M:0;asf911:53894] regionserver.HRegionServer(1081): stopping server asf911.gq1.ygridcore.net,53894,1515826821951
2018-01-13 07:03:28,871 DEBUG [M:0;asf911:53894] zookeeper.MetaTableLocator(641): Stopping MetaTableLocator
2018-01-13 07:03:28,871 INFO  [M:0;asf911:53894] zookeeper.ReadOnlyZKClient(342): Close zookeeper connection 0x39045593 to localhost:65105
2018-01-13 07:03:28,871 DEBUG [M:0;asf911:53894] ipc.AbstractRpcClient(483): Stopping rpc client
2018-01-13 07:03:28,871 INFO  [M:0;asf911:53894] regionserver.HRegionServer(1109): stopping server asf911.gq1.ygridcore.net,53894,1515826821951; all regions closed.
2018-01-13 07:03:28,871 DEBUG [M:0;asf911:53894] ipc.AbstractRpcClient(483): Stopping rpc client
2018-01-13 07:03:28,872 INFO  [M:0;asf911:53894] hbase.ChoreService(327): Chore service for: asf911.gq1.ygridcore.net,53894,1515826821951 had [[ScheduledChore: Name: asf911.gq1.ygridcore.net,53894,1515826821951-RegionNormalizerChore Period: 300000 Unit: MILLISECONDS], [ScheduledChore: Name: CatalogJanitor-asf911:53894 Period: 300000 Unit: MILLISECONDS], [ScheduledChore: Name: asf911.gq1.ygridcore.net,53894,1515826821951-ExpiredMobFileCleanerChore Period: 86400 Unit: SECONDS], [ScheduledChore: Name: asf911.gq1.ygridcore.net,53894,1515826821951-MobCompactionChore Period: 604800 Unit: SECONDS], [ScheduledChore: Name: LogsCleaner Period: 60000 Unit: MILLISECONDS], [ScheduledChore: Name: HFileCleaner Period: 60000 Unit: MILLISECONDS], [ScheduledChore: Name: asf911.gq1.ygridcore.net,53894,1515826821951-BalancerChore Period: 300000 Unit: MILLISECONDS], [ScheduledChore: Name: ReplicationMetaCleaner Period: 60000 Unit: MILLISECONDS], [ScheduledChore: Name: asf911.gq1.ygridcore.net,53894,1515826821951-ClusterStatusChore Period: 60000 Unit: MILLISECONDS]] on shutdown
2018-01-13 07:03:28,872 INFO  [M:0;asf911:53894] master.MasterMobCompactionThread(175): Waiting for Mob Compaction Thread to finish...
2018-01-13 07:03:28,872 INFO  [M:0;asf911:53894] master.MasterMobCompactionThread(175): Waiting for Region Server Mob Compaction Thread to finish...
2018-01-13 07:03:28,872 DEBUG [M:0;asf911:53894] master.HMaster(1187): Stopping service threads
2018-01-13 07:03:28,878 INFO  [RS:0;asf911:60093] regionserver.HRegionServer(1155): stopping server asf911.gq1.ygridcore.net,60093,1515826823254; zookeeper connection closed.
2018-01-13 07:03:28,878 INFO  [RS:0;asf911:60093] regionserver.HRegionServer(1157): RS:0;asf911:60093 exiting
2018-01-13 07:03:28,878 INFO  [Shutdown of org.apache.hadoop.hbase.fs.HFileSystem@1338366b] hbase.MiniHBaseCluster$SingleFileSystemShutdownThread(209): Hook closing fs=org.apache.hadoop.hbase.fs.HFileSystem@1338366b
2018-01-13 07:03:28,879 INFO  [main] util.JVMClusterUtil(322): Shutdown of 1 master(s) and 2 regionserver(s) complete
2018-01-13 07:03:28,886 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeChildrenChanged, state=SyncConnected, path=/1/rs
2018-01-13 07:03:28,886 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=NodeDeleted, state=SyncConnected, path=/1/master
2018-01-13 07:03:28,886 DEBUG [M:0;asf911:53894] zookeeper.RecoverableZooKeeper(176): Node /1/master already deleted, retry=false
2018-01-13 07:03:28,887 ERROR [M:0;asf911:53894] master.ActiveMasterManager(280): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Error deleting our own master address node
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /1/master
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:111)
	at org.apache.zookeeper.KeeperException.create(KeeperException.java:51)
	at org.apache.zookeeper.ZooKeeper.delete(ZooKeeper.java:873)
	at org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.delete(RecoverableZooKeeper.java:166)
	at org.apache.hadoop.hbase.zookeeper.ZKUtil.deleteNode(ZKUtil.java:1231)
	at org.apache.hadoop.hbase.zookeeper.ZKUtil.deleteNode(ZKUtil.java:1220)
	at org.apache.hadoop.hbase.master.ActiveMasterManager.stop(ActiveMasterManager.java:274)
	at org.apache.hadoop.hbase.master.HMaster.stopServiceThreads(HMaster.java:1196)
	at org.apache.hadoop.hbase.regionserver.HRegionServer.run(HRegionServer.java:1135)
	at org.apache.hadoop.hbase.master.HMaster.run(HMaster.java:564)
	at java.lang.Thread.run(Thread.java:748)
2018-01-13 07:03:28,887 INFO  [M:0;asf911:53894] assignment.AssignmentManager(236): Stopping assignment manager
2018-01-13 07:03:28,887 DEBUG [main-EventThread] zookeeper.ZKUtil(357): master:53894-0x160ee519be80003, quorum=localhost:65105, baseZNode=/1 Set watcher on znode that does not yet exist, /1/master
2018-01-13 07:03:28,887 INFO  [M:0;asf911:53894] zookeeper.ReadOnlyZKClient(342): Close zookeeper connection 0x521071e9 to localhost:65105
2018-01-13 07:03:28,887 DEBUG [M:0;asf911:53894] ipc.AbstractRpcClient(483): Stopping rpc client
2018-01-13 07:03:28,887 INFO  [M:0;asf911:53894] procedure2.RemoteProcedureDispatcher(116): Stopping procedure remote dispatcher
2018-01-13 07:03:28,888 INFO  [M:0;asf911:53894] procedure2.ProcedureExecutor(572): Stopping
2018-01-13 07:03:28,888 INFO  [M:0;asf911:53894] wal.WALProcedureStore(290): Stopping the WAL Procedure Store, isAbort=false
2018-01-13 07:03:28,888 DEBUG [ProcExecWrkr-6] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,889 DEBUG [ProcExecWrkr-5] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,888 DEBUG [ProcExecWrkr-9] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,890 DEBUG [ProcExecWrkr-16] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,889 DEBUG [ProcExecWrkr-2] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,890 DEBUG [ProcExecWrkr-14] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,889 DEBUG [ProcExecWrkr-8] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,891 DEBUG [ProcExecWrkr-4] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,889 DEBUG [ProcExecWrkr-15] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,889 DEBUG [ProcExecWrkr-11] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,889 DEBUG [ProcExecWrkr-3] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,892 DEBUG [ProcExecWrkr-12] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,892 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:34940 is added to blk_1073741829_1005{UCState=COMMITTED, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-f5839f39-099d-419b-8c6f-32dec519a9b9:NORMAL:127.0.0.1:36231|RBW], ReplicaUC[[DISK]DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e:NORMAL:127.0.0.1:34940|RBW]]} size 161929
2018-01-13 07:03:28,891 DEBUG [ProcExecWrkr-1] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,891 DEBUG [ProcExecWrkr-7] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,893 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:36231 is added to blk_1073741829_1005 size 161929
2018-01-13 07:03:28,890 DEBUG [ProcExecWrkr-13] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:28,890 DEBUG [ProcExecWrkr-10] procedure2.ProcedureExecutor$WorkerThread(1754): Worker terminated.
2018-01-13 07:03:29,158 INFO  [asf911.gq1.ygridcore.net,53894,1515826821951_splitLogManager__Chore_1] hbase.ScheduledChore(180): Chore: SplitLogManager Timeout Monitor was stopped
2018-01-13 07:03:29,293 INFO  [M:0;asf911:53894] hbase.ChoreService(327): Chore service for: asf911.gq1.ygridcore.net,53894,1515826821951_splitLogManager_ had [] on shutdown
2018-01-13 07:03:29,293 INFO  [M:0;asf911:53894] flush.MasterFlushTableProcedureManager(78): stop: server shutting down.
2018-01-13 07:03:29,293 INFO  [M:0;asf911:53894] ipc.NettyRpcServer(143): Stopping server on 0
2018-01-13 07:03:29,303 DEBUG [M:0;asf911:53894] zookeeper.RecoverableZooKeeper(176): Node /1/rs/asf911.gq1.ygridcore.net,53894,1515826821951 already deleted, retry=false
2018-01-13 07:03:29,312 INFO  [M:0;asf911:53894] regionserver.HRegionServer(1155): stopping server asf911.gq1.ygridcore.net,53894,1515826821951; zookeeper connection closed.
2018-01-13 07:03:29,312 INFO  [M:0;asf911:53894] regionserver.HRegionServer(1157): M:0;asf911:53894 exiting
2018-01-13 07:03:29,312 WARN  [main] datanode.DirectoryScanner(529): DirectoryScanner: shutdown has been called
2018-01-13 07:03:29,361 INFO  [main] log.Slf4jLog(67): Stopped HttpServer2$SelectChannelConnectorWithSafeStartup@localhost:0
2018-01-13 07:03:29,462 WARN  [DataNode: [[[DISK]file:/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39/dfs/data/data3/, [DISK]file:/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39/dfs/data/data4/]]  heartbeating to localhost/127.0.0.1:44892] datanode.IncrementalBlockReportManager(132): IncrementalBlockReportManager interrupted
2018-01-13 07:03:29,462 WARN  [DataNode: [[[DISK]file:/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39/dfs/data/data3/, [DISK]file:/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39/dfs/data/data4/]]  heartbeating to localhost/127.0.0.1:44892] datanode.BPServiceActor(670): Ending block pool service for: Block pool BP-1553325018-67.195.81.155-1515826816086 (Datanode Uuid fa37f8ae-4063-4419-b02f-823a76a4c0da) service to localhost/127.0.0.1:44892
2018-01-13 07:03:29,487 WARN  [main] datanode.DirectoryScanner(529): DirectoryScanner: shutdown has been called
2018-01-13 07:03:29,497 INFO  [main] log.Slf4jLog(67): Stopped HttpServer2$SelectChannelConnectorWithSafeStartup@localhost:0
2018-01-13 07:03:29,614 WARN  [DataNode: [[[DISK]file:/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39/dfs/data/data1/, [DISK]file:/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39/dfs/data/data2/]]  heartbeating to localhost/127.0.0.1:44892] datanode.IncrementalBlockReportManager(132): IncrementalBlockReportManager interrupted
2018-01-13 07:03:29,614 WARN  [DataNode: [[[DISK]file:/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39/dfs/data/data1/, [DISK]file:/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests/hbase-mapreduce/target/test-data/70afe7de-6045-4028-998a-d7db0b3df0fb/cluster_f7a1bb24-b499-49e6-a73a-62f78425ca39/dfs/data/data2/]]  heartbeating to localhost/127.0.0.1:44892] datanode.BPServiceActor(670): Ending block pool service for: Block pool BP-1553325018-67.195.81.155-1515826816086 (Datanode Uuid 76f1583d-69ee-4f07-a1e5-7cf1c5409e62) service to localhost/127.0.0.1:44892
2018-01-13 07:03:29,654 INFO  [main] log.Slf4jLog(67): Stopped HttpServer2$SelectChannelConnectorWithSafeStartup@localhost:0
2018-01-13 07:03:29,761 ERROR [main] server.ZooKeeperServer(472): ZKShutdownHandler is not registered, so ZooKeeper server won't take any action on ERROR or SHUTDOWN server state changes
2018-01-13 07:03:29,764 INFO  [main] zookeeper.MiniZooKeeperCluster(324): Shutdown MiniZK cluster with all ZK servers
2018-01-13 07:03:29,837 INFO  [main] hbase.HBaseTestingUtility(1096): Minicluster is down
2018-01-13 07:03:30,382 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): cluster1-0x160ee519be80000, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=None, state=Disconnected, path=null
2018-01-13 07:03:30,382 DEBUG [main-EventThread] zookeeper.ZKWatcher(528): cluster1-0x160ee519be80000, quorum=localhost:65105, baseZNode=/1 Received Disconnected from ZooKeeper, ignoring
2018-01-13 07:03:30,383 DEBUG [main-EventThread] zookeeper.ZKWatcher(460): cluster2-0x160ee519be80002, quorum=localhost:65105, baseZNode=/2 Received ZooKeeper Event, type=None, state=Disconnected, path=null
2018-01-13 07:03:30,383 DEBUG [main-EventThread] zookeeper.ZKWatcher(528): cluster2-0x160ee519be80002, quorum=localhost:65105, baseZNode=/2 Received Disconnected from ZooKeeper, ignoring
2018-01-13 07:03:30,386 DEBUG [M:0;asf911:48652-EventThread] zookeeper.ZKWatcher(460): replicationLogCleaner-0x160ee519be80014, quorum=localhost:65105, baseZNode=/2 Received ZooKeeper Event, type=None, state=Disconnected, path=null
2018-01-13 07:03:30,386 DEBUG [M:0;asf911:53894-EventThread] zookeeper.ZKWatcher(460): replicationLogCleaner-0x160ee519be80009, quorum=localhost:65105, baseZNode=/1 Received ZooKeeper Event, type=None, state=Disconnected, path=null
2018-01-13 07:03:30,386 DEBUG [M:0;asf911:53894-EventThread] zookeeper.ZKWatcher(528): replicationLogCleaner-0x160ee519be80009, quorum=localhost:65105, baseZNode=/1 Received Disconnected from ZooKeeper, ignoring
2018-01-13 07:03:30,386 DEBUG [M:0;asf911:48652-EventThread] zookeeper.ZKWatcher(528): replicationLogCleaner-0x160ee519be80014, quorum=localhost:65105, baseZNode=/2 Received Disconnected from ZooKeeper, ignoring
2018-01-13 07:03:30,476 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(112): Shutdown hook starting; hbase.shutdown.hook=true; fsShutdownHook=org.apache.hadoop.fs.FileSystem$Cache$ClientFinalizer@2f67a4d3
2018-01-13 07:03:30,478 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(135): Shutdown hook finished.
2018-01-13 07:03:30,479 INFO  [Finalizer] client.ConnectionImplementation(1763): Closing master protocol: MasterService
2018-01-13 07:03:30,479 INFO  [Finalizer] zookeeper.ReadOnlyZKClient(342): Close zookeeper connection 0x7a6ea47d to localhost:65105
2018-01-13 07:03:30,479 DEBUG [Finalizer] ipc.AbstractRpcClient(483): Stopping rpc client
2018-01-13 07:03:30,481 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(112): Shutdown hook starting; hbase.shutdown.hook=true; fsShutdownHook=org.apache.hadoop.fs.FileSystem$Cache$ClientFinalizer@2f67a4d3
2018-01-13 07:03:30,481 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(135): Shutdown hook finished.
2018-01-13 07:03:30,481 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(112): Shutdown hook starting; hbase.shutdown.hook=true; fsShutdownHook=org.apache.hadoop.fs.FileSystem$Cache$ClientFinalizer@2f67a4d3
2018-01-13 07:03:30,481 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(135): Shutdown hook finished.
2018-01-13 07:03:30,481 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(112): Shutdown hook starting; hbase.shutdown.hook=true; fsShutdownHook=org.apache.hadoop.fs.FileSystem$Cache$ClientFinalizer@2f67a4d3
2018-01-13 07:03:30,481 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(135): Shutdown hook finished.
2018-01-13 07:03:30,481 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(112): Shutdown hook starting; hbase.shutdown.hook=true; fsShutdownHook=org.apache.hadoop.fs.FileSystem$Cache$ClientFinalizer@2f67a4d3
2018-01-13 07:03:30,482 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(135): Shutdown hook finished.
2018-01-13 07:03:30,485 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(112): Shutdown hook starting; hbase.shutdown.hook=true; fsShutdownHook=org.apache.hadoop.fs.FileSystem$Cache$ClientFinalizer@2f67a4d3
2018-01-13 07:03:30,485 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(121): Starting fs shutdown hook thread.
2018-01-13 07:03:30,488 INFO  [Thread-4] regionserver.ShutdownHook$ShutdownHookThread(135): Shutdown hook finished.
Standard Error
2018-01-13 07:02:30,490 INFO  [main] hbase.ResourceChecker(172): after: replication.TestReplicationSmallTests#testSmallBatch Thread=1153 (was 992) - Thread LEAK? -, OpenFileDescriptor=2910 (was 2912), MaxFileDescriptor=60000 (was 60000), SystemLoadAverage=991 (was 991), ProcessCount=276 (was 273) - ProcessCount LEAK? -, AvailableMemoryMB=7837 (was 7950)
2018-01-13 07:02:30,490 WARN  [main] hbase.ResourceChecker(135): Thread=1153 is superior to 500
2018-01-13 07:02:30,490 WARN  [main] hbase.ResourceChecker(135): OpenFileDescriptor=2910 is superior to 1024
2018-01-13 07:02:30,549 INFO  [main] hbase.ResourceChecker(148): before: replication.TestReplicationSmallTests#testDisableEnable Thread=1153, OpenFileDescriptor=2910, MaxFileDescriptor=60000, SystemLoadAverage=991, ProcessCount=276, AvailableMemoryMB=7825
2018-01-13 07:02:30,549 WARN  [main] hbase.ResourceChecker(135): Thread=1153 is superior to 500
2018-01-13 07:02:30,550 WARN  [main] hbase.ResourceChecker(135): OpenFileDescriptor=2910 is superior to 1024
2018-01-13 07:02:30,550 DEBUG [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] regionserver.LogRoller(160): WAL roll requested
2018-01-13 07:02:30,551 DEBUG [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] regionserver.LogRoller(160): WAL roll requested
2018-01-13 07:02:30,554 DEBUG [RS-EventLoopGroup-5-9] asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper(737): SASL client skipping handshake in unsecured configuration for addr = 127.0.0.1/127.0.0.1, datanodeId = DatanodeInfoWithStorage[127.0.0.1:36231,DS-f5839f39-099d-419b-8c6f-32dec519a9b9,DISK]
2018-01-13 07:02:30,555 DEBUG [RS-EventLoopGroup-5-4] asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper(737): SASL client skipping handshake in unsecured configuration for addr = 127.0.0.1/127.0.0.1, datanodeId = DatanodeInfoWithStorage[127.0.0.1:34940,DS-2d1c80f9-c237-446d-b413-0d899e38665b,DISK]
2018-01-13 07:02:30,555 DEBUG [RS-EventLoopGroup-5-7] asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper(737): SASL client skipping handshake in unsecured configuration for addr = 127.0.0.1/127.0.0.1, datanodeId = DatanodeInfoWithStorage[127.0.0.1:34940,DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e,DISK]
2018-01-13 07:02:30,556 DEBUG [RS-EventLoopGroup-5-11] asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper(737): SASL client skipping handshake in unsecured configuration for addr = 127.0.0.1/127.0.0.1, datanodeId = DatanodeInfoWithStorage[127.0.0.1:36231,DS-51167812-25c5-42dd-9759-7b939eb0ef2b,DISK]
2018-01-13 07:02:30,561 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'bbb', inclusive
2018-01-13 07:02:30,562 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ccc', inclusive
2018-01-13 07:02:30,563 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ddd', inclusive
2018-01-13 07:02:30,564 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'eee', inclusive
2018-01-13 07:02:30,565 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'fff', inclusive
2018-01-13 07:02:30,566 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ggg', inclusive
2018-01-13 07:02:30,566 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'hhh', inclusive
2018-01-13 07:02:30,567 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'iii', inclusive
2018-01-13 07:02:30,568 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'jjj', inclusive
2018-01-13 07:02:30,569 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'kkk', inclusive
2018-01-13 07:02:30,571 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'lll', inclusive
2018-01-13 07:02:30,571 INFO  [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] wal.AbstractFSWAL(705): Rolled WAL /user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826949147 with entries=0, filesize=88 B; new WAL /user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826950551
2018-01-13 07:02:30,571 DEBUG [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] wal.AbstractFSWAL(772): Create new AsyncFSWAL writer with pipeline: [DatanodeInfoWithStorage[127.0.0.1:36231,DS-51167812-25c5-42dd-9759-7b939eb0ef2b,DISK], DatanodeInfoWithStorage[127.0.0.1:34940,DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e,DISK]]
2018-01-13 07:02:30,571 INFO  [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] wal.AbstractFSWAL(667): Archiving hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826949147 to hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826949147
2018-01-13 07:02:30,572 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'mmm', inclusive
2018-01-13 07:02:30,573 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'nnn', inclusive
2018-01-13 07:02:30,574 WARN  [Close-WAL-Writer-0] asyncfs.FanOutOneBlockAsyncDFSOutputHelper(841): complete file /user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826949147 not finished, retry = 0
2018-01-13 07:02:30,574 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:36231 is added to blk_1073741932_1108{UCState=COMMITTED, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-2d1c80f9-c237-446d-b413-0d899e38665b:NORMAL:127.0.0.1:34940|RBW], ReplicaUC[[DISK]DS-51167812-25c5-42dd-9759-7b939eb0ef2b:NORMAL:127.0.0.1:36231|RBW]]} size 96
2018-01-13 07:02:30,575 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:34940 is added to blk_1073741932_1108 size 96
2018-01-13 07:02:30,576 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ooo', inclusive
2018-01-13 07:02:30,581 INFO  [RS_REFRESH_PEER-asf911:34432-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(310): Log hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826949147 was moved to hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826949147
2018-01-13 07:02:30,581 INFO  [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] wal.AbstractFSWAL(705): Rolled WAL /user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,60093,1515826823254/asf911.gq1.ygridcore.net%2C60093%2C1515826823254.1515826949143 with entries=2, filesize=3.47 KB; new WAL /user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,60093,1515826823254/asf911.gq1.ygridcore.net%2C60093%2C1515826823254.1515826950551
2018-01-13 07:02:30,582 DEBUG [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] wal.AbstractFSWAL(772): Create new AsyncFSWAL writer with pipeline: [DatanodeInfoWithStorage[127.0.0.1:36231,DS-f5839f39-099d-419b-8c6f-32dec519a9b9,DISK], DatanodeInfoWithStorage[127.0.0.1:34940,DS-2d1c80f9-c237-446d-b413-0d899e38665b,DISK]]
2018-01-13 07:02:30,583 INFO  [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] wal.AbstractFSWAL(616): Too many WALs; count=13, max=10; forcing flush of 1 regions(s): c9a63c4339cfdbeb5b4e7c5a03ddcca4
2018-01-13 07:02:30,583 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:36231 is added to blk_1073741931_1107{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e:NORMAL:127.0.0.1:34940|RBW], ReplicaUC[[DISK]DS-f5839f39-099d-419b-8c6f-32dec519a9b9:NORMAL:127.0.0.1:36231|RBW]]} size 0
2018-01-13 07:02:30,583 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:34940 is added to blk_1073741931_1107{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e:NORMAL:127.0.0.1:34940|RBW], ReplicaUC[[DISK]DS-f5839f39-099d-419b-8c6f-32dec519a9b9:NORMAL:127.0.0.1:36231|RBW]]} size 0
2018-01-13 07:02:30,583 DEBUG [RS-EventLoopGroup-5-13] asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper(737): SASL client skipping handshake in unsecured configuration for addr = 127.0.0.1/127.0.0.1, datanodeId = DatanodeInfoWithStorage[127.0.0.1:36231,DS-f5839f39-099d-419b-8c6f-32dec519a9b9,DISK]
2018-01-13 07:02:30,583 DEBUG [RS-EventLoopGroup-5-14] asyncfs.FanOutOneBlockAsyncDFSOutputSaslHelper(737): SASL client skipping handshake in unsecured configuration for addr = 127.0.0.1/127.0.0.1, datanodeId = DatanodeInfoWithStorage[127.0.0.1:34940,DS-2d1c80f9-c237-446d-b413-0d899e38665b,DISK]
2018-01-13 07:02:30,584 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ppp', inclusive
2018-01-13 07:02:30,585 INFO  [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] wal.AbstractFSWAL(705): Rolled WAL /user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.meta.1515826949212.meta with entries=0, filesize=88 B; new WAL /user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.meta.1515826950574.meta
2018-01-13 07:02:30,585 DEBUG [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] wal.AbstractFSWAL(772): Create new AsyncFSWAL writer with pipeline: [DatanodeInfoWithStorage[127.0.0.1:36231,DS-f5839f39-099d-419b-8c6f-32dec519a9b9,DISK], DatanodeInfoWithStorage[127.0.0.1:34940,DS-2d1c80f9-c237-446d-b413-0d899e38665b,DISK]]
2018-01-13 07:02:30,585 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'qqq', inclusive
2018-01-13 07:02:30,585 INFO  [regionserver/asf911.gq1.ygridcore.net/67.195.81.155:0.logRoller] wal.AbstractFSWAL(667): Archiving hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.meta.1515826949212.meta to hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.meta.1515826949212.meta
2018-01-13 07:02:30,586 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'rrr', inclusive
2018-01-13 07:02:30,587 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:34940 is added to blk_1073741933_1109{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-f5839f39-099d-419b-8c6f-32dec519a9b9:NORMAL:127.0.0.1:36231|RBW], ReplicaUC[[DISK]DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e:NORMAL:127.0.0.1:34940|RBW]]} size 0
2018-01-13 07:02:30,587 INFO  [Block report processor] blockmanagement.BlockManager(2648): BLOCK* addStoredBlock: blockMap updated: 127.0.0.1:36231 is added to blk_1073741933_1109{UCState=UNDER_CONSTRUCTION, truncateBlock=null, primaryNodeIndex=-1, replicas=[ReplicaUC[[DISK]DS-f5839f39-099d-419b-8c6f-32dec519a9b9:NORMAL:127.0.0.1:36231|RBW], ReplicaUC[[DISK]DS-9c09543c-d06a-4d02-bd1c-0333dbfeee9e:NORMAL:127.0.0.1:34940|RBW]]} size 0
2018-01-13 07:02:30,588 DEBUG [RS_REFRESH_PEER-asf911:34432-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(242): Reached the end of log hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826949147
2018-01-13 07:02:30,588 DEBUG [RS_REFRESH_PEER-asf911:60093-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C60093%2C1515826823254,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C60093%2C1515826823254,2] regionserver.WALEntryStream(214): Reached the end of WAL file 'hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,60093,1515826823254/asf911.gq1.ygridcore.net%2C60093%2C1515826823254.1515826949143'. It was not closed cleanly, so we did not parse 8 bytes of data. This is normally ok.
2018-01-13 07:02:30,588 DEBUG [RS_REFRESH_PEER-asf911:60093-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C60093%2C1515826823254,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C60093%2C1515826823254,2] regionserver.WALEntryStream(242): Reached the end of log hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,60093,1515826823254/asf911.gq1.ygridcore.net%2C60093%2C1515826823254.1515826949143
2018-01-13 07:02:30,588 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'sss', inclusive
2018-01-13 07:02:30,589 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ttt', inclusive
2018-01-13 07:02:30,590 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'uuu', inclusive
2018-01-13 07:02:30,597 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'vvv', inclusive
2018-01-13 07:02:30,599 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'www', inclusive
2018-01-13 07:02:30,600 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'xxx', inclusive
2018-01-13 07:02:30,601 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'yyy', inclusive
2018-01-13 07:02:30,601 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'zzz', inclusive
2018-01-13 07:02:30,612 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'bbb', inclusive
2018-01-13 07:02:30,613 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ccc', inclusive
2018-01-13 07:02:30,614 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ddd', inclusive
2018-01-13 07:02:30,614 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'eee', inclusive
2018-01-13 07:02:30,615 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'fff', inclusive
2018-01-13 07:02:30,616 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ggg', inclusive
2018-01-13 07:02:30,617 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'hhh', inclusive
2018-01-13 07:02:30,618 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'iii', inclusive
2018-01-13 07:02:30,619 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'jjj', inclusive
2018-01-13 07:02:30,619 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'kkk', inclusive
2018-01-13 07:02:30,620 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'lll', inclusive
2018-01-13 07:02:30,621 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'mmm', inclusive
2018-01-13 07:02:30,622 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'nnn', inclusive
2018-01-13 07:02:30,622 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ooo', inclusive
2018-01-13 07:02:30,623 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ppp', inclusive
2018-01-13 07:02:30,625 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'qqq', inclusive
2018-01-13 07:02:30,626 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'rrr', inclusive
2018-01-13 07:02:30,627 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'sss', inclusive
2018-01-13 07:02:30,628 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ttt', inclusive
2018-01-13 07:02:30,629 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'uuu', inclusive
2018-01-13 07:02:30,630 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'vvv', inclusive
2018-01-13 07:02:30,631 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'www', inclusive
2018-01-13 07:02:30,631 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'xxx', inclusive
2018-01-13 07:02:30,632 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'yyy', inclusive
2018-01-13 07:02:30,633 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'zzz', inclusive
2018-01-13 07:02:30,713 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=58102] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:30,722 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=58102] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:30,734 DEBUG [RS_REFRESH_PEER-asf911:60093-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C60093%2C1515826823254,2] regionserver.ReplicationSourceManager(528): Removing 1 logs in the list: [asf911.gq1.ygridcore.net%2C60093%2C1515826823254.1515826949143]
2018-01-13 07:02:30,747 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=58102] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:30,749 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=58102] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:30,755 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=53490] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:30,757 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=53490] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:30,763 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=33584] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:30,766 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=33584] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:30,780 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=33584] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:30,785 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=33584] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:30,797 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=53490] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:30,802 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=53490] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:30,814 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=1,queue=0,port=47858] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:30,822 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=1,queue=0,port=47858] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:30,830 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=1,queue=0,port=47858] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:30,834 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=1,queue=0,port=47858] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:30,845 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=33584] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:30,852 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=33584] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:30,859 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'bbb', inclusive
2018-01-13 07:02:30,860 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ccc', inclusive
2018-01-13 07:02:30,863 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=53490] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:30,866 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=53490] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:30,868 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ddd', inclusive
2018-01-13 07:02:30,869 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'eee', inclusive
2018-01-13 07:02:30,870 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'fff', inclusive
2018-01-13 07:02:30,871 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ggg', inclusive
2018-01-13 07:02:30,871 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'hhh', inclusive
2018-01-13 07:02:30,872 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'iii', inclusive
2018-01-13 07:02:30,873 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'jjj', inclusive
2018-01-13 07:02:30,874 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'kkk', inclusive
2018-01-13 07:02:30,875 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'lll', inclusive
2018-01-13 07:02:30,875 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'mmm', inclusive
2018-01-13 07:02:30,876 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'nnn', inclusive
2018-01-13 07:02:30,877 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ooo', inclusive
2018-01-13 07:02:30,878 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=33584] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:30,879 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ppp', inclusive
2018-01-13 07:02:30,880 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=33584] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:30,880 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'qqq', inclusive
2018-01-13 07:02:30,881 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'rrr', inclusive
2018-01-13 07:02:30,882 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'sss', inclusive
2018-01-13 07:02:30,883 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'ttt', inclusive
2018-01-13 07:02:30,883 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'uuu', inclusive
2018-01-13 07:02:30,884 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'vvv', inclusive
2018-01-13 07:02:30,885 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'www', inclusive
2018-01-13 07:02:30,886 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'xxx', inclusive
2018-01-13 07:02:30,890 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'yyy', inclusive
2018-01-13 07:02:30,891 DEBUG [main] client.ClientScanner(242): Advancing internal scanner to startKey at 'zzz', inclusive
2018-01-13 07:02:30,892 INFO  [main] replication.TestReplicationSmallTests(133): Still got 30 rows
2018-01-13 07:02:31,401 INFO  [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] master.HMaster(3347): Client=jenkins//67.195.81.155 disable replication peer, id=2
2018-01-13 07:02:31,545 DEBUG [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] procedure2.ProcedureExecutor(869): Stored pid=85, state=RUNNABLE:PRE_PEER_MODIFICATION; org.apache.hadoop.hbase.master.replication.DisablePeerProcedure
2018-01-13 07:02:31,550 DEBUG [ProcExecWrkr-15] procedure2.StateMachineProcedure(180): pid=85, state=RUNNABLE:PRE_PEER_MODIFICATION; org.apache.hadoop.hbase.master.replication.DisablePeerProcedure
2018-01-13 07:02:31,553 DEBUG [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] master.MasterRpcServices(1157): Checking to see if procedure is done pid=85
2018-01-13 07:02:31,595 DEBUG [ProcExecWrkr-15] procedure2.StateMachineProcedure(180): pid=85, state=RUNNABLE:UPDATE_PEER_STORAGE; org.apache.hadoop.hbase.master.replication.DisablePeerProcedure
2018-01-13 07:02:31,655 DEBUG [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] master.MasterRpcServices(1157): Checking to see if procedure is done pid=85
2018-01-13 07:02:31,662 DEBUG [ProcExecWrkr-15] procedure2.StateMachineProcedure(180): pid=85, state=RUNNABLE:REFRESH_PEER_ON_RS; org.apache.hadoop.hbase.master.replication.DisablePeerProcedure
2018-01-13 07:02:31,663 INFO  [ProcExecWrkr-15] procedure2.ProcedureExecutor(1499): Initialized subprocedures=[{pid=86, ppid=85, state=RUNNABLE; org.apache.hadoop.hbase.master.replication.RefreshPeerProcedure}, {pid=87, ppid=85, state=RUNNABLE; org.apache.hadoop.hbase.master.replication.RefreshPeerProcedure}]
2018-01-13 07:02:31,856 DEBUG [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] master.MasterRpcServices(1157): Checking to see if procedure is done pid=85
2018-01-13 07:02:31,877 INFO  [ProcedureDispatcherTimeoutThread] procedure.RSProcedureDispatcher(110): Using procedure batch rpc execution for serverName=asf911.gq1.ygridcore.net,60093,1515826823254 version=3145728
2018-01-13 07:02:31,879 INFO  [RS_REFRESH_PEER-asf911:60093-1] regionserver.RefreshPeerCallable(52): Received a peer change event, peerId=2, type=DISABLE_PEER
2018-01-13 07:02:31,881 INFO  [ProcedureDispatcherTimeoutThread] procedure.RSProcedureDispatcher(110): Using procedure batch rpc execution for serverName=asf911.gq1.ygridcore.net,34432,1515826823377 version=3145728
2018-01-13 07:02:31,882 INFO  [RS_REFRESH_PEER-asf911:34432-1] regionserver.RefreshPeerCallable(52): Received a peer change event, peerId=2, type=DISABLE_PEER
2018-01-13 07:02:31,906 INFO  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(182): queueId=2, ReplicationSource : 2, currentBandwidth=0
2018-01-13 07:02:31,906 INFO  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSourceManager(384): Terminate replication source for 2
2018-01-13 07:02:31,906 INFO  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(470): Closing source 2 because: Peer 2 state or config changed. Will close the previous replication source and open a new one
2018-01-13 07:02:31,913 INFO  [RS_REFRESH_PEER-asf911:60093-1] regionserver.ReplicationSource(182): queueId=2, ReplicationSource : 2, currentBandwidth=0
2018-01-13 07:02:31,913 INFO  [RS_REFRESH_PEER-asf911:60093-1] regionserver.ReplicationSourceManager(384): Terminate replication source for 2
2018-01-13 07:02:31,913 INFO  [RS_REFRESH_PEER-asf911:60093-1] regionserver.ReplicationSource(470): Closing source 2 because: Peer 2 state or config changed. Will close the previous replication source and open a new one
2018-01-13 07:02:31,931 INFO  [RS_REFRESH_PEER-asf911:34432-1] zookeeper.ReadOnlyZKClient(342): Close zookeeper connection 0x55a1faf0 to localhost:65105
2018-01-13 07:02:31,931 DEBUG [RS_REFRESH_PEER-asf911:34432-1] ipc.AbstractRpcClient(483): Stopping rpc client
2018-01-13 07:02:31,932 INFO  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(495): ReplicationSourceWorker RS_REFRESH_PEER-asf911:34432-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2 terminated
2018-01-13 07:02:31,933 WARN  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(212): WAL group asf911.gq1.ygridcore.net%2C34432%2C1515826823377 queue size: 3 exceeds value of replication.source.log.queue.warn: 2
2018-01-13 07:02:31,933 WARN  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(212): WAL group asf911.gq1.ygridcore.net%2C34432%2C1515826823377 queue size: 4 exceeds value of replication.source.log.queue.warn: 2
2018-01-13 07:02:31,933 WARN  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(212): WAL group asf911.gq1.ygridcore.net%2C34432%2C1515826823377 queue size: 5 exceeds value of replication.source.log.queue.warn: 2
2018-01-13 07:02:31,933 WARN  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(212): WAL group asf911.gq1.ygridcore.net%2C34432%2C1515826823377 queue size: 6 exceeds value of replication.source.log.queue.warn: 2
2018-01-13 07:02:31,934 WARN  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(212): WAL group asf911.gq1.ygridcore.net%2C34432%2C1515826823377 queue size: 7 exceeds value of replication.source.log.queue.warn: 2
2018-01-13 07:02:31,934 WARN  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(212): WAL group asf911.gq1.ygridcore.net%2C34432%2C1515826823377 queue size: 8 exceeds value of replication.source.log.queue.warn: 2
2018-01-13 07:02:31,935 WARN  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(212): WAL group asf911.gq1.ygridcore.net%2C34432%2C1515826823377 queue size: 9 exceeds value of replication.source.log.queue.warn: 2
2018-01-13 07:02:31,935 WARN  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(212): WAL group asf911.gq1.ygridcore.net%2C34432%2C1515826823377 queue size: 10 exceeds value of replication.source.log.queue.warn: 2
2018-01-13 07:02:31,936 WARN  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(212): WAL group asf911.gq1.ygridcore.net%2C34432%2C1515826823377 queue size: 11 exceeds value of replication.source.log.queue.warn: 2
2018-01-13 07:02:31,936 WARN  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSource(212): WAL group asf911.gq1.ygridcore.net%2C34432%2C1515826823377 queue size: 12 exceeds value of replication.source.log.queue.warn: 2
2018-01-13 07:02:31,936 INFO  [RS_REFRESH_PEER-asf911:34432-1] regionserver.ReplicationSourceManager(391): Startup replication source for 2
2018-01-13 07:02:31,937 INFO  [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] replication.RefreshPeerProcedure(132): Refresh peer 2 for DISABLE on asf911.gq1.ygridcore.net,34432,1515826823377 suceeded
2018-01-13 07:02:31,944 INFO  [RS_REFRESH_PEER-asf911:60093-1] zookeeper.ReadOnlyZKClient(342): Close zookeeper connection 0x12cdb881 to localhost:65105
2018-01-13 07:02:31,944 DEBUG [RS_REFRESH_PEER-asf911:60093-1] ipc.AbstractRpcClient(483): Stopping rpc client
2018-01-13 07:02:31,945 INFO  [RS_REFRESH_PEER-asf911:60093-1] regionserver.ReplicationSource(495): ReplicationSourceWorker RS_REFRESH_PEER-asf911:60093-0.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C60093%2C1515826823254,2 terminated
2018-01-13 07:02:31,945 INFO  [RS_REFRESH_PEER-asf911:60093-1] regionserver.ReplicationSourceManager(391): Startup replication source for 2
2018-01-13 07:02:31,946 INFO  [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] replication.RefreshPeerProcedure(132): Refresh peer 2 for DISABLE on asf911.gq1.ygridcore.net,60093,1515826823254 suceeded
2018-01-13 07:02:31,995 INFO  [ProcExecWrkr-2] procedure2.ProcedureExecutor(1613): Finish suprocedure pid=86, ppid=85, state=SUCCESS; org.apache.hadoop.hbase.master.replication.RefreshPeerProcedure
2018-01-13 07:02:31,996 INFO  [ProcExecWrkr-2] procedure2.ProcedureExecutor(1250): Finished pid=86, ppid=85, state=SUCCESS; org.apache.hadoop.hbase.master.replication.RefreshPeerProcedure in 280msec
2018-01-13 07:02:32,003 INFO  [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2] zookeeper.ReadOnlyZKClient(130): Start read only zookeeper connection 0x0a40a750 to localhost:65105, session timeout 90000 ms, retries 1, retry interval 10 ms, keep alive 60000 ms
2018-01-13 07:02:32,008 INFO  [RS_REFRESH_PEER-asf911:60093-1.replicationSource,2] zookeeper.ReadOnlyZKClient(130): Start read only zookeeper connection 0x12d1b2c3 to localhost:65105, session timeout 90000 ms, retries 1, retry interval 10 ms, keep alive 60000 ms
2018-01-13 07:02:32,037 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2] ipc.AbstractRpcClient(200): Codec=org.apache.hadoop.hbase.codec.KeyValueCodecWithTags@5306f607, compressor=null, tcpKeepAlive=true, tcpNoDelay=true, connectTO=10000, readTO=20000, writeTO=60000, minIdleTimeBeforeClose=120000, maxRetries=0, fallbackAllowed=true, bind address=null
2018-01-13 07:02:32,037 INFO  [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2] zookeeper.RecoverableZooKeeper(106): Process identifier=connection to cluster: 2 connecting to ZooKeeper ensemble=localhost:65105
2018-01-13 07:02:32,037 DEBUG [RS_REFRESH_PEER-asf911:60093-1.replicationSource,2] ipc.AbstractRpcClient(200): Codec=org.apache.hadoop.hbase.codec.KeyValueCodecWithTags@7326415e, compressor=null, tcpKeepAlive=true, tcpNoDelay=true, connectTO=10000, readTO=20000, writeTO=60000, minIdleTimeBeforeClose=120000, maxRetries=0, fallbackAllowed=true, bind address=null
2018-01-13 07:02:32,038 INFO  [RS_REFRESH_PEER-asf911:60093-1.replicationSource,2] zookeeper.RecoverableZooKeeper(106): Process identifier=connection to cluster: 2 connecting to ZooKeeper ensemble=localhost:65105
2018-01-13 07:02:32,046 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2-EventThread] zookeeper.ZKWatcher(460): connection to cluster: 20x0, quorum=localhost:65105, baseZNode=/2 Received ZooKeeper Event, type=None, state=SyncConnected, path=null
2018-01-13 07:02:32,051 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2-EventThread] zookeeper.ZKWatcher(523): connection to cluster: 2-0x160ee519be800cf connected
2018-01-13 07:02:32,054 DEBUG [RS_REFRESH_PEER-asf911:60093-1.replicationSource,2-EventThread] zookeeper.ZKWatcher(460): connection to cluster: 20x0, quorum=localhost:65105, baseZNode=/2 Received ZooKeeper Event, type=None, state=SyncConnected, path=null
2018-01-13 07:02:32,058 DEBUG [RS_REFRESH_PEER-asf911:60093-1.replicationSource,2-EventThread] zookeeper.ZKWatcher(523): connection to cluster: 2-0x160ee519be800d0 connected
2018-01-13 07:02:32,059 INFO  [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2] regionserver.ReplicationSource(437): Replicating c57babe9-d8c4-4f5d-8c8d-7884be49786a -> 087ebcfa-03fc-42dc-8fc9-c86729e7b507
2018-01-13 07:02:32,059 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2] regionserver.ReplicationSource(294): Starting up worker for wal group asf911.gq1.ygridcore.net%2C34432%2C1515826823377
2018-01-13 07:02:32,059 INFO  [RS_REFRESH_PEER-asf911:60093-1.replicationSource,2] regionserver.ReplicationSource(437): Replicating c57babe9-d8c4-4f5d-8c8d-7884be49786a -> 087ebcfa-03fc-42dc-8fc9-c86729e7b507
2018-01-13 07:02:32,059 INFO  [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2] regionserver.ReplicationSourceWALReader(118): peerClusterZnode=2, ReplicationSourceWALReaderThread : 2 inited, replicationBatchSizeCapacity=102400, replicationBatchCountCapacity=25000, replicationBatchQueueCapacity=1
2018-01-13 07:02:32,059 DEBUG [RS_REFRESH_PEER-asf911:60093-1.replicationSource,2] regionserver.ReplicationSource(294): Starting up worker for wal group asf911.gq1.ygridcore.net%2C60093%2C1515826823254
2018-01-13 07:02:32,060 INFO  [RS_REFRESH_PEER-asf911:60093-1.replicationSource,2] regionserver.ReplicationSourceWALReader(118): peerClusterZnode=2, ReplicationSourceWALReaderThread : 2 inited, replicationBatchSizeCapacity=102400, replicationBatchCountCapacity=25000, replicationBatchQueueCapacity=1
2018-01-13 07:02:32,066 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(242): Reached the end of log hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826871676
2018-01-13 07:02:32,067 INFO  [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(310): Log hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826882232 was moved to hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826882232
2018-01-13 07:02:32,072 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(242): Reached the end of log hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826882232
2018-01-13 07:02:32,074 INFO  [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(310): Log hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826887023 was moved to hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826887023
2018-01-13 07:02:32,081 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(242): Reached the end of log hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826887023
2018-01-13 07:02:32,082 INFO  [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(310): Log hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826910411 was moved to hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826910411
2018-01-13 07:02:32,137 INFO  [ProcExecWrkr-16] procedure2.ProcedureExecutor(1613): Finish suprocedure pid=87, ppid=85, state=SUCCESS; org.apache.hadoop.hbase.master.replication.RefreshPeerProcedure
2018-01-13 07:02:32,157 DEBUG [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] master.MasterRpcServices(1157): Checking to see if procedure is done pid=85
2018-01-13 07:02:32,179 INFO  [ProcExecWrkr-16] procedure2.ProcedureExecutor(1619): Finished subprocedure(s) of pid=85, state=RUNNABLE:POST_PEER_MODIFICATION; org.apache.hadoop.hbase.master.replication.DisablePeerProcedure; resume parent processing.
2018-01-13 07:02:32,179 DEBUG [ProcExecWrkr-10] procedure2.StateMachineProcedure(180): pid=85, state=RUNNABLE:POST_PEER_MODIFICATION; org.apache.hadoop.hbase.master.replication.DisablePeerProcedure
2018-01-13 07:02:32,179 INFO  [ProcExecWrkr-16] procedure2.ProcedureExecutor(1250): Finished pid=87, ppid=85, state=SUCCESS; org.apache.hadoop.hbase.master.replication.RefreshPeerProcedure in 287msec
2018-01-13 07:02:32,179 INFO  [ProcExecWrkr-10] replication.DisablePeerProcedure(64): Successfully disabled peer 2
2018-01-13 07:02:32,188 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(242): Reached the end of log hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826910411
2018-01-13 07:02:32,190 INFO  [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(310): Log hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826911299 was moved to hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826911299
2018-01-13 07:02:32,221 INFO  [ProcExecWrkr-10] procedure2.ProcedureExecutor(1250): Finished pid=85, state=SUCCESS; org.apache.hadoop.hbase.master.replication.DisablePeerProcedure in 777msec
2018-01-13 07:02:32,296 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(242): Reached the end of log hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/oldWALs/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826911299
2018-01-13 07:02:32,298 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:32,298 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(153): Failed to read stream of replication entries: java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:32,501 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:32,501 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(153): Failed to read stream of replication entries: java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:32,659 DEBUG [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] master.MasterRpcServices(1157): Checking to see if procedure is done pid=85
2018-01-13 07:02:32,662 INFO  [Time-limited test] replication.TestReplicationSmallTests(347): Row not replicated, let's wait a bit more...
2018-01-13 07:02:32,803 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:32,803 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(153): Failed to read stream of replication entries: java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:33,164 INFO  [Time-limited test] replication.TestReplicationSmallTests(347): Row not replicated, let's wait a bit more...
2018-01-13 07:02:33,205 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:33,205 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(153): Failed to read stream of replication entries: java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:33,666 INFO  [Time-limited test] replication.TestReplicationSmallTests(347): Row not replicated, let's wait a bit more...
2018-01-13 07:02:33,711 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:33,711 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(153): Failed to read stream of replication entries: java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:33,977 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(315): 0x24c3cfeb no activities for 60000 ms, close active connection. Will reconnect next time when there are new requests.
2018-01-13 07:02:33,977 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(315): 0x5d2b2738 no activities for 60000 ms, close active connection. Will reconnect next time when there are new requests.
2018-01-13 07:02:33,977 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(315): 0x6ff2d8b4 no activities for 60000 ms, close active connection. Will reconnect next time when there are new requests.
2018-01-13 07:02:33,977 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(315): 0x39abaee1 no activities for 60000 ms, close active connection. Will reconnect next time when there are new requests.
2018-01-13 07:02:34,168 INFO  [Time-limited test] replication.TestReplicationSmallTests(347): Row not replicated, let's wait a bit more...
2018-01-13 07:02:34,183 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(315): 0x0b16d077 no activities for 60000 ms, close active connection. Will reconnect next time when there are new requests.
2018-01-13 07:02:34,183 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(315): 0x6abfb7bd no activities for 60000 ms, close active connection. Will reconnect next time when there are new requests.
2018-01-13 07:02:34,193 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(315): 0x316fcb4f no activities for 60000 ms, close active connection. Will reconnect next time when there are new requests.
2018-01-13 07:02:34,193 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(315): 0x2a5ddc9c no activities for 60000 ms, close active connection. Will reconnect next time when there are new requests.
2018-01-13 07:02:34,232 INFO  [asf911:58102Replication Statistics #0] regionserver.Replication$ReplicationStatisticsThread(352): Sink: age in ms of last applied edit: 0, total replicated edits: 351
2018-01-13 07:02:34,234 INFO  [asf911:47858Replication Statistics #0] regionserver.Replication$ReplicationStatisticsThread(352): Sink: age in ms of last applied edit: 0, total replicated edits: 430
2018-01-13 07:02:34,242 INFO  [asf911:33584Replication Statistics #0] regionserver.Replication$ReplicationStatisticsThread(352): Sink: age in ms of last applied edit: 0, total replicated edits: 373
2018-01-13 07:02:34,242 INFO  [asf911:53490Replication Statistics #0] regionserver.Replication$ReplicationStatisticsThread(352): Sink: age in ms of last applied edit: 0, total replicated edits: 470
2018-01-13 07:02:34,313 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:34,313 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(153): Failed to read stream of replication entries: java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:34,370 DEBUG [ForkJoinPool-1-worker-7] cleaner.CleanerChore$CleanerTask(409): CleanerTask 11416 starts cleaning dirs and files under hdfs://localhost:50203/user/jenkins/test-data/46fa9c95-c966-48c8-b4e5-611a15bf0613/archive and itself.
2018-01-13 07:02:34,386 DEBUG [asf911.gq1.ygridcore.net,48652,1515826833139_Chore_1] cleaner.CleanerChore(223): Cleaned old files/dirs under hdfs://localhost:50203/user/jenkins/test-data/46fa9c95-c966-48c8-b4e5-611a15bf0613/archive successfully
2018-01-13 07:02:34,386 DEBUG [asf911.gq1.ygridcore.net,48652,1515826833139_Chore_1] zookeeper.ZKUtil(704): replicationLogCleaner-0x160ee519be80014, quorum=localhost:65105, baseZNode=/2 Unable to get data of znode /2/replication/rs because node does not exist (not necessarily an error)
2018-01-13 07:02:34,387 DEBUG [asf911.gq1.ygridcore.net,48652,1515826833139_Chore_1] replication.ZKReplicationQueueStorage(326): Didn't find any region server that replicates, won't prevent any deletions.
2018-01-13 07:02:34,387 DEBUG [ForkJoinPool-1-worker-7] cleaner.CleanerChore$CleanerTask(409): CleanerTask 11416 starts cleaning dirs and files under hdfs://localhost:50203/user/jenkins/test-data/46fa9c95-c966-48c8-b4e5-611a15bf0613/oldWALs and itself.
2018-01-13 07:02:34,388 DEBUG [asf911.gq1.ygridcore.net,48652,1515826833139_Chore_1] cleaner.CleanerChore(223): Cleaned old files/dirs under hdfs://localhost:50203/user/jenkins/test-data/46fa9c95-c966-48c8-b4e5-611a15bf0613/oldWALs successfully
2018-01-13 07:02:34,670 INFO  [Time-limited test] replication.TestReplicationSmallTests(347): Row not replicated, let's wait a bit more...
2018-01-13 07:02:35,016 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:35,016 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(153): Failed to read stream of replication entries: java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:35,043 INFO  [asf911:34432Replication Statistics #0] regionserver.Replication$ReplicationStatisticsThread(352): Normal source for cluster 2: Total replicated edits: 0, current progress: 
walGroup [asf911.gq1.ygridcore.net%2C34432%2C1515826823377]: currently replicating from: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826871676 at position: -1


2018-01-13 07:02:35,043 INFO  [asf911:60093Replication Statistics #0] regionserver.Replication$ReplicationStatisticsThread(352): Normal source for cluster 2: Total replicated edits: 0, current progress: 
walGroup [asf911.gq1.ygridcore.net%2C60093%2C1515826823254]: currently replicating from: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,60093,1515826823254/asf911.gq1.ygridcore.net%2C60093%2C1515826823254.1515826950551 at position: -1


2018-01-13 07:02:35,171 INFO  [Time-limited test] replication.TestReplicationSmallTests(347): Row not replicated, let's wait a bit more...
2018-01-13 07:02:35,673 INFO  [Time-limited test] replication.TestReplicationSmallTests(347): Row not replicated, let's wait a bit more...
2018-01-13 07:02:35,818 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:35,818 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(153): Failed to read stream of replication entries: java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:36,174 INFO  [Time-limited test] replication.TestReplicationSmallTests(347): Row not replicated, let's wait a bit more...
2018-01-13 07:02:36,260 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(315): 0x693cbdfc no activities for 60000 ms, close active connection. Will reconnect next time when there are new requests.
2018-01-13 07:02:36,676 INFO  [Time-limited test] replication.TestReplicationSmallTests(347): Row not replicated, let's wait a bit more...
2018-01-13 07:02:36,720 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:36,720 DEBUG [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(153): Failed to read stream of replication entries: java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:37,177 INFO  [Time-limited test] replication.TestReplicationSmallTests(347): Row not replicated, let's wait a bit more...
2018-01-13 07:02:37,322 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(315): 0x755fac2d no activities for 60000 ms, close active connection. Will reconnect next time when there are new requests.
2018-01-13 07:02:37,686 INFO  [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] master.HMaster(3341): Client=jenkins//67.195.81.155 enable replication peer, id=2
2018-01-13 07:02:37,723 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:37,723 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(156): Failed to read stream of replication entries
java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1309)
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1301)
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1317)
	at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:428)
	at org.apache.hadoop.hbase.regionserver.wal.ReaderBase.init(ReaderBase.java:64)
	at org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.init(ProtobufLogReader.java:166)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:298)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:280)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:268)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:403)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openReader(WALEntryStream.java:344)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openNextLog(WALEntryStream.java:295)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.checkReader(WALEntryStream.java:286)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:171)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:99)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:169)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:137)
2018-01-13 07:02:37,837 DEBUG [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] procedure2.ProcedureExecutor(869): Stored pid=88, state=RUNNABLE:PRE_PEER_MODIFICATION; org.apache.hadoop.hbase.master.replication.EnablePeerProcedure
2018-01-13 07:02:37,840 DEBUG [ProcExecWrkr-7] procedure2.StateMachineProcedure(180): pid=88, state=RUNNABLE:PRE_PEER_MODIFICATION; org.apache.hadoop.hbase.master.replication.EnablePeerProcedure
2018-01-13 07:02:37,844 DEBUG [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] master.MasterRpcServices(1157): Checking to see if procedure is done pid=88
2018-01-13 07:02:37,895 DEBUG [ProcExecWrkr-7] procedure2.StateMachineProcedure(180): pid=88, state=RUNNABLE:UPDATE_PEER_STORAGE; org.apache.hadoop.hbase.master.replication.EnablePeerProcedure
2018-01-13 07:02:37,946 DEBUG [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] master.MasterRpcServices(1157): Checking to see if procedure is done pid=88
2018-01-13 07:02:37,954 DEBUG [ProcExecWrkr-7] procedure2.StateMachineProcedure(180): pid=88, state=RUNNABLE:REFRESH_PEER_ON_RS; org.apache.hadoop.hbase.master.replication.EnablePeerProcedure
2018-01-13 07:02:37,954 INFO  [ProcExecWrkr-7] procedure2.ProcedureExecutor(1499): Initialized subprocedures=[{pid=89, ppid=88, state=RUNNABLE; org.apache.hadoop.hbase.master.replication.RefreshPeerProcedure}, {pid=90, ppid=88, state=RUNNABLE; org.apache.hadoop.hbase.master.replication.RefreshPeerProcedure}]
2018-01-13 07:02:38,147 DEBUG [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] master.MasterRpcServices(1157): Checking to see if procedure is done pid=88
2018-01-13 07:02:38,154 INFO  [ProcedureDispatcherTimeoutThread] procedure.RSProcedureDispatcher(110): Using procedure batch rpc execution for serverName=asf911.gq1.ygridcore.net,60093,1515826823254 version=3145728
2018-01-13 07:02:38,156 INFO  [RS_REFRESH_PEER-asf911:60093-0] regionserver.RefreshPeerCallable(52): Received a peer change event, peerId=2, type=ENABLE_PEER
2018-01-13 07:02:38,157 INFO  [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] replication.RefreshPeerProcedure(132): Refresh peer 2 for ENABLE on asf911.gq1.ygridcore.net,60093,1515826823254 suceeded
2018-01-13 07:02:38,166 INFO  [ProcedureDispatcherTimeoutThread] procedure.RSProcedureDispatcher(110): Using procedure batch rpc execution for serverName=asf911.gq1.ygridcore.net,34432,1515826823377 version=3145728
2018-01-13 07:02:38,167 INFO  [RS_REFRESH_PEER-asf911:34432-0] regionserver.RefreshPeerCallable(52): Received a peer change event, peerId=2, type=ENABLE_PEER
2018-01-13 07:02:38,168 INFO  [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] replication.RefreshPeerProcedure(132): Refresh peer 2 for ENABLE on asf911.gq1.ygridcore.net,34432,1515826823377 suceeded
2018-01-13 07:02:38,184 DEBUG [pool-450-thread-1] ipc.NettyRpcConnection(255): Connecting to asf911.gq1.ygridcore.net/67.195.81.155:33584
2018-01-13 07:02:38,184 DEBUG [pool-451-thread-1] ipc.NettyRpcConnection(255): Connecting to asf911.gq1.ygridcore.net/67.195.81.155:47858
2018-01-13 07:02:38,186 INFO  [RS-EventLoopGroup-10-13] ipc.ServerRpcConnection(527): Auth successful for jenkins.hfs.1 (auth:SIMPLE)
2018-01-13 07:02:38,186 INFO  [RS-EventLoopGroup-8-20] ipc.ServerRpcConnection(527): Auth successful for jenkins.hfs.0 (auth:SIMPLE)
2018-01-13 07:02:38,186 INFO  [RS-EventLoopGroup-10-13] ipc.ServerRpcConnection(557): Connection from 67.195.81.155 port: 42781 with version info: version: "3.0.0-SNAPSHOT" url: "git://asf911.gq1.ygridcore.net/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests" revision: "4ddfecac563384de6f087fc7fc8bc62c0362877d" user: "jenkins" date: "Sat Jan 13 06:19:51 UTC 2018" src_checksum: "d0b248ebd327a6c13d61ead0aa4930e9" version_major: 3 version_minor: 0
2018-01-13 07:02:38,186 INFO  [RS-EventLoopGroup-8-20] ipc.ServerRpcConnection(557): Connection from 67.195.81.155 port: 52433 with version info: version: "3.0.0-SNAPSHOT" url: "git://asf911.gq1.ygridcore.net/home/jenkins/jenkins-slave/workspace/HBase-Flaky-Tests" revision: "4ddfecac563384de6f087fc7fc8bc62c0362877d" user: "jenkins" date: "Sat Jan 13 06:19:51 UTC 2018" src_checksum: "d0b248ebd327a6c13d61ead0aa4930e9" version_major: 3 version_minor: 0
2018-01-13 07:02:38,187 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=33584] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:38,187 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=1,queue=0,port=47858] regionserver.ReplicationSink(235): Started replicating mutations.
2018-01-13 07:02:38,190 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=2,queue=0,port=33584] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:38,192 DEBUG [RpcServer.replication.FPBQ.Fifo.handler=1,queue=0,port=47858] regionserver.ReplicationSink(239): Finished replicating mutations.
2018-01-13 07:02:38,229 INFO  [ProcExecWrkr-1] procedure2.ProcedureExecutor(1613): Finish suprocedure pid=90, ppid=88, state=SUCCESS; org.apache.hadoop.hbase.master.replication.RefreshPeerProcedure
2018-01-13 07:02:38,229 INFO  [ProcExecWrkr-1] procedure2.ProcedureExecutor(1250): Finished pid=90, ppid=88, state=SUCCESS; org.apache.hadoop.hbase.master.replication.RefreshPeerProcedure in 207msec
2018-01-13 07:02:38,371 INFO  [ProcExecWrkr-4] procedure2.ProcedureExecutor(1613): Finish suprocedure pid=89, ppid=88, state=SUCCESS; org.apache.hadoop.hbase.master.replication.RefreshPeerProcedure
2018-01-13 07:02:38,412 INFO  [ProcExecWrkr-4] procedure2.ProcedureExecutor(1619): Finished subprocedure(s) of pid=88, state=RUNNABLE:POST_PEER_MODIFICATION; org.apache.hadoop.hbase.master.replication.EnablePeerProcedure; resume parent processing.
2018-01-13 07:02:38,412 DEBUG [ProcExecWrkr-12] procedure2.StateMachineProcedure(180): pid=88, state=RUNNABLE:POST_PEER_MODIFICATION; org.apache.hadoop.hbase.master.replication.EnablePeerProcedure
2018-01-13 07:02:38,412 INFO  [ProcExecWrkr-4] procedure2.ProcedureExecutor(1250): Finished pid=89, ppid=88, state=SUCCESS; org.apache.hadoop.hbase.master.replication.RefreshPeerProcedure in 217msec
2018-01-13 07:02:38,412 INFO  [ProcExecWrkr-12] replication.EnablePeerProcedure(64): Successfully enabled peer 2
2018-01-13 07:02:38,448 DEBUG [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] master.MasterRpcServices(1157): Checking to see if procedure is done pid=88
2018-01-13 07:02:38,470 INFO  [ProcExecWrkr-12] procedure2.ProcedureExecutor(1250): Finished pid=88, state=SUCCESS; org.apache.hadoop.hbase.master.replication.EnablePeerProcedure in 726msec
2018-01-13 07:02:38,725 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:38,725 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(156): Failed to read stream of replication entries
java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1309)
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1301)
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1317)
	at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:428)
	at org.apache.hadoop.hbase.regionserver.wal.ReaderBase.init(ReaderBase.java:64)
	at org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.init(ProtobufLogReader.java:166)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:298)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:280)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:268)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:403)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openReader(WALEntryStream.java:344)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openNextLog(WALEntryStream.java:295)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.checkReader(WALEntryStream.java:286)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:171)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:99)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:169)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:137)
2018-01-13 07:02:38,868 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(315): 0x297c9a9b no activities for 60000 ms, close active connection. Will reconnect next time when there are new requests.
2018-01-13 07:02:38,914 INFO  [ReadOnlyZKClient] zookeeper.ReadOnlyZKClient(315): 0x479ac2cb no activities for 60000 ms, close active connection. Will reconnect next time when there are new requests.
2018-01-13 07:02:38,950 DEBUG [RpcServer.default.FPBQ.Fifo.handler=3,queue=0,port=53894] master.MasterRpcServices(1157): Checking to see if procedure is done pid=88
2018-01-13 07:02:38,951 INFO  [Time-limited test] replication.TestReplicationSmallTests(358): Row not available
2018-01-13 07:02:39,232 INFO  [asf911:58102Replication Statistics #0] regionserver.Replication$ReplicationStatisticsThread(352): Sink: age in ms of last applied edit: 0, total replicated edits: 351
2018-01-13 07:02:39,234 INFO  [asf911:47858Replication Statistics #0] regionserver.Replication$ReplicationStatisticsThread(352): Sink: age in ms of last applied edit: 0, total replicated edits: 530
2018-01-13 07:02:39,242 INFO  [asf911:33584Replication Statistics #0] regionserver.Replication$ReplicationStatisticsThread(352): Sink: age in ms of last applied edit: 0, total replicated edits: 374
2018-01-13 07:02:39,242 INFO  [asf911:53490Replication Statistics #0] regionserver.Replication$ReplicationStatisticsThread(352): Sink: age in ms of last applied edit: 0, total replicated edits: 470
2018-01-13 07:02:39,453 INFO  [Time-limited test] replication.TestReplicationSmallTests(358): Row not available
2018-01-13 07:02:39,727 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:39,727 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(156): Failed to read stream of replication entries
java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1309)
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1301)
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1317)
	at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:428)
	at org.apache.hadoop.hbase.regionserver.wal.ReaderBase.init(ReaderBase.java:64)
	at org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.init(ProtobufLogReader.java:166)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:298)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:280)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:268)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:403)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openReader(WALEntryStream.java:344)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openNextLog(WALEntryStream.java:295)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.checkReader(WALEntryStream.java:286)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:171)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:99)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:169)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:137)
2018-01-13 07:02:39,955 INFO  [Time-limited test] replication.TestReplicationSmallTests(358): Row not available
2018-01-13 07:02:40,043 INFO  [asf911:34432Replication Statistics #0] regionserver.Replication$ReplicationStatisticsThread(352): Normal source for cluster 2: Total replicated edits: 1, current progress: 
walGroup [asf911.gq1.ygridcore.net%2C34432%2C1515826823377]: currently replicating from: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592 at position: 213


2018-01-13 07:02:40,044 INFO  [asf911:60093Replication Statistics #0] regionserver.Replication$ReplicationStatisticsThread(352): Normal source for cluster 2: Total replicated edits: 100, current progress: 
walGroup [asf911.gq1.ygridcore.net%2C60093%2C1515826823254]: currently replicating from: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,60093,1515826823254/asf911.gq1.ygridcore.net%2C60093%2C1515826823254.1515826950551 at position: 11868


2018-01-13 07:02:40,456 INFO  [Time-limited test] replication.TestReplicationSmallTests(358): Row not available
2018-01-13 07:02:40,879 INFO  [asf911.gq1.ygridcore.net,33584,1515826833321_Chore_2] hbase.ScheduledChore(176): Chore: MemstoreFlusherChore missed its start time
2018-01-13 07:02:40,879 INFO  [asf911.gq1.ygridcore.net,33584,1515826833321_Chore_1] hbase.ScheduledChore(176): Chore: CompactionChecker missed its start time
2018-01-13 07:02:40,879 INFO  [asf911.gq1.ygridcore.net,34432,1515826823377_Chore_2] hbase.ScheduledChore(176): Chore: MemstoreFlusherChore missed its start time
2018-01-13 07:02:40,879 INFO  [asf911.gq1.ygridcore.net,34432,1515826823377_Chore_3] hbase.ScheduledChore(176): Chore: CompactionChecker missed its start time
2018-01-13 07:02:40,879 INFO  [asf911.gq1.ygridcore.net,58102,1515826833265_Chore_1] hbase.ScheduledChore(176): Chore: MemstoreFlusherChore missed its start time
2018-01-13 07:02:40,879 INFO  [asf911.gq1.ygridcore.net,60093,1515826823254_Chore_2] hbase.ScheduledChore(176): Chore: MemstoreFlusherChore missed its start time
2018-01-13 07:02:40,879 INFO  [asf911.gq1.ygridcore.net,60093,1515826823254_Chore_3] hbase.ScheduledChore(176): Chore: CompactionChecker missed its start time
2018-01-13 07:02:40,879 INFO  [asf911.gq1.ygridcore.net,58102,1515826833265_Chore_2] hbase.ScheduledChore(176): Chore: CompactionChecker missed its start time
2018-01-13 07:02:40,880 INFO  [asf911.gq1.ygridcore.net,53490,1515826833371_Chore_2] hbase.ScheduledChore(176): Chore: MemstoreFlusherChore missed its start time
2018-01-13 07:02:40,880 INFO  [asf911.gq1.ygridcore.net,53490,1515826833371_Chore_1] hbase.ScheduledChore(176): Chore: CompactionChecker missed its start time
2018-01-13 07:02:40,881 INFO  [asf911.gq1.ygridcore.net,47858,1515826833211_Chore_3] hbase.ScheduledChore(176): Chore: MemstoreFlusherChore missed its start time
2018-01-13 07:02:40,881 INFO  [asf911.gq1.ygridcore.net,47858,1515826833211_Chore_2] hbase.ScheduledChore(176): Chore: CompactionChecker missed its start time
2018-01-13 07:02:40,883 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:40,884 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(156): Failed to read stream of replication entries
java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1309)
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1301)
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1317)
	at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:428)
	at org.apache.hadoop.hbase.regionserver.wal.ReaderBase.init(ReaderBase.java:64)
	at org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.init(ProtobufLogReader.java:166)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:298)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:280)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:268)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:403)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openReader(WALEntryStream.java:344)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openNextLog(WALEntryStream.java:295)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.checkReader(WALEntryStream.java:286)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:171)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:99)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:169)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:137)
2018-01-13 07:02:40,965 INFO  [Time-limited test] replication.TestReplicationSmallTests(358): Row not available
2018-01-13 07:02:41,467 INFO  [Time-limited test] replication.TestReplicationSmallTests(358): Row not available
2018-01-13 07:02:41,889 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:41,889 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(156): Failed to read stream of replication entries
java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1309)
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1301)
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1317)
	at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:428)
	at org.apache.hadoop.hbase.regionserver.wal.ReaderBase.init(ReaderBase.java:64)
	at org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.init(ProtobufLogReader.java:166)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:298)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:280)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:268)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:403)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openReader(WALEntryStream.java:344)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openNextLog(WALEntryStream.java:295)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.checkReader(WALEntryStream.java:286)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:171)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:99)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:169)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:137)
2018-01-13 07:02:41,968 INFO  [Time-limited test] replication.TestReplicationSmallTests(358): Row not available
2018-01-13 07:02:42,470 INFO  [Time-limited test] replication.TestReplicationSmallTests(358): Row not available
2018-01-13 07:02:42,891 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:42,891 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(156): Failed to read stream of replication entries
java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1309)
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1301)
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1317)
	at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:428)
	at org.apache.hadoop.hbase.regionserver.wal.ReaderBase.init(ReaderBase.java:64)
	at org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.init(ProtobufLogReader.java:166)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:298)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:280)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:268)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:403)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openReader(WALEntryStream.java:344)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openNextLog(WALEntryStream.java:295)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.checkReader(WALEntryStream.java:286)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:171)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:99)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:169)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:137)
2018-01-13 07:02:42,971 INFO  [Time-limited test] replication.TestReplicationSmallTests(358): Row not available
2018-01-13 07:02:43,472 INFO  [Time-limited test] replication.TestReplicationSmallTests(358): Row not available
2018-01-13 07:02:43,894 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.WALEntryStream(324): Couldn't locate log: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
2018-01-13 07:02:43,894 ERROR [RS_REFRESH_PEER-asf911:34432-1.replicationSource,2.replicationSource.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2.replicationSource.wal-reader.asf911.gq1.ygridcore.net%2C34432%2C1515826823377,2] regionserver.ReplicationSourceWALReader(156): Failed to read stream of replication entries
java.io.FileNotFoundException: File does not exist: hdfs://localhost:44892/user/jenkins/test-data/b1477742-f8b2-42df-a666-4ef44a21ad00/WALs/asf911.gq1.ygridcore.net,34432,1515826823377/asf911.gq1.ygridcore.net%2C34432%2C1515826823377.1515826913592
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1309)
	at org.apache.hadoop.hdfs.DistributedFileSystem$22.doCall(DistributedFileSystem.java:1301)
	at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
	at org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1317)
	at org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:428)
	at org.apache.hadoop.hbase.regionserver.wal.ReaderBase.init(ReaderBase.java:64)
	at org.apache.hadoop.hbase.regionserver.wal.ProtobufLogReader.init(ProtobufLogReader.java:166)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:298)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:280)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:268)
	at org.apache.hadoop.hbase.wal.WALFactory.createReader(WALFactory.java:403)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openReader(WALEntryStream.java:344)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.openNextLog(WALEntryStream.java:295)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.checkReader(WALEntryStream.java:286)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.tryAdvanceEntry(WALEntryStream.java:171)
	at org.apache.hadoop.hbase.replication.regionserver.WALEntryStream.hasNext(WALEntryStream.java:99)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.readWALEntries(ReplicationSourceWALReader.java:169)
	at org.apache.hadoop.hbase.replication.regionserver.ReplicationSourceWALReader.run(ReplicationSourceWALReader.java:
{noformat}

> TestReplicationSmallTests.testDisableEnable fails
> -------------------------------------------------
>
>                 Key: HBASE-19792
>                 URL: https://issues.apache.org/jira/browse/HBASE-19792
>             Project: HBase
>          Issue Type: Bug
>            Reporter: Duo Zhang
>




--
This message was sent by Atlassian JIRA
(v6.4.14#64029)