You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by la...@apache.org on 2016/01/05 06:36:06 UTC

hbase git commit: HBASE-15059 Allow 0.94 to compile against Hadoop 2.7.x.

Repository: hbase
Updated Branches:
  refs/heads/0.94 30b0c9824 -> dd5660444


HBASE-15059 Allow 0.94 to compile against Hadoop 2.7.x.


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

Branch: refs/heads/0.94
Commit: dd5660444a2587840bf982559d5b48f8a8e74e3c
Parents: 30b0c98
Author: Lars Hofhansl <la...@apache.org>
Authored: Mon Jan 4 21:36:19 2016 -0800
Committer: Lars Hofhansl <la...@apache.org>
Committed: Mon Jan 4 21:36:19 2016 -0800

----------------------------------------------------------------------
 dev-support/build-proto.sh                      |  37 +++++++
 pom.xml                                         |  98 ++++++++++++++++
 .../hadoop/hbase/master/MasterDumpServlet.java  |   5 +-
 .../metrics/file/TimeStampingFileContext.java   | 111 -------------------
 .../hbase/regionserver/RSDumpServlet.java       |   5 +-
 .../org/apache/hadoop/hbase/util/Threads.java   |  49 +++++++-
 .../file/TestTimeStampingMetricsContext.java    | 107 ------------------
 7 files changed, 191 insertions(+), 221 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dd566044/dev-support/build-proto.sh
----------------------------------------------------------------------
diff --git a/dev-support/build-proto.sh b/dev-support/build-proto.sh
new file mode 100644
index 0000000..96380f8
--- /dev/null
+++ b/dev-support/build-proto.sh
@@ -0,0 +1,37 @@
+#!/bin/bash
+##
+# 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.
+##
+# script to run protoc to generate protocol buf files.
+# usage: ./build-proto.sh
+#
+
+which protoc
+if [ $? != 0 ] ; then
+  echo "Must have protoc compiler in your path to generate code"
+  exit 1
+fi
+
+HBASE_DIR=`dirname $0`/..
+PROTO_DIR=$HBASE_DIR/src/main/protobuf
+JAVA_DIR=$HBASE_DIR/src/main/java
+
+set -x
+for f in $PROTO_DIR/*.proto ; do
+  protoc -I$PROTO_DIR --java_out=$JAVA_DIR $f
+done
+

http://git-wip-us.apache.org/repos/asf/hbase/blob/dd566044/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 19c7ed5..6507012 100644
--- a/pom.xml
+++ b/pom.xml
@@ -2385,6 +2385,104 @@
       </build>
     </profile>
 
+    <!--
+      profile for building against Hadoop 2.7. Activate using:
+       mvn -Dhadoop.profile=2.7
+
+      *** NOTE:
+       You MUST ensure that the protobuf Java classes are generated with protoc 2.5.0.
+       Do that by calling  dev-support/build-proto.sh  with protoc 2.5.0 in your path
+       before building HBase; that will regenerate the Java classes.
+       (HBase will not work with Hadoop 2.6+ with the older classes generated with protoc 2.4.1)
+
+       The necessary runtime dependencies for protobuf and commons.io are pulled in automatically.
+
+      Example:
+       $ protoc - -version
+       check that the output indicates 2.5.0
+
+       $ dev-support/build-proto.sh
+       $ mvn -Dhadoop.profile=2.7 ...
+    -->
+    <profile>
+      <id>hadoop-2.7</id>
+      <activation>
+        <property>
+          <name>hadoop.profile</name>
+          <value>2.7</value>
+        </property>
+      </activation>
+      <properties>
+        <hadoop.version>2.7.1</hadoop.version>
+        <slf4j.version>1.6.1</slf4j.version>
+        <protobuf.version>2.5.0</protobuf.version>
+        <commons-io.version>2.4</commons-io.version>
+      </properties>
+      <dependencies>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-common</artifactId>
+          <version>${hadoop.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-annotations</artifactId>
+          <version>${hadoop.version}</version>
+        </dependency>
+        <!-- test deps for hadoop-2.0 profile -->
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-minicluster</artifactId>
+          <version>${hadoop.version}</version>
+          <scope>compile</scope>
+        </dependency>
+      </dependencies>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>build-helper-maven-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>add-test-resource</id>
+                <goals>
+                  <goal>add-test-resource</goal>
+                </goals>
+                <configuration>
+                  <resources>
+                    <resource>
+                      <directory>src/test/resources</directory>
+                      <includes>
+                        <include>hbase-site.xml</include>
+                      </includes>
+                    </resource>
+                  </resources>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <artifactId>maven-dependency-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>create-mrapp-generated-classpath</id>
+                <phase>generate-test-resources</phase>
+                <goals>
+                  <goal>build-classpath</goal>
+                </goals>
+                <configuration>
+                  <!-- needed to run the unit test for DS to generate
+                  the required classpath that is required in the env
+                  of the launch container in the mini mr/yarn cluster
+                  -->
+                  <outputFile>${project.build.directory}/test-classes/mrapp-generated-classpath</outputFile>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
 
     <!-- profiles for the tests
          See as well the properties of the project for the values

http://git-wip-us.apache.org/repos/asf/hbase/blob/dd566044/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java b/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
index 876eda4..608979a 100644
--- a/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
+++ b/src/main/java/org/apache/hadoop/hbase/master/MasterDumpServlet.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.master;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.util.Date;
 import java.util.Map;
@@ -37,6 +38,7 @@ import org.apache.hadoop.hbase.master.AssignmentManager.RegionState;
 import org.apache.hadoop.hbase.monitoring.LogMonitoring;
 import org.apache.hadoop.hbase.monitoring.StateDumpServlet;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.util.ReflectionUtils;
 
 public class MasterDumpServlet extends StateDumpServlet {
@@ -79,7 +81,8 @@ public class MasterDumpServlet extends StateDumpServlet {
     
     out.println("\n\nStacks:");
     out.println(LINE);
-    ReflectionUtils.printThreadInfo(out, "");
+    PrintStream ps = new PrintStream(os, false, "UTF-8");
+    Threads.printThreadInfo(ps, "");
     
     out.println("\n\nMaster configuration:");
     out.println(LINE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/dd566044/src/main/java/org/apache/hadoop/hbase/metrics/file/TimeStampingFileContext.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/hadoop/hbase/metrics/file/TimeStampingFileContext.java b/src/main/java/org/apache/hadoop/hbase/metrics/file/TimeStampingFileContext.java
deleted file mode 100644
index 000e0d3..0000000
--- a/src/main/java/org/apache/hadoop/hbase/metrics/file/TimeStampingFileContext.java
+++ /dev/null
@@ -1,111 +0,0 @@
-/**
- * Copyright 2008 The Apache Software Foundation
- *
- * 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.metrics.file;
-
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileWriter;
-import java.io.IOException;
-import java.io.PrintWriter;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-import org.apache.hadoop.metrics.ContextFactory;
-import org.apache.hadoop.metrics.file.FileContext;
-import org.apache.hadoop.metrics.spi.OutputRecord;
-
-/**
- * Add timestamp to {@link org.apache.hadoop.metrics.file.FileContext#emitRecord(String, String, OutputRecord)}.
- */
-public class TimeStampingFileContext extends FileContext {
-  // Copies bunch of FileContext here because writer and file are private in
-  // superclass.
-  private File file = null;
-  private PrintWriter writer = null;
-  private final SimpleDateFormat sdf;
-
-  public TimeStampingFileContext() {
-    super();
-    this.sdf = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss");
-  }
-
-  @Override
-  public void init(String contextName, ContextFactory factory) {
-    super.init(contextName, factory);
-    String fileName = getAttribute(FILE_NAME_PROPERTY);
-    if (fileName != null) {
-      file = new File(fileName);
-    }
-  }
-
-  @Override
-  public void startMonitoring() throws IOException {
-    if (file == null) {
-      writer = new PrintWriter(new BufferedOutputStream(System.out));
-    } else {
-      writer = new PrintWriter(new FileWriter(file, true));
-    }
-    super.startMonitoring();
-  }
-
-  @Override
-  public void stopMonitoring() {
-    super.stopMonitoring();
-    if (writer != null) {
-      writer.close();
-      writer = null;
-    }
-  }
-
-  private synchronized String iso8601() {
-    return this.sdf.format(new Date());
-  }
-
-  @Override
-  public void emitRecord(String contextName, String recordName,
-      OutputRecord outRec) {
-    writer.print(iso8601());
-    writer.print(" ");
-    writer.print(contextName);
-    writer.print(".");
-    writer.print(recordName);
-    String separator = ": ";
-    for (String tagName : outRec.getTagNames()) {
-      writer.print(separator);
-      separator = ", ";
-      writer.print(tagName);
-      writer.print("=");
-      writer.print(outRec.getTag(tagName));
-    }
-    for (String metricName : outRec.getMetricNames()) {
-      writer.print(separator);
-      separator = ", ";
-      writer.print(metricName);
-      writer.print("=");
-      writer.print(outRec.getMetric(metricName));
-    }
-    writer.println();
-  }
-
-  @Override
-  public void flush() {
-    writer.flush();
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hbase/blob/dd566044/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java b/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java
index 979cd52..2df824b 100644
--- a/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java
+++ b/src/main/java/org/apache/hadoop/hbase/regionserver/RSDumpServlet.java
@@ -21,6 +21,7 @@ package org.apache.hadoop.hbase.regionserver;
 
 import java.io.IOException;
 import java.io.OutputStream;
+import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.util.Date;
 
@@ -31,6 +32,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.monitoring.LogMonitoring;
 import org.apache.hadoop.hbase.monitoring.StateDumpServlet;
 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
+import org.apache.hadoop.hbase.util.Threads;
 import org.apache.hadoop.util.ReflectionUtils;
 
 public class RSDumpServlet extends StateDumpServlet {
@@ -77,7 +79,8 @@ public class RSDumpServlet extends StateDumpServlet {
     
     out.println("\n\nStacks:");
     out.println(LINE);
-    ReflectionUtils.printThreadInfo(out, "");
+    PrintStream ps = new PrintStream(os, false, "UTF-8");
+    Threads.printThreadInfo(ps, "");
     
     out.println("\n\nRS Configuration:");
     out.println(LINE);

http://git-wip-us.apache.org/repos/asf/hbase/blob/dd566044/src/main/java/org/apache/hadoop/hbase/util/Threads.java
----------------------------------------------------------------------
diff --git a/src/main/java/org/apache/hadoop/hbase/util/Threads.java b/src/main/java/org/apache/hadoop/hbase/util/Threads.java
index a5a4d13..5a51bcc 100644
--- a/src/main/java/org/apache/hadoop/hbase/util/Threads.java
+++ b/src/main/java/org/apache/hadoop/hbase/util/Threads.java
@@ -19,8 +19,11 @@
  */
 package org.apache.hadoop.hbase.util;
 
+import java.io.PrintStream;
 import java.io.PrintWriter;
 import java.lang.Thread.UncaughtExceptionHandler;
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -116,7 +119,7 @@ public class Threads {
     while (t.isAlive()) {
       t.join(60 * 1000);
       if (t.isAlive()) {
-        ReflectionUtils.printThreadInfo(new PrintWriter(System.out),
+        printThreadInfo(System.out,
             "Automatic Stack Trace every 60 seconds waiting on " +
             t.getName());
       }
@@ -229,4 +232,48 @@ public class Threads {
 
     };
   }
+
+  private static Method printThreadInfoMethod = null;
+  private static boolean printThreadInfoMethodWithPrintStream = true;
+
+  /**
+   * Print all of the thread's information and stack traces. Wrapper around Hadoop's method.
+   *
+   * @param stream the stream to
+   * @param title a string title for the stack trace
+   */
+  public static void printThreadInfo(PrintStream stream, String title) {
+
+    if (printThreadInfoMethod == null) {
+      try {
+        // Hadoop 2.7+ declares printThreadInfo(PrintStream, String)
+        printThreadInfoMethod = ReflectionUtils.class.getMethod("printThreadInfo",
+          PrintStream.class, String.class);
+      } catch (NoSuchMethodException e) {
+        // Hadoop 2.6 and earlier declares printThreadInfo(PrintWriter, String)
+        printThreadInfoMethodWithPrintStream = false;
+        try {
+          printThreadInfoMethod = ReflectionUtils.class.getMethod("printThreadInfo",
+            PrintWriter.class, String.class);
+        } catch (NoSuchMethodException e1) {
+          throw new RuntimeException("Cannot find method. Check hadoop jars linked", e1);
+        }
+      }
+      printThreadInfoMethod.setAccessible(true);
+    }
+
+    try {
+      if (printThreadInfoMethodWithPrintStream) {
+        printThreadInfoMethod.invoke(null, stream, title);
+      } else {
+        printThreadInfoMethod.invoke(null, new PrintWriter(stream), title);
+      }
+    } catch (IllegalAccessException e) {
+      throw new RuntimeException(e.getCause());
+    } catch (IllegalArgumentException e) {
+      throw new RuntimeException(e.getCause());
+    } catch (InvocationTargetException e) {
+      throw new RuntimeException(e.getCause());
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hbase/blob/dd566044/src/test/java/org/apache/hadoop/hbase/metrics/file/TestTimeStampingMetricsContext.java
----------------------------------------------------------------------
diff --git a/src/test/java/org/apache/hadoop/hbase/metrics/file/TestTimeStampingMetricsContext.java b/src/test/java/org/apache/hadoop/hbase/metrics/file/TestTimeStampingMetricsContext.java
deleted file mode 100644
index 0700508..0000000
--- a/src/test/java/org/apache/hadoop/hbase/metrics/file/TestTimeStampingMetricsContext.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/**
- * 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.metrics.file;
-
-import java.io.File;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.Collection;
-import java.util.Date;
-import java.util.Map;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.hadoop.fs.FileUtil;
-import org.apache.hadoop.hbase.SmallTests;
-
-import org.junit.*;
-import org.junit.experimental.categories.Category;
-
-import static org.junit.Assert.*;
-
-/**
- * Test for TimeStampingMetricsContext functionality.
- * (FQN class names are used to suppress javac warnings in imports.)
- */
-@Category(SmallTests.class)
-@SuppressWarnings("deprecation")
-public class TestTimeStampingMetricsContext {
-
-  private static final int updatePeriodSeconds = 2;
-
-  private TimeStampingFileContext mc;
-
-  @Test
-  public void testFileUpdate() throws Exception {
-    final Date start = new Date();
-    final File metricOutFile = FileUtil.createLocalTempFile(
-      new File(FileUtils.getTempDirectory(),getClass().getName() + "-out-"), "", true);
-    assertTrue(metricOutFile.exists());
-    assertEquals(0L, metricOutFile.length());
-
-    mc = new TimeStampingFileContext();
-    org.apache.hadoop.metrics.ContextFactory cf
-      = org.apache.hadoop.metrics.ContextFactory.getFactory();
-    cf.setAttribute("test1.fileName", metricOutFile.getAbsolutePath());
-    cf.setAttribute("test1.period", Integer.toString(updatePeriodSeconds));
-    mc.init("test1", cf);
-
-    assertEquals("test1", mc.getContextName());
-
-    org.apache.hadoop.metrics.MetricsRecord r = mc.createRecord("testRecord");
-    r.setTag("testTag1", "testTagValue1");
-    r.setTag("testTag2", "testTagValue2");
-    r.setMetric("testMetric1", 1);
-    r.setMetric("testMetric2", 33);
-    r.update();
-
-    mc.startMonitoring();
-    assertTrue(mc.isMonitoring());
-
-    // wait 3/2 of the update period:
-    Thread.sleep((1000 * updatePeriodSeconds * 3)/2);
-
-    mc.stopMonitoring();
-    assertFalse(mc.isMonitoring());
-
-    mc.close();
-
-    Map<String, Collection<org.apache.hadoop.metrics.spi.OutputRecord>> m = mc.getAllRecords();
-    assertEquals(1, m.size());
-    Collection<org.apache.hadoop.metrics.spi.OutputRecord> outputRecords = m.get("testRecord");
-    assertNotNull(outputRecords);
-    assertEquals(1, outputRecords.size());
-    org.apache.hadoop.metrics.spi.OutputRecord outputRecord = outputRecords.iterator().next();
-    assertNotNull(outputRecord);
-
-    String outStr = FileUtils.readFileToString(metricOutFile);
-    assertTrue(outStr.length() > 0);
-    int pos = outStr.indexOf(" ");
-    String time = outStr.substring(0, pos);
-
-    DateFormat df = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss");
-    Date date = df.parse(time);
-    assertTrue(date.after(start));
-    assertTrue(date.before(new Date()));
-
-    String reminder = outStr.substring(pos);
-    assertEquals(" test1.testRecord: testTag1=testTagValue1, testTag2=testTagValue2, testMetric1=1,"
-      +" testMetric2=33\n", reminder);
-  }
-
-}