[jira] [Commented] (HBASE-23669) ZNodeClearer#clear() throws IllegalArgumentException

2020-02-23 Thread kangkang.guo (Jira)


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

kangkang.guo commented on HBASE-23669:
--

[~stack] Thanks for the review. Yes, I think it still needs to be fixed, 
because this is the error that occurred when the master deleted its 
corresponding znode when exiting, regardless of whether there was any region on 
the master

> ZNodeClearer#clear() throws IllegalArgumentException 
> -
>
> Key: HBASE-23669
> URL: https://issues.apache.org/jira/browse/HBASE-23669
> Project: HBase
>  Issue Type: Bug
>Affects Versions: 2.1.5
>Reporter: kangkang.guo
>Assignee: kangkang.guo
>Priority: Major
> Fix For: 2.1.5, 2.1.6, 2.1.8
>
> Attachments: HBASE-23669.branch-2.1.0001.patch, 
> HBASE-23669.master.patch
>
>
> Precondition:
> hbase.balancer.tablesOnMaster = true
> hbase.balancer.tablesOnMaster.systemTablesOnly = true
> In case of master-rs collocation HBASE_ZNODE_FILE is overwritten by 
> regionserver process in HRegionServer#handleReportForDutyResponse() here is 
> how it looks on master server:
> [hadoop@master-node data]$ cat hbase-hadoop-master.znode 
> /hbase/rs/master-node,16000,1578371505720
> When ZNodeClearer#clear(), the rsZNode is also splicing with the contents of 
> the file, causing an exception:
> java.lang.IllegalArgumentException: Invalid path string 
> "/hbase/rs//hbase/rs/master-node,16000,1578371503003" caused by empty node 
> name specified @10
>         at 
> org.apache.zookeeper.common.PathUtils.validatePath(PathUtils.java:99)
>         at org.apache.zookeeper.ZooKeeper.delete(ZooKeeper.java:851)
>         at 
> org.apache.hadoop.hbase.zookeeper.RecoverableZooKeeper.delete(RecoverableZooKeeper.java:166)
>         at 
> org.apache.hadoop.hbase.zookeeper.ZKUtil.deleteNodeFailSilent(ZKUtil.java:1259)
>         at 
> org.apache.hadoop.hbase.zookeeper.ZKUtil.deleteNodeFailSilent(ZKUtil.java:1251)
>         at org.apache.hadoop.hbase.ZNodeClearer.clear(ZNodeClearer.java:188)
>         at 
> org.apache.hadoop.hbase.master.HMasterCommandLine.run(HMasterCommandLine.java:144)
>         at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
>         at 
> org.apache.hadoop.hbase.util.ServerCommandLine.doMain(ServerCommandLine.java:149)
>         at org.apache.hadoop.hbase.master.HMaster.main(HMaster.java:3117)
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-23 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r383112056
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + 

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-23 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r383110349
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + 

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-23 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r383113149
 
 

 ##
 File path: 
hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/slowlog/TestSlowLogRecorder.java
 ##
 @@ -0,0 +1,522 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.security.PrivilegedAction;
+import java.security.PrivilegedExceptionAction;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.CellScanner;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.ipc.RpcCallback;
+import org.apache.hadoop.hbase.security.User;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.HBaseProtos;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.RPCProtos;
+import org.apache.hadoop.hbase.testclassification.MasterTests;
+import org.apache.hadoop.hbase.testclassification.MediumTests;
+import org.apache.hbase.thirdparty.com.google.protobuf.BlockingService;
+import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+import org.junit.Assert;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import 
org.apache.hbase.thirdparty.com.google.common.util.concurrent.Uninterruptibles;
+
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Tests for Online SlowLog Provider Service
+ */
+@Category({MasterTests.class, MediumTests.class})
+public class TestSlowLogRecorder {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE =
+HBaseClassTestRule.forClass(TestSlowLogRecorder.class);
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(TestSlowLogRecorder.class);
+
+  private static final HBaseTestingUtility HBASE_TESTING_UTILITY = new 
HBaseTestingUtility();
+
+  private SlowLogRecorder slowLogRecorder;
+
+  private static Configuration applySlowLogRecorderConf(int eventSize) {
+Configuration conf = HBASE_TESTING_UTILITY.getConfiguration();
+conf.setBoolean(HConstants.SLOW_LOG_BUFFER_ENABLED_KEY, true);
+conf.setInt("hbase.regionserver.slowlog.ringbuffer.size", eventSize);
+return conf;
+  }
+
+  /**
+   * confirm that for a ringbuffer of slow logs, payload on given index of 
buffer
+   * has expected elements
+   *
+   * @param i index of ringbuffer logs
+   * @param j data value that was put on index i
+   * @param slowLogPayloads list of payload retrieved from {@link 
SlowLogRecorder}
+   */
+  private void confirmPayloadParams(int i, int j, List 
slowLogPayloads) {
+Assert.assertEquals(slowLogPayloads.get(i).getClientAddress(), "client_" + 
j);
+Assert.assertEquals(slowLogPayloads.get(i).getUserName(), "userName_" + j);
+Assert.assertEquals(slowLogPayloads.get(i).getServerClass(), "class_" + j);
+  }
+
+  @Test
+  public void testOnlieSlowLogConsumption() throws Exception {
+Configuration conf = applySlowLogRecorderConf(8);
+slowLogRecorder = new SlowLogRecorder(conf);
+AdminProtos.SlowLogResponseRequest request =
+  AdminProtos.SlowLogResponseRequest.newBuilder().setLimit(15).build();
+Assert.assertEquals(slowLogRecorder.getSlowLogPayloads(request).size(), 0);
+LOG.debug("Initially ringbuffer of Slow Log records is empty");
+
+int i = 0;
+
+// add 5 records initially
+for (; i < 5; i++) {
+  RpcLogDetails rpcLogDetails =
+

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-23 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r383109014
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + 

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-23 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r383110145
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + 

[GitHub] [hbase] bharathv commented on a change in pull request #754: HBASE-22978 : Online slow response log

2020-02-23 Thread GitBox
bharathv commented on a change in pull request #754: HBASE-22978 : Online slow 
response log
URL: https://github.com/apache/hbase/pull/754#discussion_r383110494
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/slowlog/SlowLogEventHandler.java
 ##
 @@ -0,0 +1,205 @@
+/*
+ *
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hbase.regionserver.slowlog;
+
+import com.lmax.disruptor.EventHandler;
+import com.lmax.disruptor.RingBuffer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.stream.Collectors;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.hadoop.hbase.client.SlowLogParams;
+import org.apache.hadoop.hbase.ipc.RpcCall;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
+import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors;
+import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.hbase.thirdparty.com.google.common.collect.EvictingQueue;
+import org.apache.hbase.thirdparty.com.google.common.collect.Queues;
+import org.apache.hbase.thirdparty.com.google.protobuf.Message;
+
+import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
+import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos;
+import 
org.apache.hadoop.hbase.shaded.protobuf.generated.TooSlowLog.SlowLogPayload;
+
+/**
+ * Event Handler run by disruptor ringbuffer consumer
+ */
+@InterfaceAudience.Private
+class SlowLogEventHandler implements EventHandler {
+
+  private static final Logger LOG = 
LoggerFactory.getLogger(SlowLogEventHandler.class);
+
+  private final Queue queue;
+
+  SlowLogEventHandler(int eventCount) {
+EvictingQueue evictingQueue = 
EvictingQueue.create(eventCount);
+queue = Queues.synchronizedQueue(evictingQueue);
+  }
+
+  /**
+   * Called when a publisher has published an event to the {@link RingBuffer}
+   *
+   * @param event published to the {@link RingBuffer}
+   * @param sequence of the event being processed
+   * @param endOfBatch flag to indicate if this is the last event in a batch 
from
+   *   the {@link RingBuffer}
+   * @throws Exception if the EventHandler would like the exception handled 
further up the chain
+   */
+  @Override
+  public void onEvent(RingBufferEnvelope event, long sequence, boolean 
endOfBatch)
+  throws Exception {
+final RpcLogDetails rpcCallDetails = event.getPayload();
+final RpcCall rpcCall = rpcCallDetails.getRpcCall();
+final String clientAddress = rpcCallDetails.getClientAddress();
+final long responseSize = rpcCallDetails.getResponseSize();
+final String className = rpcCallDetails.getClassName();
+Descriptors.MethodDescriptor methodDescriptor = rpcCall.getMethod();
+Message param = rpcCall.getParam();
+long receiveTime = rpcCall.getReceiveTime();
+long startTime = rpcCall.getStartTime();
+long endTime = System.currentTimeMillis();
+int processingTime = (int) (endTime - startTime);
+int qTime = (int) (startTime - receiveTime);
+final SlowLogParams slowLogParams = ProtobufUtil.getSlowLogParams(param);
+int numGets = 0;
+int numMutations = 0;
+int numServiceCalls = 0;
+if (param instanceof ClientProtos.MultiRequest) {
+  ClientProtos.MultiRequest multi = (ClientProtos.MultiRequest) param;
+  for (ClientProtos.RegionAction regionAction : 
multi.getRegionActionList()) {
+for (ClientProtos.Action action : regionAction.getActionList()) {
+  if (action.hasMutation()) {
+numMutations++;
+  }
+  if (action.hasGet()) {
+numGets++;
+  }
+  if (action.hasServiceCall()) {
+numServiceCalls++;
+  }
+}
+  }
+}
+final String userName = 
rpcCall.getRequestUserName().orElse(StringUtils.EMPTY);
+final String methodDescriptorName =
+  methodDescriptor != null ? methodDescriptor.getName() : 
StringUtils.EMPTY;
+SlowLogPayload slowLogPayload = SlowLogPayload.newBuilder()
+  .setCallDetails(methodDescriptorName + 

[jira] [Commented] (HBASE-22514) Move rsgroup feature into core of HBase

2020-02-23 Thread Anoop Sam John (Jira)


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

Anoop Sam John commented on HBASE-22514:


bq. Moved rs group feature into core. Use this flag to enable or disable it.
What is the flag config key? Seems missing in RN

> Move rsgroup feature into core of HBase
> ---
>
> Key: HBASE-22514
> URL: https://issues.apache.org/jira/browse/HBASE-22514
> Project: HBase
>  Issue Type: Umbrella
>  Components: Admin, Client, rsgroup
>Reporter: Yechao Chen
>Assignee: Duo Zhang
>Priority: Major
> Attachments: HBASE-22514.master.001.patch, 
> image-2019-05-31-18-25-38-217.png
>
>
> The class RSGroupAdminClient is not public 
> we need to use java api  RSGroupAdminClient  to manager RSG 
> so  RSGroupAdminClient should be public
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (HBASE-22978) Online slow response log

2020-02-23 Thread Viraj Jasani (Jira)


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

Viraj Jasani edited comment on HBASE-22978 at 2/24/20 6:55 AM:
---

Update on the progress:

I have addressed all concerns in latest reviews, almost ready for commit to 
master branch and then will prepare branch-2 backport. Will create all follow 
up tasks soon.

[~ndimiduk] if you could take one pass, that would be great.


was (Author: vjasani):
Update on the progress:

I have addressed all concerns in latest reviews, almost ready for commit to 
master branch and then will prepare branch-2 backport. Will create all follow 
up tasks soon.

> Online slow response log
> 
>
> Key: HBASE-22978
> URL: https://issues.apache.org/jira/browse/HBASE-22978
> Project: HBase
>  Issue Type: New Feature
>  Components: Admin, Operability, regionserver, shell
>Affects Versions: 3.0.0, 2.3.0, 1.5.1
>Reporter: Andrew Kyle Purtell
>Assignee: Viraj Jasani
>Priority: Minor
> Fix For: 3.0.0, 2.3.0, 1.7.0
>
> Attachments: Screen Shot 2019-10-19 at 2.31.59 AM.png, Screen Shot 
> 2019-10-19 at 2.32.54 AM.png, Screen Shot 2019-10-19 at 2.34.11 AM.png, 
> Screen Shot 2019-10-19 at 2.36.14 AM.png
>
>
> Today when an individual RPC exceeds a configurable time bound we log a 
> complaint by way of the logging subsystem. These log lines look like:
> {noformat}
> 2019-08-30 22:10:36,195 WARN [,queue=15,port=60020] ipc.RpcServer - 
> (responseTooSlow):
> {"call":"Scan(org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ScanRequest)",
> "starttimems":1567203007549,
> "responsesize":6819737,
> "method":"Scan",
> "param":"region { type: REGION_NAME value: 
> \"tsdb,\\000\\000\\215\\f)o\\024\\302\\220\\000\\000\\000\\000\\000\\001\\000\\000\\000\\000\\000\\006\\000\\000\\000\\000\\000\\005\\000\\000",
> "processingtimems":28646,
> "client":"10.253.196.215:41116",
> "queuetimems":22453,
> "class":"HRegionServer"}
> {noformat}
> Unfortunately we often truncate the request parameters, like in the above 
> example. We do this because the human readable representation is verbose, the 
> rate of too slow warnings may be high, and the combination of these things 
> can overwhelm the log capture system. The truncation is unfortunate because 
> it eliminates much of the utility of the warnings. For example, the region 
> name, the start and end keys, and the filter hierarchy are all important 
> clues for debugging performance problems caused by moderate to low 
> selectivity queries or queries made at a high rate.
> We can maintain an in-memory ring buffer of requests that were judged to be 
> too slow in addition to the responseTooSlow logging. The in-memory 
> representation can be complete and compressed. A new admin API and shell 
> command can provide access to the ring buffer for online performance 
> debugging. A modest sizing of the ring buffer will prevent excessive memory 
> utilization for a minor performance debugging feature by limiting the total 
> number of retained records. There is some chance a high rate of requests will 
> cause information on other interesting requests to be overwritten before it 
> can be read. This is the nature of a ring buffer and an acceptable trade off.
> The write request types do not require us to retain all information submitted 
> in the request. We don't need to retain all key-values in the mutation, which 
> may be too large to comfortably retain. We only need a unique set of row 
> keys, or even a min/max range, and total counts.
> The consumers of this information will be debugging tools. We can afford to 
> apply fast compression to ring buffer entries (if codec support is 
> available), something like snappy or zstandard, and decompress on the fly 
> when servicing the retrieval API request. This will minimize the impact of 
> retaining more information about slow requests than we do today.
> This proposal is for retention of request information only, the same 
> information provided by responseTooSlow warnings. Total size of response 
> serialization, possibly also total cell or row counts, should be sufficient 
> to characterize the response.
> Optionally persist new entries added to the ring buffer into one or more 
> files in HDFS in a write-behind manner. If the HDFS writer blocks or falls 
> behind and we are unable to persist an entry before it is overwritten, that 
> is fine. Response too slow logging is best effort. If we can detect this make 
> a note of it in the log file. Provide a tool for parsing, dumping, filtering, 
> and pretty printing the slow logs written to HDFS. The tool and the shell can 
> share and reuse some utility classes and methods for accomplishing that. 
> —
> New shell commands:
> 

[jira] [Updated] (HBASE-23881) TestShadeSaslAuthenticationProvider failures

2020-02-23 Thread Bharath Vissapragada (Jira)


 [ 
https://issues.apache.org/jira/browse/HBASE-23881?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Bharath Vissapragada updated HBASE-23881:
-
Affects Version/s: 2.3.0

> TestShadeSaslAuthenticationProvider failures
> 
>
> Key: HBASE-23881
> URL: https://issues.apache.org/jira/browse/HBASE-23881
> Project: HBase
>  Issue Type: Bug
>  Components: test
>Affects Versions: 3.0.0, 2.3.0
>Reporter: Bharath Vissapragada
>Assignee: Josh Elser
>Priority: Major
>
> TestShadeSaslAuthenticationProvider now fails deterministically with the 
> following exception..
> {noformat}
> java.lang.Exception: Unexpected exception, 
> expected but 
> was
>   at 
> org.apache.hadoop.hbase.security.provider.example.TestShadeSaslAuthenticationProvider.testNegativeAuthentication(TestShadeSaslAuthenticationProvider.java:233)
> {noformat}
> The test now fails a different place than before merging HBASE-18095 because 
> the RPCs are also a part of connection setup. We might need to rewrite the 
> test..  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (HBASE-23881) TestShadeSaslAuthenticationProvider failures

2020-02-23 Thread Bharath Vissapragada (Jira)


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

Bharath Vissapragada commented on HBASE-23881:
--

Ya, something is swallowing the exception. Thanks for taking a look.

> TestShadeSaslAuthenticationProvider failures
> 
>
> Key: HBASE-23881
> URL: https://issues.apache.org/jira/browse/HBASE-23881
> Project: HBase
>  Issue Type: Bug
>  Components: test
>Affects Versions: 3.0.0
>Reporter: Bharath Vissapragada
>Assignee: Josh Elser
>Priority: Major
>
> TestShadeSaslAuthenticationProvider now fails deterministically with the 
> following exception..
> {noformat}
> java.lang.Exception: Unexpected exception, 
> expected but 
> was
>   at 
> org.apache.hadoop.hbase.security.provider.example.TestShadeSaslAuthenticationProvider.testNegativeAuthentication(TestShadeSaslAuthenticationProvider.java:233)
> {noformat}
> The test now fails a different place than before merging HBASE-18095 because 
> the RPCs are also a part of connection setup. We might need to rewrite the 
> test..  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (HBASE-22514) Move rsgroup feature into core of HBase

2020-02-23 Thread Hudson (Jira)


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

Hudson commented on HBASE-22514:


Results for branch HBASE-22514
[build #283 on 
builds.a.o|https://builds.apache.org/job/HBase%20Nightly/job/HBASE-22514/283/]: 
(x) *{color:red}-1 overall{color}*

details (if available):

(/) {color:green}+1 general checks{color}
-- For more information [see general 
report|https://builds.apache.org/job/HBase%20Nightly/job/HBASE-22514/283//General_Nightly_Build_Report/]




(x) {color:red}-1 jdk8 hadoop2 checks{color}
-- For more information [see jdk8 (hadoop2) 
report|https://builds.apache.org/job/HBase%20Nightly/job/HBASE-22514/283//JDK8_Nightly_Build_Report_(Hadoop2)/]


(x) {color:red}-1 jdk8 hadoop3 checks{color}
-- For more information [see jdk8 (hadoop3) 
report|https://builds.apache.org/job/HBase%20Nightly/job/HBASE-22514/283//JDK8_Nightly_Build_Report_(Hadoop3)/]


(/) {color:green}+1 source release artifact{color}
-- See build output for details.


(/) {color:green}+1 client integration test{color}


> Move rsgroup feature into core of HBase
> ---
>
> Key: HBASE-22514
> URL: https://issues.apache.org/jira/browse/HBASE-22514
> Project: HBase
>  Issue Type: Umbrella
>  Components: Admin, Client, rsgroup
>Reporter: Yechao Chen
>Assignee: Duo Zhang
>Priority: Major
> Attachments: HBASE-22514.master.001.patch, 
> image-2019-05-31-18-25-38-217.png
>
>
> The class RSGroupAdminClient is not public 
> we need to use java api  RSGroupAdminClient  to manager RSG 
> so  RSGroupAdminClient should be public
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [hbase-thirdparty] anoopsjohn commented on issue #14: HBASE-19146 Update Protobuf to 3.11.4

2020-02-23 Thread GitBox
anoopsjohn commented on issue #14: HBASE-19146 Update Protobuf to 3.11.4
URL: https://github.com/apache/hbase-thirdparty/pull/14#issuecomment-590154810
 
 
   Are we using these versions?  Within our server code base we use version 
built based on our 3rd party repo right?  We have even patched some of the 
classes. That should be based on some 3.x version.  So this will make the 
version what we release in HBase distro to be 3.11.4 .. Thats it right?  But 
not use the new version..  Correct @saintstack ?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] Apache-HBase commented on issue #1198: HBASE-23740 Invalid StoreFile WARN log message printed for recovered.…

2020-02-23 Thread GitBox
Apache-HBase commented on issue #1198: HBASE-23740 Invalid StoreFile WARN log 
message printed for recovered.…
URL: https://github.com/apache/hbase/pull/1198#issuecomment-590121262
 
 
   :confetti_ball: **+1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |::|--:|:|:|
   | +0 :ok: |  reexec  |   1m 10s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files 
found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any 
anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any 
@author tags.  |
   | -0 :warning: |  test4tests  |   0m  0s |  The patch doesn't appear to 
include any new or modified tests. Please justify why no new tests are needed 
for this patch. Also please list what manual steps were performed to verify 
this patch.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   6m 21s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  9s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 19s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m 36s |  branch has no errors when 
building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 46s |  master passed  |
   | +0 :ok: |  spotbugs  |   5m 23s |  Used deprecated FindBugs config; 
considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   5m 21s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   5m 46s |  the patch passed  |
   | +1 :green_heart: |  compile  |   1m 10s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 10s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 14s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace 
issues.  |
   | +1 :green_heart: |  shadedjars  |   5m 29s |  patch has no errors when 
building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  18m 22s |  Patch does not cause any 
errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   5m  4s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  | 110m 51s |  hbase-server in the patch passed.  
|
   | +1 :green_heart: |  asflicense  |   0m 29s |  The patch does not generate 
ASF License warnings.  |
   |  |   | 178m  1s |   |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | Client=19.03.6 Server=19.03.6 base: 
https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1198/2/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hbase/pull/1198 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs 
shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux a8b8bfcfa0b6 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 
08:06:28 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | 
/home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1198/out/precommit/personality/provided.sh
 |
   | git revision | master / 400b7ce954 |
   | Default Java | 1.8.0_181 |
   |  Test Results | 
https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1198/2/testReport/
 |
   | Max. process+thread count | 6304 (vs. ulimit of 1) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | 
https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1198/2/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[jira] [Commented] (HBASE-23743) Release 1.4.13

2020-02-23 Thread Hudson (Jira)


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

Hudson commented on HBASE-23743:


Results for branch branch-1.4
[build #1188 on 
builds.a.o|https://builds.apache.org/job/HBase%20Nightly/job/branch-1.4/1188/]: 
(x) *{color:red}-1 overall{color}*

details (if available):

(/) {color:green}+1 general checks{color}
-- For more information [see general 
report|https://builds.apache.org/job/HBase%20Nightly/job/branch-1.4/1188//General_Nightly_Build_Report/]


(x) {color:red}-1 jdk7 checks{color}
-- For more information [see jdk7 
report|https://builds.apache.org/job/HBase%20Nightly/job/branch-1.4/1188//JDK7_Nightly_Build_Report/]


(/) {color:green}+1 jdk8 hadoop2 checks{color}
-- For more information [see jdk8 (hadoop2) 
report|https://builds.apache.org/job/HBase%20Nightly/job/branch-1.4/1188//JDK8_Nightly_Build_Report_(Hadoop2)/]




(/) {color:green}+1 source release artifact{color}
-- See build output for details.


> Release 1.4.13
> --
>
> Key: HBASE-23743
> URL: https://issues.apache.org/jira/browse/HBASE-23743
> Project: HBase
>  Issue Type: Task
>  Components: build, community
>Affects Versions: 1.4.13
>Reporter: Sean Busbey
>Assignee: Sakthi
>Priority: Major
> Fix For: 1.4.13
>
>
> over due for 1.4.13. I'll try to use the release manager scripts this time.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Commented] (HBASE-23881) TestShadeSaslAuthenticationProvider failures

2020-02-23 Thread Josh Elser (Jira)


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

Josh Elser commented on HBASE-23881:


{noformat}
2020-02-23 13:56:50,204 WARN  [RS-EventLoopGroup-1-6] 
ipc.ServerRpcConnection(377): Auth failed for  192.168.40.150:50589: Unknown
{noformat}

It looks like something client-side is mangling the InvalidToken exception 
thrown server-side. The server sends the proper exception across, but it is not 
re-thrown in the same way client-side. Going to take some time to dig into the 
async code and unwind this.

> TestShadeSaslAuthenticationProvider failures
> 
>
> Key: HBASE-23881
> URL: https://issues.apache.org/jira/browse/HBASE-23881
> Project: HBase
>  Issue Type: Bug
>  Components: test
>Affects Versions: 3.0.0
>Reporter: Bharath Vissapragada
>Assignee: Josh Elser
>Priority: Major
>
> TestShadeSaslAuthenticationProvider now fails deterministically with the 
> following exception..
> {noformat}
> java.lang.Exception: Unexpected exception, 
> expected but 
> was
>   at 
> org.apache.hadoop.hbase.security.provider.example.TestShadeSaslAuthenticationProvider.testNegativeAuthentication(TestShadeSaslAuthenticationProvider.java:233)
> {noformat}
> The test now fails a different place than before merging HBASE-18095 because 
> the RPCs are also a part of connection setup. We might need to rewrite the 
> test..  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [hbase] virajjasani commented on issue #707: HBASE-23066 Allow cache on write during compactions when prefetching …

2020-02-23 Thread GitBox
virajjasani commented on issue #707: HBASE-23066 Allow cache on write during 
compactions when prefetching …
URL: https://github.com/apache/hbase/pull/707#issuecomment-590104533
 
 
   > is this made obsolete by #919?
   
   I believe so. Closing this PR @jacob-leblanc @ramkrish86 


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] virajjasani closed pull request #707: HBASE-23066 Allow cache on write during compactions when prefetching …

2020-02-23 Thread GitBox
virajjasani closed pull request #707: HBASE-23066 Allow cache on write during 
compactions when prefetching …
URL: https://github.com/apache/hbase/pull/707
 
 
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] Apache-HBase commented on issue #1184: HBASE-23848 Removed deprecated setStopRow from Scan

2020-02-23 Thread GitBox
Apache-HBase commented on issue #1184: HBASE-23848 Removed deprecated 
setStopRow from Scan
URL: https://github.com/apache/hbase/pull/1184#issuecomment-590103858
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |::|--:|:|:|
   | +0 :ok: |  reexec  |   1m  4s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  1s |  No case conflicting files 
found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any 
anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any 
@author tags.  |
   | +1 :green_heart: |  test4tests  |   0m  0s |  The patch appears to include 
12 new or modified test files.  |
   ||| _ master Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 38s |  Maven dependency ordering for branch  |
   | +1 :green_heart: |  mvninstall  |   5m 56s |  master passed  |
   | +1 :green_heart: |  compile  |   3m 53s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   3m 34s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  7s |  branch has no errors when 
building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   2m 41s |  master passed  |
   | +0 :ok: |  spotbugs  |   1m 39s |  Used deprecated FindBugs config; 
considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |  10m  0s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | +0 :ok: |  mvndep  |   0m 13s |  Maven dependency ordering for patch  |
   | +1 :green_heart: |  mvninstall  |   5m 32s |  the patch passed  |
   | +1 :green_heart: |  compile  |   3m 56s |  the patch passed  |
   | +1 :green_heart: |  javac  |   3m 56s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   0m 30s |  The patch passed checkstyle 
in hbase-client  |
   | +1 :green_heart: |  checkstyle  |   1m 13s |  hbase-server: The patch 
generated 0 new + 18 unchanged - 2 fixed = 18 total (was 20)  |
   | +1 :green_heart: |  checkstyle  |   0m 20s |  The patch passed checkstyle 
in hbase-mapreduce  |
   | +1 :green_heart: |  checkstyle  |   0m 41s |  The patch passed checkstyle 
in hbase-thrift  |
   | +1 :green_heart: |  checkstyle  |   0m 15s |  The patch passed checkstyle 
in hbase-backup  |
   | +1 :green_heart: |  checkstyle  |   0m 16s |  hbase-it: The patch 
generated 0 new + 34 unchanged - 1 fixed = 34 total (was 35)  |
   | +1 :green_heart: |  checkstyle  |   0m 15s |  The patch passed checkstyle 
in hbase-rest  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace 
issues.  |
   | +1 :green_heart: |  shadedjars  |   5m  6s |  patch has no errors when 
building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  17m 30s |  Patch does not cause any 
errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   2m 39s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |  10m 43s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |   0m 56s |  hbase-client in the patch passed.  
|
   | -1 :x: |  unit  |  90m 56s |  hbase-server in the patch failed.  |
   | -1 :x: |  unit  |   9m 19s |  hbase-mapreduce in the patch failed.  |
   | +1 :green_heart: |  unit  |   2m 42s |  hbase-thrift in the patch passed.  
|
   | +1 :green_heart: |  unit  |   8m 43s |  hbase-backup in the patch passed.  
|
   | +1 :green_heart: |  unit  |   1m  4s |  hbase-it in the patch passed.  |
   | -1 :x: |  unit  |   3m 24s |  hbase-rest in the patch failed.  |
   | +1 :green_heart: |  asflicense  |   3m  0s |  The patch does not generate 
ASF License warnings.  |
   |  |   | 210m  8s |   |
   
   
   | Reason | Tests |
   |---:|:--|
   | Failed junit tests | hadoop.hbase.regionserver.TestScanner |
   |   | hadoop.hbase.regionserver.TestHRegionWithInMemoryFlush |
   |   | hadoop.hbase.client.TestFromClientSideWithCoprocessor |
   |   | hadoop.hbase.client.TestTableSnapshotScanner |
   |   | hadoop.hbase.filter.TestFilter |
   |   | hadoop.hbase.regionserver.TestHRegion |
   |   | hadoop.hbase.regionserver.TestSeekOptimizations |
   |   | hadoop.hbase.mapreduce.TestTableSnapshotInputFormat |
   |   | hadoop.hbase.snapshot.TestExportSnapshotV1NoCluster |
   |   | hadoop.hbase.snapshot.TestExportSnapshotV2NoCluster |
   |   | hadoop.hbase.rest.TestScannersWithFilters |
   |   | hadoop.hbase.rest.TestGetAndPutResource |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | Client=19.03.6 Server=19.03.6 base: 
https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1184/4/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hbase/pull/1184 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs 
shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 82319eceb663 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 
08:06:28 UTC 2019 x86_64 

[GitHub] [hbase] virajjasani commented on a change in pull request #1198: HBASE-23740 Invalid StoreFile WARN log message printed for recovered.…

2020-02-23 Thread GitBox
virajjasani commented on a change in pull request #1198: HBASE-23740 Invalid 
StoreFile WARN log message printed for recovered.…
URL: https://github.com/apache/hbase/pull/1198#discussion_r383029457
 
 

 ##
 File path: 
hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionFileSystem.java
 ##
 @@ -245,7 +245,11 @@ public String getStoragePolicyName(String familyName) {
 ArrayList storeFiles = new ArrayList<>(files.length);
 for (FileStatus status: files) {
   if (validate && !StoreFileInfo.isValid(status)) {
-LOG.warn("Invalid StoreFile: " + status.getPath());
+// recovered.hfiles directory is expected inside CF path when 
hbase.wal.split.to.hfile to
+// true, refer HBASE-23740
+if 
(!HConstants.RECOVERED_HFILES_DIR.equals(status.getPath().getName())) {
+  LOG.warn("Invalid StoreFile: " + status.getPath());
 
 Review comment:
   Might want to use {} for `status.getPath()`?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] Apache9 commented on issue #1165: HBASE-22514 Move rsgroup feature into core of HBase

2020-02-23 Thread GitBox
Apache9 commented on issue #1165: HBASE-22514 Move rsgroup feature into core of 
HBase
URL: https://github.com/apache/hbase/pull/1165#issuecomment-590079306
 
 
   > Unit test failures look related?
   
   Let me take a look. All the UTs are not in a good state recently, even for 
master, the flakey job list are increased a lot...


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] Apache-HBase commented on issue #1198: HBASE-23740 Invalid StoreFile WARN log message printed for recovered.…

2020-02-23 Thread GitBox
Apache-HBase commented on issue #1198: HBASE-23740 Invalid StoreFile WARN log 
message printed for recovered.…
URL: https://github.com/apache/hbase/pull/1198#issuecomment-590073588
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |::|--:|:|:|
   | +0 :ok: |  reexec  |   4m 24s |  Docker mode activated.  |
   ||| _ Prechecks _ |
   | +1 :green_heart: |  dupname  |   0m  0s |  No case conflicting files 
found.  |
   | +1 :green_heart: |  hbaseanti  |   0m  0s |  Patch does not have any 
anti-patterns.  |
   | +1 :green_heart: |  @author  |   0m  0s |  The patch does not contain any 
@author tags.  |
   | -0 :warning: |  test4tests  |   0m  0s |  The patch doesn't appear to 
include any new or modified tests. Please justify why no new tests are needed 
for this patch. Also please list what manual steps were performed to verify 
this patch.  |
   ||| _ master Compile Tests _ |
   | +1 :green_heart: |  mvninstall  |   6m  8s |  master passed  |
   | +1 :green_heart: |  compile  |   1m  0s |  master passed  |
   | +1 :green_heart: |  checkstyle  |   1m 14s |  master passed  |
   | +1 :green_heart: |  shadedjars  |   5m  8s |  branch has no errors when 
building our shaded downstream artifacts.  |
   | +1 :green_heart: |  javadoc  |   0m 37s |  master passed  |
   | +0 :ok: |  spotbugs  |   4m 52s |  Used deprecated FindBugs config; 
considering switching to SpotBugs.  |
   | +1 :green_heart: |  findbugs  |   4m 50s |  master passed  |
   ||| _ Patch Compile Tests _ |
   | -1 :x: |  mvninstall  |   1m 53s |  root in the patch failed.  |
   | +1 :green_heart: |  compile  |   1m 37s |  the patch passed  |
   | +1 :green_heart: |  javac  |   1m 37s |  the patch passed  |
   | +1 :green_heart: |  checkstyle  |   1m 41s |  the patch passed  |
   | +1 :green_heart: |  whitespace  |   0m  0s |  The patch has no whitespace 
issues.  |
   | +1 :green_heart: |  shadedjars  |   5m 54s |  patch has no errors when 
building our shaded downstream artifacts.  |
   | +1 :green_heart: |  hadoopcheck  |  20m 44s |  Patch does not cause any 
errors with Hadoop 2.8.5 2.9.2 or 3.1.2.  |
   | +1 :green_heart: |  javadoc  |   0m 35s |  the patch passed  |
   | +1 :green_heart: |  findbugs  |   4m 42s |  the patch passed  |
   ||| _ Other Tests _ |
   | +1 :green_heart: |  unit  |  91m 26s |  hbase-server in the patch passed.  
|
   | +1 :green_heart: |  asflicense  |   0m 26s |  The patch does not generate 
ASF License warnings.  |
   |  |   | 159m 23s |   |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | Client=19.03.6 Server=19.03.6 base: 
https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1198/1/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hbase/pull/1198 |
   | Optional Tests | dupname asflicense javac javadoc unit spotbugs findbugs 
shadedjars hadoopcheck hbaseanti checkstyle compile |
   | uname | Linux 9206c6f7cca7 4.15.0-74-generic #84-Ubuntu SMP Thu Dec 19 
08:06:28 UTC 2019 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | 
/home/jenkins/jenkins-slave/workspace/Base-PreCommit-GitHub-PR_PR-1198/out/precommit/personality/provided.sh
 |
   | git revision | master / 400b7ce954 |
   | Default Java | 1.8.0_181 |
   | mvninstall | 
https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1198/1/artifact/out/patch-mvninstall-root.txt
 |
   |  Test Results | 
https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1198/1/testReport/
 |
   | Max. process+thread count | 6563 (vs. ulimit of 1) |
   | modules | C: hbase-server U: hbase-server |
   | Console output | 
https://builds.apache.org/job/HBase-PreCommit-GitHub-PR/job/PR-1198/1/console |
   | versions | git=2.11.0 maven=2018-06-17T18:33:14Z) findbugs=3.1.11 |
   | Powered by | Apache Yetus 0.11.1 https://yetus.apache.org |
   
   
   This message was automatically generated.
   
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master by encoded region name is O(n)

2020-02-23 Thread GitBox
mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master 
by encoded region name is O(n)
URL: https://github.com/apache/hbase/pull/1193#issuecomment-590070993
 
 
   I think the current code may have different values for the two maps.
   What do you think about changing it like below?
   
   ```java
   // ...
   
   private final Object regionsMapLock = new Object();
   
   // ...
   
   RegionStateNode createRegionStateNode(RegionInfo regionInfo) {
 synchronized (regionsMapLock) {
   RegionStateNode node = 
regionsMap.computeIfAbsent(regionInfo.getRegionName(), key -> new 
RegionStateNod(regionInfo, regionInTransition));
   encodedRegionsMap.putIfAbsent(node.getRegionInfo().getEncodedName(), 
node);
   return node;
 }
   }
   ```
   
   Also there is a remove logic, it seems safe to use a synchronized.
   ```java
   public void deleteRegion(final RegionInfo regionInfo) {
 synchronized (regionsMapLock) {
   RegionStateNode removeNode = 
regionsMap.remove(regionInfo.getRegionName());
   encodedRegionsMap.remove(regionInfo.getEncodedName(), removeNode);
 }
   
 // ...
   }
   ```


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master by encoded region name is O(n)

2020-02-23 Thread GitBox
mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master 
by encoded region name is O(n)
URL: https://github.com/apache/hbase/pull/1193#issuecomment-590070993
 
 
   I think the current code may have different values for the two maps.
   What do you think about changing it like below?
   
   ```java
   RegionStateNode createRegionStateNode(RegionInfo regionInfo) {
 synchronized (regionsMapLock) {
   RegionStateNode node = 
regionsMap.computeIfAbsent(regionInfo.getRegionName(), key -> new 
RegionStateNod(regionInfo, regionInTransition));
   encodedRegionsMap.putIfAbsent(node.getRegionInfo().getEncodedName(), 
node);
   return node;
 }
   }
   ```
   
   Also there is a remove logic, it seems safe to use a synchronized.
   ```java
   public void deleteRegion(final RegionInfo regionInfo) {
 synchronized (regionsMapLock) {
   RegionStateNode removeNode = 
regionsMap.remove(regionInfo.getRegionName());
   encodedRegionsMap.remove(regionInfo.getEncodedName(), removeNode);
 }
   
 // ...
   }
   ```


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master by encoded region name is O(n)

2020-02-23 Thread GitBox
mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master 
by encoded region name is O(n)
URL: https://github.com/apache/hbase/pull/1193#issuecomment-590070993
 
 
   I think the current code may have different values for the two maps.
   What do you think about changing it like below?
   
   ```java
   RegionStateNode createRegionStateNode(RegionInfo regionInfo) {
 synchronized (regionsMapLock) {
   RegionStateNode node = 
regionsMap.computeIfAbsent(regionInfo.getRegionName(), key -> new 
RegionStateNod(regionInfo, regionInTransition));
   encodedRegionsMap.putIfAbsent(node.getRegionInfo().getEncodedName(), 
node);
   return node;
 }
   }
   ```
   
   Also there is a remove logic, it seems safe to use a synchronized.
   ```java
   public void deleteRegion(final RegionInfo regionInfo) {
 synchronized (regionsMapLock) {
   regionsMap.remove(regionInfo.getRegionName());
   encodedRegionsMap.remove(regionInfo.getEncodedName());
 }
   
 // ...
   }
   ```


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master by encoded region name is O(n)

2020-02-23 Thread GitBox
mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master 
by encoded region name is O(n)
URL: https://github.com/apache/hbase/pull/1193#issuecomment-590070993
 
 
   I think the current code may have different values for the two maps.
   What do you think about changing it like below?
   
   ```java
   RegionStateNode createRegionStateNode(RegionInfo regionInfo) {
 RegionStateNode node = 
regionsMap.computeIfAbsent(regionInfo.getRegionName(), key -> new 
RegionStateNod(regionInfo, regionInTransition));
 encodedRegionsMap.putIfAbsent(node.getRegionInfo().getEncodedName(), node);
 return node;
   }
   ```
   
   Also there is a remove logic, it seems safe to use a synchronized.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master by encoded region name is O(n)

2020-02-23 Thread GitBox
mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master 
by encoded region name is O(n)
URL: https://github.com/apache/hbase/pull/1193#issuecomment-590069862
 
 
   @saintstack I think it looks okay. However, I am not sure only a single 
thread accessing them. And as you have already mentioned, If I want to 
synchronize two maps, I should use synchronize or locks. It was my mistake. I 
didn't think about it then. Should do I use synchronized or locks?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master by encoded region name is O(n)

2020-02-23 Thread GitBox
mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master 
by encoded region name is O(n)
URL: https://github.com/apache/hbase/pull/1193#issuecomment-590070993
 
 
   I think the current code may have different values for the two maps.
   What do you think about changing it like below?
   
   ```java
   RegionStateNode createRegionStateNode(RegionInfo regionInfo) {
 RegionStateNode node = 
regionsMap.computeIfAbsent(regionInfo.getRegionName(), key -> new 
RegionStateNod(regionInfo, regionInTransition));
 encodedRegionsMap.putIfAbsent(node.getRegionInfo().getEncodedName(), node);
 return node;
   }
   ```
   
   Also there is a remove logic, it seems safe to use a lock.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master by encoded region name is O(n)

2020-02-23 Thread GitBox
mwkang edited a comment on issue #1193: HBASE-23561 Look up of Region in Master 
by encoded region name is O(n)
URL: https://github.com/apache/hbase/pull/1193#issuecomment-590070993
 
 
   I think the current code may have different values for the two maps.
   What do you think about changing it like below?
   
   ```java
   RegionStateNode createRegionStateNode(RegionInfo regionInfo) {
 RegionStateNode node = 
regionsMap.computeIfAbsent(regionInfo.getRegionName(), key -> new 
RegionStateNod(regionInfo, regionInTransition));
 encodedRegionsMap.putIfAbsent(node.getRegionInfo().getEncodedName(), node);
 return node;
   }
   ```


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] mwkang commented on issue #1193: HBASE-23561 Look up of Region in Master by encoded region name is O(n)

2020-02-23 Thread GitBox
mwkang commented on issue #1193: HBASE-23561 Look up of Region in Master by 
encoded region name is O(n)
URL: https://github.com/apache/hbase/pull/1193#issuecomment-590070993
 
 
   I think the current code may have different values for the two maps.
   What do you think about changing it like below?
   
   ```java
   RegionStateNode node = 
regionsMap.computeIfAbsent(regionInfo.getRegionName(),key -> new 
RegionStateNode(regionInfo, regionInTransition));
   encodedRegionsMap.putIfAbsent(node.getRegionInfo().getEncodedName(), node);
   return node;
   ```


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [hbase] mwkang commented on issue #1193: HBASE-23561 Look up of Region in Master by encoded region name is O(n)

2020-02-23 Thread GitBox
mwkang commented on issue #1193: HBASE-23561 Look up of Region in Master by 
encoded region name is O(n)
URL: https://github.com/apache/hbase/pull/1193#issuecomment-590069862
 
 
   @saintstack I think it looks okay. However, I am not sure only a single 
thread accessing them. And as you have already mentioned, If I want to 
synchronize two maps, I should use synchronize or locks. It was my mistake. I 
didn't think about it then. Should do I use synchronization or locks?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services


[jira] [Commented] (HBASE-23881) TestShadeSaslAuthenticationProvider failures

2020-02-23 Thread Josh Elser (Jira)


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

Josh Elser commented on HBASE-23881:


Sorry, I just realized my last comment was dumb . I shouldn't jira after a 
long week.

> TestShadeSaslAuthenticationProvider failures
> 
>
> Key: HBASE-23881
> URL: https://issues.apache.org/jira/browse/HBASE-23881
> Project: HBase
>  Issue Type: Bug
>  Components: test
>Affects Versions: 3.0.0
>Reporter: Bharath Vissapragada
>Assignee: Josh Elser
>Priority: Major
>
> TestShadeSaslAuthenticationProvider now fails deterministically with the 
> following exception..
> {noformat}
> java.lang.Exception: Unexpected exception, 
> expected but 
> was
>   at 
> org.apache.hadoop.hbase.security.provider.example.TestShadeSaslAuthenticationProvider.testNegativeAuthentication(TestShadeSaslAuthenticationProvider.java:233)
> {noformat}
> The test now fails a different place than before merging HBASE-18095 because 
> the RPCs are also a part of connection setup. We might need to rewrite the 
> test..  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [hbase] pankaj72981 opened a new pull request #1198: HBASE-23740 Invalid StoreFile WARN log message printed for recovered.…

2020-02-23 Thread GitBox
pankaj72981 opened a new pull request #1198: HBASE-23740 Invalid StoreFile WARN 
log message printed for recovered.…
URL: https://github.com/apache/hbase/pull/1198
 
 
   Skipping the WARN log for the recovered.hfile dir.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services