Browse Source

YARN-6318. timeline service schema creator fails if executed from a remote machine (Sangjin Lee via Varun Saxena)

Varun Saxena 8 years ago
parent
commit
583250ee45

+ 4 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/main/java/org/apache/hadoop/yarn/server/timelineservice/storage/TimelineSchemaCreator.java

@@ -36,6 +36,7 @@ import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.Connection;
 import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.util.GenericOptionsParser;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.server.timelineservice.storage.application.ApplicationTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.apptoflow.AppToFlowTable;
 import org.apache.hadoop.yarn.server.timelineservice.storage.common.HBaseTimelineStorageUtils;
@@ -71,8 +72,10 @@ public final class TimelineSchemaCreator {
 
   public static void main(String[] args) throws Exception {
 
+    LOG.info("Starting the schema creation");
     Configuration hbaseConf =
-        HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(null);
+        HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(
+            new YarnConfiguration());
     // Grab input args and allow for -Dxyz style arguments
     String[] otherArgs = new GenericOptionsParser(hbaseConf, args)
         .getRemainingArgs();

+ 9 - 9
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

@@ -17,14 +17,18 @@
 
 package org.apache.hadoop.yarn.server.timelineservice.storage.common;
 
+import java.io.IOException;
 import java.net.MalformedURLException;
 import java.net.URL;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseConfiguration;
 import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.HRegionInfo;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.Tag;
 import org.apache.hadoop.hbase.util.Bytes;
@@ -33,15 +37,10 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 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 org.apache.hadoop.yarn.server.timelineservice.storage.flow.FlowRunTable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.text.NumberFormat;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Map;
 
 /**
  * A bunch of utility functions used in HBase TimelineService backend.
@@ -274,16 +273,17 @@ public final class HBaseTimelineStorageUtils {
    */
   public static Configuration getTimelineServiceHBaseConf(Configuration conf)
       throws MalformedURLException {
-    Configuration hbaseConf;
-
     if (conf == null) {
-      return HBaseConfiguration.create();
+      throw new NullPointerException();
     }
 
+    Configuration hbaseConf;
     String timelineServiceHBaseConfFileURL =
         conf.get(YarnConfiguration.TIMELINE_SERVICE_HBASE_CONFIGURATION_FILE);
     if (timelineServiceHBaseConfFileURL != null
         && timelineServiceHBaseConfFileURL.length() > 0) {
+      LOG.info("Using hbase configuration at " +
+          timelineServiceHBaseConfFileURL);
       // create a clone so that we don't mess with out input one
       hbaseConf = new Configuration(conf);
       Configuration plainHBaseConf = new Configuration(false);

+ 33 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-timelineservice-hbase/src/test/java/org/apache/hadoop/yarn/server/timelineservice/storage/common/TestHBaseTimelineStorageUtils.java

@@ -0,0 +1,33 @@
+/**
+ * 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.junit.Test;
+
+/**
+ * Unit tests for HBaseTimelineStorageUtils static methos.
+ */
+public class TestHBaseTimelineStorageUtils {
+
+  @Test(expected=NullPointerException.class)
+  public void testGetTimelineServiceHBaseConfNullArgument() throws Exception {
+    HBaseTimelineStorageUtils.getTimelineServiceHBaseConf(null);
+  }
+
+}