You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by va...@apache.org on 2017/08/30 21:37:01 UTC

[1/2] hadoop git commit: YARN-6905 Multiple HBaseTimelineStorage test failures due to missing FastNumberFormat (Contributed by Haibo Chen)

Repository: hadoop
Updated Branches:
  refs/heads/YARN-5355_branch2 9aee24df3 -> 2f81c1102


YARN-6905 Multiple HBaseTimelineStorage test failures due to missing FastNumberFormat (Contributed by Haibo Chen)

(cherry picked from commit 608a06cca5d68b3155bd70a94bf29ae0942b9ca0)

Conflicts:
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
	hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java


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

Branch: refs/heads/YARN-5355_branch2
Commit: f735024e8d47d582220a8e849c7b49625f065029
Parents: 9aee24d
Author: Vrushali C <vr...@apache.org>
Authored: Mon Aug 14 11:40:27 2017 -0700
Committer: Varun Saxena <va...@apache.org>
Committed: Thu Aug 31 02:56:49 2017 +0530

----------------------------------------------------------------------
 .../storage/TestHBaseTimelineStorageApps.java   |  4 +-
 .../TestHBaseTimelineStorageEntities.java       | 13 ++++---
 .../storage/common/AppIdKeyConverter.java       |  3 +-
 .../common/HBaseTimelineStorageUtils.java       | 34 +++++++++++++++++
 .../TestCustomApplicationIdConversion.java      | 39 ++++++++++++++++++++
 5 files changed, 86 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f735024e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
index 65c7034..111008a 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageApps.java
@@ -71,6 +71,7 @@ import org.apache.hadoop.yarn.server.timelineservice.storage.application.Applica
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
@@ -499,7 +500,8 @@ public class TestHBaseTimelineStorageApps {
     event.addInfo(expKey, expVal);
 
     final TimelineEntity entity = new ApplicationEntity();
-    entity.setId(ApplicationId.newInstance(0, 1).toString());
+    entity.setId(HBaseTimelineStorageUtils.convertApplicationIdToString(
+        ApplicationId.newInstance(0, 1)));
     entity.addEvent(event);
 
     TimelineEntities entities = new TimelineEntities();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f735024e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
index 422316b..ae8f821 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/TestHBaseTimelineStorageEntities.java
@@ -64,6 +64,7 @@ import org.apache.hadoop.yarn.server.timelineservice.reader.filter.TimelinePrefi
 import org.apache.hadoop.yarn.server.timelineservice.storage.TimelineReader.Field;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnName;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.EventColumnNameConverter;
+import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.KeyConverter;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.Separator;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.StringKeyConverter;
@@ -207,8 +208,8 @@ public class TestHBaseTimelineStorageEntities {
       String flow = "some_flow_name";
       String flowVersion = "AB7822C10F1111";
       long runid = 1002345678919L;
-      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
-          9000000L, 1).toString();
+      String appName = HBaseTimelineStorageUtils.convertApplicationIdToString(
+          ApplicationId.newInstance(System.currentTimeMillis() + 9000000L, 1));
       hbi.write(new TimelineCollectorContext(cluster, user, flow, flowVersion,
           runid, appName), te,
           UserGroupInformation.createRemoteUser(subAppUser));
@@ -509,8 +510,8 @@ public class TestHBaseTimelineStorageEntities {
       String flow = "other_flow_name";
       String flowVersion = "1111F01C2287BA";
       long runid = 1009876543218L;
-      String appName = ApplicationId.newInstance(System.currentTimeMillis() +
-          9000000L, 1).toString();
+      String appName = HBaseTimelineStorageUtils.convertApplicationIdToString(
+          ApplicationId.newInstance(System.currentTimeMillis() + 9000000L, 1));
       byte[] startRow =
           new EntityRowKeyPrefix(cluster, user, flow, runid, appName)
               .getRowKeyPrefix();
@@ -601,7 +602,9 @@ public class TestHBaseTimelineStorageEntities {
     event.addInfo(expKey, expVal);
 
     final TimelineEntity entity = new ApplicationEntity();
-    entity.setId(ApplicationId.newInstance(0, 1).toString());
+    entity.setId(
+        HBaseTimelineStorageUtils.convertApplicationIdToString(
+            ApplicationId.newInstance(0, 1)));
     entity.addEvent(event);
 
     TimelineEntities entities = new TimelineEntities();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f735024e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
index c165801..51604f0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/AppIdKeyConverter.java
@@ -82,7 +82,8 @@ public final class AppIdKeyConverter implements KeyConverter<String> {
         Bytes.toLong(appIdBytes, 0, Bytes.SIZEOF_LONG));
     int seqId = HBaseTimelineStorageUtils.invertInt(
         Bytes.toInt(appIdBytes, Bytes.SIZEOF_LONG, Bytes.SIZEOF_INT));
-    return ApplicationId.newInstance(clusterTs, seqId).toString();
+    return HBaseTimelineStorageUtils.convertApplicationIdToString(
+        ApplicationId.newInstance(clusterTs, seqId));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f735024e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
index 7e4713f..531420f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/HBaseTimelineStorageUtils.java
@@ -36,10 +36,13 @@ import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.client.Query;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationCompactionDimension;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.AggregationOperation;
 import org.apache.hadoop.yarn.server.timelineservice.storage.flow.Attribute;
 
+import java.text.NumberFormat;
+
 /**
  * A bunch of utility functions used in HBase TimelineService backend.
  */
@@ -228,6 +231,37 @@ public final class HBaseTimelineStorageUtils {
     return dayTimestamp;
   }
 
+  private static final ThreadLocal<NumberFormat> APP_ID_FORMAT =
+      new ThreadLocal<NumberFormat>() {
+        @Override
+        public NumberFormat initialValue() {
+          NumberFormat fmt = NumberFormat.getInstance();
+          fmt.setGroupingUsed(false);
+          fmt.setMinimumIntegerDigits(4);
+          return fmt;
+        }
+      };
+
+  /**
+   * A utility method that converts ApplicationId to string without using
+   * FastNumberFormat in order to avoid the incompatibility issue caused
+   * by mixing hadoop-common 2.5.1 and hadoop-yarn-api 3.0 in this module.
+   * This is a work-around implementation as discussed in YARN-6905.
+   *
+   * @param appId application id
+   * @return the string representation of the given application id
+   *
+   */
+  public static String convertApplicationIdToString(ApplicationId appId) {
+    StringBuilder sb = new StringBuilder(64);
+    sb.append(ApplicationId.appIdStrPrefix);
+    sb.append("_");
+    sb.append(appId.getClusterTimestamp());
+    sb.append('_');
+    sb.append(APP_ID_FORMAT.get().format(appId.getId()));
+    return sb.toString();
+  }
+
   /**
    * @param conf Yarn configuration. Used to see if there is an explicit config
    *          pointing to the HBase config file to read. It should not be null

http://git-wip-us.apache.org/repos/asf/hadoop/blob/f735024e/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java
new file mode 100644
index 0000000..73bc29e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestCustomApplicationIdConversion.java
@@ -0,0 +1,39 @@
+/**
+ * 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.yarn.server.timelineservice.storage.common;
+
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for HBaseTimelineStorageUtils.convertApplicationIdToString(),
+ * a custom conversion from ApplicationId to String that avoids the
+ * incompatibility issue caused by mixing hadoop-common 2.5.1 and
+ * hadoop-yarn-api 3.0. See YARN-6905.
+ */
+public class TestCustomApplicationIdConversion {
+  @Test
+  public void testConvertAplicationIdToString() {
+    ApplicationId applicationId = ApplicationId.newInstance(0, 1);
+    String applicationIdStr =
+        HBaseTimelineStorageUtils.convertApplicationIdToString(applicationId);
+    Assert.assertEquals(applicationId,
+        ApplicationId.fromString(applicationIdStr));
+  }
+}


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org


[2/2] hadoop git commit: YARN-6905. Addendum to fix TestTimelineReaderWebServicesHBaseStorage due to missing FastNumberFormat

Posted by va...@apache.org.
YARN-6905. Addendum to fix TestTimelineReaderWebServicesHBaseStorage due to missing FastNumberFormat

(cherry picked from commit e52f26e5cf0b7269635a685a0a70d20b190ce476)


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

Branch: refs/heads/YARN-5355_branch2
Commit: 2f81c11028c257ae3cd79c5ea98206b5263dc25f
Parents: f735024
Author: Varun Saxena <va...@apache.org>
Authored: Tue Aug 22 18:48:18 2017 +0530
Committer: Varun Saxena <va...@apache.org>
Committed: Thu Aug 31 02:57:24 2017 +0530

----------------------------------------------------------------------
 .../reader/TestTimelineReaderWebServicesHBaseStorage.java         | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2f81c110/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
index 6b4f014..4bbab25 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase-tests/src/test/java/org/apache/hadoop/yarn/server/timelineservice/reader/TestTimelineReaderWebServicesHBaseStorage.java
@@ -370,7 +370,8 @@ public class TestTimelineReaderWebServicesHBaseStorage
         ApplicationId appId =
             BuilderUtils.newApplicationId(timestamp, count++);
         ApplicationEntity appEntity = new ApplicationEntity();
-        appEntity.setId(appId.toString());
+        appEntity.setId(
+            HBaseTimelineStorageUtils.convertApplicationIdToString(appId));
         appEntity.setCreatedTime(timestamp);
 
         TimelineEvent created = new TimelineEvent();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org