You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by el...@apache.org on 2018/08/28 15:32:42 UTC

[1/5] hbase git commit: HBASE-20942 Fix ArrayIndexOutOfBoundsException for RpcServer TRACE logging

Repository: hbase
Updated Branches:
  refs/heads/branch-1 3c17c8391 -> 084042751
  refs/heads/branch-1.3 2750a574a -> a2ea72dab
  refs/heads/branch-1.4 525a6f9f8 -> 5d5ede9dd
  refs/heads/branch-2 fc21dc854 -> 22254cf4c
  refs/heads/master 3afe9fb7e -> fcd883b5d


HBASE-20942 Fix ArrayIndexOutOfBoundsException for RpcServer TRACE logging

Also makes the trace log message length configurable.

Signed-off-by: Josh Elser <el...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/fcd883b5
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/fcd883b5
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/fcd883b5

Branch: refs/heads/master
Commit: fcd883b5dd7f3e2987646407f8f03c5513d93a9b
Parents: 3afe9fb
Author: krish.dey <kr...@gmail.com>
Authored: Tue Aug 28 10:49:10 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Aug 28 11:01:49 2018 -0400

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 25 +++++-
 .../hbase/ipc/TestRpcServerTraceLogging.java    | 95 ++++++++++++++++++++
 2 files changed, 118 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/fcd883b5/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index fb2ee40..0207b03 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -191,6 +191,10 @@ public abstract class RpcServer implements RpcServerInterface,
   protected static final int DEFAULT_WARN_RESPONSE_TIME = 10000; // milliseconds
   protected static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024;
 
+  protected static final int DEFAULT_TRACE_LOG_MAX_LENGTH = 1000;
+  protected static final String TRACE_LOG_MAX_LENGTH = "hbase.ipc.trace.log.max.length";
+  protected static final String KEY_WORD_TRUNCATED = " <TRUNCATED>";
+
   protected static final ObjectMapper MAPPER = new ObjectMapper();
 
   protected final int maxRequestSize;
@@ -500,8 +504,7 @@ public abstract class RpcServer implements RpcServerInterface,
     String stringifiedParam = ProtobufUtil.getShortTextFormat(param);
     if (stringifiedParam.length() > 150) {
       // Truncate to 1000 chars if TRACE is on, else to 150 chars
-      stringifiedParam = stringifiedParam.subSequence(
-          0, LOG.isTraceEnabled() ? 1000 : 150) + " <TRUNCATED>";
+      stringifiedParam = truncateTraceLog(stringifiedParam);
     }
     responseInfo.put("param", stringifiedParam);
     if (param instanceof ClientProtos.ScanRequest && rsRpcServices != null) {
@@ -518,6 +521,24 @@ public abstract class RpcServer implements RpcServerInterface,
   }
 
   /**
+   * Truncate to number of chars decided by conf hbase.ipc.trace.log.max.length
+   * if TRACE is on else to 150 chars Refer to Jira HBASE-20826 and HBASE-20942
+   * @param strParam stringifiedParam to be truncated
+   * @return truncated trace log string
+   */
+  @VisibleForTesting
+  String truncateTraceLog(String strParam) {
+    if (LOG.isTraceEnabled()) {
+      int traceLogMaxLength = getConf().getInt(TRACE_LOG_MAX_LENGTH, DEFAULT_TRACE_LOG_MAX_LENGTH);
+      int truncatedLength =
+          strParam.length() < traceLogMaxLength ? strParam.length() : traceLogMaxLength;
+      String truncatedFlag = truncatedLength == strParam.length() ? "" : KEY_WORD_TRUNCATED;
+      return strParam.subSequence(0, truncatedLength) + truncatedFlag;
+    }
+    return strParam.subSequence(0, 150) + KEY_WORD_TRUNCATED;
+  }
+
+  /**
    * Set the handler for calling out of RPC for error conditions.
    * @param handler the handler implementation
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/fcd883b5/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
new file mode 100644
index 0000000..e95c88c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
@@ -0,0 +1,95 @@
+/**
+ * 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.ipc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+
+@Category(SmallTests.class)
+public class TestRpcServerTraceLogging {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule
+      .forClass(TestRpcServerTraceLogging.class);
+
+  static Logger rpcServerLog = Logger.getLogger(RpcServer.class);
+
+  static final String TRACE_LOG_MSG =
+      "This is dummy message for testing:: region { type: REGION_NAME value: \"hbase:meta,,1\" }"
+          + " scan { column { family: \"info\" } time_range { from: 0 to: 9223372036854775807 } "
+      + "max_versions: 1 cache_blocks: true max_result_size: 2097152 caching: 2147483647 } "
+      + "number_of_rows: 2147483647 close_scanner: false client_handles_partials: "
+      + "true client_handles_heartbeats: true track_scan_metrics: false";
+
+  static final int TRACE_LOG_LENGTH = TRACE_LOG_MSG.length();
+
+  static final RpcServer mockRpcServer = Mockito.mock(RpcServer.class);
+
+  static final Configuration conf = new Configuration(false);
+
+  @BeforeClass
+  public static void setUp() {
+    Mockito.when(mockRpcServer.getConf()).thenReturn(conf);
+    Mockito.when(mockRpcServer.truncateTraceLog(Mockito.any(String.class))).thenCallRealMethod();
+  }
+
+  @Test
+  public void testLoggingWithTraceOff() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 250);
+    rpcServerLog.setLevel(Level.DEBUG);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(150 + RpcServer.KEY_WORD_TRUNCATED.length(), truncatedString.length());
+    assertTrue(truncatedString.contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+
+  @Test
+  public void testLoggingWithTraceOn() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 250);
+    rpcServerLog.setLevel(Level.TRACE);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(250 + RpcServer.KEY_WORD_TRUNCATED.length(), truncatedString.length());
+    assertTrue(truncatedString.contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+
+  @Test
+  public void testLoggingWithTraceOnLargeMax() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 2000);
+    rpcServerLog.setLevel(Level.TRACE);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(TRACE_LOG_LENGTH, truncatedString.length());
+    assertFalse(
+      mockRpcServer.truncateTraceLog(TRACE_LOG_MSG).contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+}


[5/5] hbase git commit: HBASE-20942 Fix ArrayIndexOutOfBoundsException for RpcServer TRACE logging

Posted by el...@apache.org.
HBASE-20942 Fix ArrayIndexOutOfBoundsException for RpcServer TRACE logging

Also makes the trace log message length configurable.

Signed-off-by: Josh Elser <el...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/a2ea72da
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/a2ea72da
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/a2ea72da

Branch: refs/heads/branch-1.3
Commit: a2ea72dabaca836be5572d1e93c68ae050390c97
Parents: 2750a57
Author: krish.dey <kr...@gmail.com>
Authored: Tue Aug 28 10:49:10 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Aug 28 11:30:33 2018 -0400

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 26 +++++-
 .../hbase/ipc/TestRpcServerTraceLogging.java    | 90 ++++++++++++++++++++
 2 files changed, 114 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/a2ea72da/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 5742851..cbdc75c 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -137,6 +137,7 @@ import org.apache.hadoop.util.StringUtils;
 import org.codehaus.jackson.map.ObjectMapper;
 import org.apache.htrace.TraceInfo;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.protobuf.BlockingService;
 import com.google.protobuf.CodedInputStream;
@@ -289,6 +290,10 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
 
   private static final ObjectMapper MAPPER = new ObjectMapper();
 
+  protected static final int DEFAULT_TRACE_LOG_MAX_LENGTH = 1000;
+  protected static final String TRACE_LOG_MAX_LENGTH = "hbase.ipc.trace.log.max.length";
+  protected static final String KEY_WORD_TRUNCATED = " <TRUNCATED>";
+
   private final int maxRequestSize;
   private final int warnResponseTime;
   private final int warnResponseSize;
@@ -2440,8 +2445,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     String stringifiedParam = ProtobufUtil.getShortTextFormat(param);
     if (stringifiedParam.length() > 150) {
       // Truncate to 1000 chars if TRACE is on, else to 150 chars
-      stringifiedParam = stringifiedParam.subSequence(
-          0, LOG.isTraceEnabled() ? 1000 : 150) + " <TRUNCATED>";
+      stringifiedParam = truncateTraceLog(stringifiedParam);
     }
     responseInfo.put("param", stringifiedParam);
     if (param instanceof ClientProtos.ScanRequest && rsRpcServices != null) {
@@ -2500,6 +2504,24 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
   }
 
   /**
+   * Truncate to number of chars decided by conf hbase.ipc.trace.log.max.length
+   * if TRACE is on else to 150 chars Refer to Jira HBASE-20826 and HBASE-20942
+   * @param strParam stringifiedParam to be truncated
+   * @return truncated trace log string
+   */
+  @VisibleForTesting
+  String truncateTraceLog(String strParam) {
+    if (LOG.isTraceEnabled()) {
+      int traceLogMaxLength = getConf().getInt(TRACE_LOG_MAX_LENGTH, DEFAULT_TRACE_LOG_MAX_LENGTH);
+      int truncatedLength =
+          strParam.length() < traceLogMaxLength ? strParam.length() : traceLogMaxLength;
+      String truncatedFlag = truncatedLength == strParam.length() ? "" : KEY_WORD_TRUNCATED;
+      return strParam.subSequence(0, truncatedLength) + truncatedFlag;
+    }
+    return strParam.subSequence(0, 150) + KEY_WORD_TRUNCATED;
+  }
+
+  /**
    * Set the handler for calling out of RPC for error conditions.
    * @param handler the handler implementation
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/a2ea72da/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
new file mode 100644
index 0000000..83ddfdb
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
@@ -0,0 +1,90 @@
+/**
+ * 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.ipc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+
+@Category(SmallTests.class)
+public class TestRpcServerTraceLogging {
+
+  static Logger rpcServerLog = Logger.getLogger(RpcServer.class);
+
+  static final String TRACE_LOG_MSG =
+      "This is dummy message for testing:: region { type: REGION_NAME value: \"hbase:meta,,1\" }"
+          + " scan { column { family: \"info\" } time_range { from: 0 to: 9223372036854775807 } "
+      + "max_versions: 1 cache_blocks: true max_result_size: 2097152 caching: 2147483647 } "
+      + "number_of_rows: 2147483647 close_scanner: false client_handles_partials: "
+      + "true client_handles_heartbeats: true track_scan_metrics: false";
+
+  static final int TRACE_LOG_LENGTH = TRACE_LOG_MSG.length();
+
+  static final RpcServer mockRpcServer = Mockito.mock(RpcServer.class);
+
+  static final Configuration conf = new Configuration(false);
+
+  @BeforeClass
+  public static void setUp() {
+    Mockito.when(mockRpcServer.getConf()).thenReturn(conf);
+    Mockito.when(mockRpcServer.truncateTraceLog(Mockito.any(String.class))).thenCallRealMethod();
+  }
+
+  @Test
+  public void testLoggingWithTraceOff() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 250);
+    rpcServerLog.setLevel(Level.DEBUG);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(150 + RpcServer.KEY_WORD_TRUNCATED.length(), truncatedString.length());
+    assertTrue(truncatedString.contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+
+  @Test
+  public void testLoggingWithTraceOn() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 250);
+    rpcServerLog.setLevel(Level.TRACE);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(250 + RpcServer.KEY_WORD_TRUNCATED.length(), truncatedString.length());
+    assertTrue(truncatedString.contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+
+  @Test
+  public void testLoggingWithTraceOnLargeMax() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 2000);
+    rpcServerLog.setLevel(Level.TRACE);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(TRACE_LOG_LENGTH, truncatedString.length());
+    assertFalse(
+      mockRpcServer.truncateTraceLog(TRACE_LOG_MSG).contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+}


[2/5] hbase git commit: HBASE-20942 Fix ArrayIndexOutOfBoundsException for RpcServer TRACE logging

Posted by el...@apache.org.
HBASE-20942 Fix ArrayIndexOutOfBoundsException for RpcServer TRACE logging

Also makes the trace log message length configurable.

Signed-off-by: Josh Elser <el...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/22254cf4
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/22254cf4
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/22254cf4

Branch: refs/heads/branch-2
Commit: 22254cf4ce9c863ff754968a3b699baa2bbec599
Parents: fc21dc8
Author: krish.dey <kr...@gmail.com>
Authored: Tue Aug 28 10:49:10 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Aug 28 11:02:18 2018 -0400

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 25 +++++-
 .../hbase/ipc/TestRpcServerTraceLogging.java    | 95 ++++++++++++++++++++
 2 files changed, 118 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/22254cf4/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index fb2ee40..0207b03 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -191,6 +191,10 @@ public abstract class RpcServer implements RpcServerInterface,
   protected static final int DEFAULT_WARN_RESPONSE_TIME = 10000; // milliseconds
   protected static final int DEFAULT_WARN_RESPONSE_SIZE = 100 * 1024 * 1024;
 
+  protected static final int DEFAULT_TRACE_LOG_MAX_LENGTH = 1000;
+  protected static final String TRACE_LOG_MAX_LENGTH = "hbase.ipc.trace.log.max.length";
+  protected static final String KEY_WORD_TRUNCATED = " <TRUNCATED>";
+
   protected static final ObjectMapper MAPPER = new ObjectMapper();
 
   protected final int maxRequestSize;
@@ -500,8 +504,7 @@ public abstract class RpcServer implements RpcServerInterface,
     String stringifiedParam = ProtobufUtil.getShortTextFormat(param);
     if (stringifiedParam.length() > 150) {
       // Truncate to 1000 chars if TRACE is on, else to 150 chars
-      stringifiedParam = stringifiedParam.subSequence(
-          0, LOG.isTraceEnabled() ? 1000 : 150) + " <TRUNCATED>";
+      stringifiedParam = truncateTraceLog(stringifiedParam);
     }
     responseInfo.put("param", stringifiedParam);
     if (param instanceof ClientProtos.ScanRequest && rsRpcServices != null) {
@@ -518,6 +521,24 @@ public abstract class RpcServer implements RpcServerInterface,
   }
 
   /**
+   * Truncate to number of chars decided by conf hbase.ipc.trace.log.max.length
+   * if TRACE is on else to 150 chars Refer to Jira HBASE-20826 and HBASE-20942
+   * @param strParam stringifiedParam to be truncated
+   * @return truncated trace log string
+   */
+  @VisibleForTesting
+  String truncateTraceLog(String strParam) {
+    if (LOG.isTraceEnabled()) {
+      int traceLogMaxLength = getConf().getInt(TRACE_LOG_MAX_LENGTH, DEFAULT_TRACE_LOG_MAX_LENGTH);
+      int truncatedLength =
+          strParam.length() < traceLogMaxLength ? strParam.length() : traceLogMaxLength;
+      String truncatedFlag = truncatedLength == strParam.length() ? "" : KEY_WORD_TRUNCATED;
+      return strParam.subSequence(0, truncatedLength) + truncatedFlag;
+    }
+    return strParam.subSequence(0, 150) + KEY_WORD_TRUNCATED;
+  }
+
+  /**
    * Set the handler for calling out of RPC for error conditions.
    * @param handler the handler implementation
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/22254cf4/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
new file mode 100644
index 0000000..e95c88c
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
@@ -0,0 +1,95 @@
+/**
+ * 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.ipc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseClassTestRule;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+
+@Category(SmallTests.class)
+public class TestRpcServerTraceLogging {
+
+  @ClassRule
+  public static final HBaseClassTestRule CLASS_RULE = HBaseClassTestRule
+      .forClass(TestRpcServerTraceLogging.class);
+
+  static Logger rpcServerLog = Logger.getLogger(RpcServer.class);
+
+  static final String TRACE_LOG_MSG =
+      "This is dummy message for testing:: region { type: REGION_NAME value: \"hbase:meta,,1\" }"
+          + " scan { column { family: \"info\" } time_range { from: 0 to: 9223372036854775807 } "
+      + "max_versions: 1 cache_blocks: true max_result_size: 2097152 caching: 2147483647 } "
+      + "number_of_rows: 2147483647 close_scanner: false client_handles_partials: "
+      + "true client_handles_heartbeats: true track_scan_metrics: false";
+
+  static final int TRACE_LOG_LENGTH = TRACE_LOG_MSG.length();
+
+  static final RpcServer mockRpcServer = Mockito.mock(RpcServer.class);
+
+  static final Configuration conf = new Configuration(false);
+
+  @BeforeClass
+  public static void setUp() {
+    Mockito.when(mockRpcServer.getConf()).thenReturn(conf);
+    Mockito.when(mockRpcServer.truncateTraceLog(Mockito.any(String.class))).thenCallRealMethod();
+  }
+
+  @Test
+  public void testLoggingWithTraceOff() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 250);
+    rpcServerLog.setLevel(Level.DEBUG);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(150 + RpcServer.KEY_WORD_TRUNCATED.length(), truncatedString.length());
+    assertTrue(truncatedString.contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+
+  @Test
+  public void testLoggingWithTraceOn() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 250);
+    rpcServerLog.setLevel(Level.TRACE);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(250 + RpcServer.KEY_WORD_TRUNCATED.length(), truncatedString.length());
+    assertTrue(truncatedString.contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+
+  @Test
+  public void testLoggingWithTraceOnLargeMax() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 2000);
+    rpcServerLog.setLevel(Level.TRACE);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(TRACE_LOG_LENGTH, truncatedString.length());
+    assertFalse(
+      mockRpcServer.truncateTraceLog(TRACE_LOG_MSG).contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+}


[3/5] hbase git commit: HBASE-20942 Fix ArrayIndexOutOfBoundsException for RpcServer TRACE logging

Posted by el...@apache.org.
HBASE-20942 Fix ArrayIndexOutOfBoundsException for RpcServer TRACE logging

Also makes the trace log message length configurable.

Signed-off-by: Josh Elser <el...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/08404275
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/08404275
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/08404275

Branch: refs/heads/branch-1
Commit: 08404275161a792b2ee9c60aef86e700d15bac81
Parents: 3c17c83
Author: krish.dey <kr...@gmail.com>
Authored: Tue Aug 28 10:49:10 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Aug 28 11:25:08 2018 -0400

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 26 +++++-
 .../hbase/ipc/TestRpcServerTraceLogging.java    | 90 ++++++++++++++++++++
 2 files changed, 114 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/08404275/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 3950939..7d7dd9d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.ipc;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.protobuf.BlockingService;
 import com.google.protobuf.CodedInputStream;
@@ -280,6 +281,10 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
 
   private static final ObjectMapper MAPPER = new ObjectMapper();
 
+  protected static final int DEFAULT_TRACE_LOG_MAX_LENGTH = 1000;
+  protected static final String TRACE_LOG_MAX_LENGTH = "hbase.ipc.trace.log.max.length";
+  protected static final String KEY_WORD_TRUNCATED = " <TRUNCATED>";
+
   private final int maxRequestSize;
   private final int warnResponseTime;
   private final int warnResponseSize;
@@ -2445,8 +2450,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     String stringifiedParam = ProtobufUtil.getShortTextFormat(param);
     if (stringifiedParam.length() > 150) {
       // Truncate to 1000 chars if TRACE is on, else to 150 chars
-      stringifiedParam = stringifiedParam.subSequence(
-          0, LOG.isTraceEnabled() ? 1000 : 150) + " <TRUNCATED>";
+      stringifiedParam = truncateTraceLog(stringifiedParam);
     }
     responseInfo.put("param", stringifiedParam);
     if (param instanceof ClientProtos.ScanRequest && rsRpcServices != null) {
@@ -2505,6 +2509,24 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
   }
 
   /**
+   * Truncate to number of chars decided by conf hbase.ipc.trace.log.max.length
+   * if TRACE is on else to 150 chars Refer to Jira HBASE-20826 and HBASE-20942
+   * @param strParam stringifiedParam to be truncated
+   * @return truncated trace log string
+   */
+  @VisibleForTesting
+  String truncateTraceLog(String strParam) {
+    if (LOG.isTraceEnabled()) {
+      int traceLogMaxLength = getConf().getInt(TRACE_LOG_MAX_LENGTH, DEFAULT_TRACE_LOG_MAX_LENGTH);
+      int truncatedLength =
+          strParam.length() < traceLogMaxLength ? strParam.length() : traceLogMaxLength;
+      String truncatedFlag = truncatedLength == strParam.length() ? "" : KEY_WORD_TRUNCATED;
+      return strParam.subSequence(0, truncatedLength) + truncatedFlag;
+    }
+    return strParam.subSequence(0, 150) + KEY_WORD_TRUNCATED;
+  }
+
+  /**
    * Set the handler for calling out of RPC for error conditions.
    * @param handler the handler implementation
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/08404275/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
new file mode 100644
index 0000000..83ddfdb
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
@@ -0,0 +1,90 @@
+/**
+ * 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.ipc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+
+@Category(SmallTests.class)
+public class TestRpcServerTraceLogging {
+
+  static Logger rpcServerLog = Logger.getLogger(RpcServer.class);
+
+  static final String TRACE_LOG_MSG =
+      "This is dummy message for testing:: region { type: REGION_NAME value: \"hbase:meta,,1\" }"
+          + " scan { column { family: \"info\" } time_range { from: 0 to: 9223372036854775807 } "
+      + "max_versions: 1 cache_blocks: true max_result_size: 2097152 caching: 2147483647 } "
+      + "number_of_rows: 2147483647 close_scanner: false client_handles_partials: "
+      + "true client_handles_heartbeats: true track_scan_metrics: false";
+
+  static final int TRACE_LOG_LENGTH = TRACE_LOG_MSG.length();
+
+  static final RpcServer mockRpcServer = Mockito.mock(RpcServer.class);
+
+  static final Configuration conf = new Configuration(false);
+
+  @BeforeClass
+  public static void setUp() {
+    Mockito.when(mockRpcServer.getConf()).thenReturn(conf);
+    Mockito.when(mockRpcServer.truncateTraceLog(Mockito.any(String.class))).thenCallRealMethod();
+  }
+
+  @Test
+  public void testLoggingWithTraceOff() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 250);
+    rpcServerLog.setLevel(Level.DEBUG);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(150 + RpcServer.KEY_WORD_TRUNCATED.length(), truncatedString.length());
+    assertTrue(truncatedString.contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+
+  @Test
+  public void testLoggingWithTraceOn() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 250);
+    rpcServerLog.setLevel(Level.TRACE);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(250 + RpcServer.KEY_WORD_TRUNCATED.length(), truncatedString.length());
+    assertTrue(truncatedString.contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+
+  @Test
+  public void testLoggingWithTraceOnLargeMax() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 2000);
+    rpcServerLog.setLevel(Level.TRACE);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(TRACE_LOG_LENGTH, truncatedString.length());
+    assertFalse(
+      mockRpcServer.truncateTraceLog(TRACE_LOG_MSG).contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+}


[4/5] hbase git commit: HBASE-20942 Fix ArrayIndexOutOfBoundsException for RpcServer TRACE logging

Posted by el...@apache.org.
HBASE-20942 Fix ArrayIndexOutOfBoundsException for RpcServer TRACE logging

Also makes the trace log message length configurable.

Signed-off-by: Josh Elser <el...@apache.org>


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/5d5ede9d
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/5d5ede9d
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/5d5ede9d

Branch: refs/heads/branch-1.4
Commit: 5d5ede9ddea48580943cbdc185fe928040739e43
Parents: 525a6f9
Author: krish.dey <kr...@gmail.com>
Authored: Tue Aug 28 10:49:10 2018 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Tue Aug 28 11:25:15 2018 -0400

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/ipc/RpcServer.java  | 26 +++++-
 .../hbase/ipc/TestRpcServerTraceLogging.java    | 90 ++++++++++++++++++++
 2 files changed, 114 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5d5ede9d/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
index 3950939..7d7dd9d 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/ipc/RpcServer.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hbase.ipc;
 
 import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 import com.google.protobuf.BlockingService;
 import com.google.protobuf.CodedInputStream;
@@ -280,6 +281,10 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
 
   private static final ObjectMapper MAPPER = new ObjectMapper();
 
+  protected static final int DEFAULT_TRACE_LOG_MAX_LENGTH = 1000;
+  protected static final String TRACE_LOG_MAX_LENGTH = "hbase.ipc.trace.log.max.length";
+  protected static final String KEY_WORD_TRUNCATED = " <TRUNCATED>";
+
   private final int maxRequestSize;
   private final int warnResponseTime;
   private final int warnResponseSize;
@@ -2445,8 +2450,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
     String stringifiedParam = ProtobufUtil.getShortTextFormat(param);
     if (stringifiedParam.length() > 150) {
       // Truncate to 1000 chars if TRACE is on, else to 150 chars
-      stringifiedParam = stringifiedParam.subSequence(
-          0, LOG.isTraceEnabled() ? 1000 : 150) + " <TRUNCATED>";
+      stringifiedParam = truncateTraceLog(stringifiedParam);
     }
     responseInfo.put("param", stringifiedParam);
     if (param instanceof ClientProtos.ScanRequest && rsRpcServices != null) {
@@ -2505,6 +2509,24 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
   }
 
   /**
+   * Truncate to number of chars decided by conf hbase.ipc.trace.log.max.length
+   * if TRACE is on else to 150 chars Refer to Jira HBASE-20826 and HBASE-20942
+   * @param strParam stringifiedParam to be truncated
+   * @return truncated trace log string
+   */
+  @VisibleForTesting
+  String truncateTraceLog(String strParam) {
+    if (LOG.isTraceEnabled()) {
+      int traceLogMaxLength = getConf().getInt(TRACE_LOG_MAX_LENGTH, DEFAULT_TRACE_LOG_MAX_LENGTH);
+      int truncatedLength =
+          strParam.length() < traceLogMaxLength ? strParam.length() : traceLogMaxLength;
+      String truncatedFlag = truncatedLength == strParam.length() ? "" : KEY_WORD_TRUNCATED;
+      return strParam.subSequence(0, truncatedLength) + truncatedFlag;
+    }
+    return strParam.subSequence(0, 150) + KEY_WORD_TRUNCATED;
+  }
+
+  /**
    * Set the handler for calling out of RPC for error conditions.
    * @param handler the handler implementation
    */

http://git-wip-us.apache.org/repos/asf/hbase/blob/5d5ede9d/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
new file mode 100644
index 0000000..83ddfdb
--- /dev/null
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/ipc/TestRpcServerTraceLogging.java
@@ -0,0 +1,90 @@
+/**
+ * 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.ipc;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.testclassification.SmallTests;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.BeforeClass;
+import org.junit.ClassRule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+
+@Category(SmallTests.class)
+public class TestRpcServerTraceLogging {
+
+  static Logger rpcServerLog = Logger.getLogger(RpcServer.class);
+
+  static final String TRACE_LOG_MSG =
+      "This is dummy message for testing:: region { type: REGION_NAME value: \"hbase:meta,,1\" }"
+          + " scan { column { family: \"info\" } time_range { from: 0 to: 9223372036854775807 } "
+      + "max_versions: 1 cache_blocks: true max_result_size: 2097152 caching: 2147483647 } "
+      + "number_of_rows: 2147483647 close_scanner: false client_handles_partials: "
+      + "true client_handles_heartbeats: true track_scan_metrics: false";
+
+  static final int TRACE_LOG_LENGTH = TRACE_LOG_MSG.length();
+
+  static final RpcServer mockRpcServer = Mockito.mock(RpcServer.class);
+
+  static final Configuration conf = new Configuration(false);
+
+  @BeforeClass
+  public static void setUp() {
+    Mockito.when(mockRpcServer.getConf()).thenReturn(conf);
+    Mockito.when(mockRpcServer.truncateTraceLog(Mockito.any(String.class))).thenCallRealMethod();
+  }
+
+  @Test
+  public void testLoggingWithTraceOff() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 250);
+    rpcServerLog.setLevel(Level.DEBUG);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(150 + RpcServer.KEY_WORD_TRUNCATED.length(), truncatedString.length());
+    assertTrue(truncatedString.contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+
+  @Test
+  public void testLoggingWithTraceOn() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 250);
+    rpcServerLog.setLevel(Level.TRACE);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(250 + RpcServer.KEY_WORD_TRUNCATED.length(), truncatedString.length());
+    assertTrue(truncatedString.contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+
+  @Test
+  public void testLoggingWithTraceOnLargeMax() {
+    conf.setInt("hbase.ipc.trace.log.max.length", 2000);
+    rpcServerLog.setLevel(Level.TRACE);
+    String truncatedString = mockRpcServer.truncateTraceLog(TRACE_LOG_MSG);
+
+    assertEquals(TRACE_LOG_LENGTH, truncatedString.length());
+    assertFalse(
+      mockRpcServer.truncateTraceLog(TRACE_LOG_MSG).contains(RpcServer.KEY_WORD_TRUNCATED));
+  }
+}