You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by md...@apache.org on 2018/05/09 03:21:00 UTC

[1/6] hbase git commit: Revert "change getConf as first sequence instead of jobContext in TableOutputFormat.checkOutputSpec, add unit tests"

Repository: hbase
Updated Branches:
  refs/heads/branch-2 8d60a89c3 -> 421ed6ca5
  refs/heads/branch-2.0 9e3b28804 -> c15fd6b00
  refs/heads/master 0dcae90ea -> c51e9adc7


Revert "change getConf as first sequence instead of jobContext in TableOutputFormat.checkOutputSpec, add unit tests"

This reverts commit 9e68719014a62f37e7559329e44e2df49738ef6c.


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

Branch: refs/heads/branch-2
Commit: d91908b0d46156fa364ba11d476b9cdbc01d0411
Parents: 8d60a89
Author: Mike Drob <md...@apache.org>
Authored: Tue May 8 22:18:42 2018 -0500
Committer: Mike Drob <md...@apache.org>
Committed: Tue May 8 22:18:42 2018 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/d91908b0/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
index 78be5af..0a1928b 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
@@ -174,9 +174,9 @@ implements Configurable {
   @Override
   public void checkOutputSpecs(JobContext context) throws IOException,
       InterruptedException {
-    Configuration hConf = getConf();
-    if (hConf == null) {
-      hConf = context.getConfiguration();
+    Configuration hConf = context.getConfiguration();
+    if(hConf == null) {
+      hConf = this.conf;
     }
 
     try (Admin admin = ConnectionFactory.createConnection(hConf).getAdmin()) {


[3/6] hbase git commit: Revert "change getConf as first sequence instead of jobContext in TableOutputFormat.checkOutputSpec, add unit tests

Posted by md...@apache.org.
Revert "change getConf as first sequence instead of jobContext in TableOutputFormat.checkOutputSpec, add unit tests

This reverts commit 9e3b28804d28ad586d12df24a2e5417c25cff857.


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

Branch: refs/heads/branch-2.0
Commit: 18ca1fcb560236b8c9a0ca057f163ebdaab89f0e
Parents: 9e3b288
Author: Mike Drob <md...@apache.org>
Authored: Tue May 8 22:19:31 2018 -0500
Committer: Mike Drob <md...@apache.org>
Committed: Tue May 8 22:19:31 2018 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/18ca1fcb/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
index 78be5af..0a1928b 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
@@ -174,9 +174,9 @@ implements Configurable {
   @Override
   public void checkOutputSpecs(JobContext context) throws IOException,
       InterruptedException {
-    Configuration hConf = getConf();
-    if (hConf == null) {
-      hConf = context.getConfiguration();
+    Configuration hConf = context.getConfiguration();
+    if(hConf == null) {
+      hConf = this.conf;
     }
 
     try (Admin admin = ConnectionFactory.createConnection(hConf).getAdmin()) {


[6/6] hbase git commit: HBASE-20521 change getConf as first sequence instead of jobContext in TableOutputFormat.checkOutputSpec, add unit tests

Posted by md...@apache.org.
HBASE-20521 change getConf as first sequence instead of jobContext in TableOutputFormat.checkOutputSpec, add unit tests


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

Branch: refs/heads/master
Commit: c51e9adc78492cd9a12b513bdc83a4a10a043c52
Parents: 82e6fae
Author: Mike Drob <md...@apache.org>
Authored: Tue May 8 22:20:12 2018 -0500
Committer: Mike Drob <md...@apache.org>
Committed: Tue May 8 22:20:12 2018 -0500

----------------------------------------------------------------------
 .../hbase/mapreduce/TableOutputFormat.java      |   4 +-
 .../hbase/spark/TableOutputFormatSuite.scala    | 130 +++++++++++++++++++
 2 files changed, 132 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c51e9adc/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
index 4eb2654..78be5af 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
@@ -174,9 +174,9 @@ implements Configurable {
   @Override
   public void checkOutputSpecs(JobContext context) throws IOException,
       InterruptedException {
-    Configuration hConf = context.getConfiguration();
+    Configuration hConf = getConf();
     if (hConf == null) {
-      hConf = this.conf;
+      hConf = context.getConfiguration();
     }
 
     try (Admin admin = ConnectionFactory.createConnection(hConf).getAdmin()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/c51e9adc/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/TableOutputFormatSuite.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/TableOutputFormatSuite.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/TableOutputFormatSuite.scala
new file mode 100644
index 0000000..b306f08
--- /dev/null
+++ b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/TableOutputFormatSuite.scala
@@ -0,0 +1,130 @@
+/*
+ * 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.spark
+
+
+import java.text.SimpleDateFormat
+import java.util.{Date, Locale}
+
+import org.apache.hadoop.hbase.mapreduce.TableOutputFormat
+import org.apache.hadoop.hbase.{HBaseTestingUtility, TableName, TableNotFoundException}
+import org.apache.hadoop.hbase.util.Bytes
+import org.apache.hadoop.mapreduce.{Job, TaskAttemptID, TaskType}
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.spark.{SparkConf, SparkContext}
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
+
+import scala.util.{Failure, Success, Try}
+
+
+// Unit tests for HBASE-20521: change get configuration(TableOutputFormat.conf) object first sequence from jobContext to getConf
+// this suite contains two tests, one for normal case(getConf return null, use jobContext), create new TableOutputformat object without init TableOutputFormat.conf object,
+// configuration object inside checkOutputSpecs came from jobContext.
+// The other one(getConf return conf object) we manually call "setConf" to init TableOutputFormat.conf, for making it more straight forward, we specify a nonexistent table
+// name in conf object, checkOutputSpecs will then throw TableNotFoundException exception
+class TableOutputFormatSuite extends FunSuite with
+  BeforeAndAfterEach with BeforeAndAfterAll with Logging{
+  @transient var sc: SparkContext = null
+  var TEST_UTIL = new HBaseTestingUtility
+
+  val tableName = "TableOutputFormatTest"
+  val tableNameTest = "NonExistentTable"
+  val columnFamily = "cf"
+
+  override protected def beforeAll(): Unit = {
+    TEST_UTIL.startMiniCluster
+
+    logInfo(" - minicluster started")
+    try {
+      TEST_UTIL.deleteTable(TableName.valueOf(tableName))
+    }
+    catch {
+      case e: Exception => logInfo(" - no table " + tableName + " found")
+    }
+
+    TEST_UTIL.createTable(TableName.valueOf(tableName), Bytes.toBytes(columnFamily))
+    logInfo(" - created table")
+
+    //set "validateOutputSpecs" true anyway, force to validate output spec
+    val sparkConf = new SparkConf()
+      .setMaster("local")
+      .setAppName("test")
+
+    sc = new SparkContext(sparkConf)
+  }
+
+  override protected def afterAll(): Unit = {
+    logInfo(" - delete table: " + tableName)
+    TEST_UTIL.deleteTable(TableName.valueOf(tableName))
+    logInfo(" - shutting down minicluster")
+    TEST_UTIL.shutdownMiniCluster()
+
+    TEST_UTIL.cleanupTestDir()
+    sc.stop()
+  }
+
+  def getJobContext() = {
+    val hConf = TEST_UTIL.getConfiguration
+    hConf.set(TableOutputFormat.OUTPUT_TABLE, tableName)
+    val job = Job.getInstance(hConf)
+    job.setOutputFormatClass(classOf[TableOutputFormat[String]])
+
+    val jobTrackerId = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(new Date())
+    val jobAttemptId = new TaskAttemptID(jobTrackerId, 1, TaskType.MAP, 0, 0)
+    new TaskAttemptContextImpl(job.getConfiguration, jobAttemptId)
+  }
+
+  // Mock up jobContext object and execute actions in "write" function
+  // from "org.apache.spark.internal.io.SparkHadoopMapReduceWriter"
+  // this case should run normally without any exceptions
+  test("TableOutputFormat.checkOutputSpecs test without setConf called, should return true and without exceptions") {
+    val jobContext = getJobContext()
+    val format = jobContext.getOutputFormatClass
+    val jobFormat = format.newInstance
+    Try {
+      jobFormat.checkOutputSpecs(jobContext)
+    } match {
+      case Success(_) => assert(true)
+      case Failure(_) => assert(false)
+    }
+  }
+
+  // Set configuration externally, checkOutputSpec should use configuration object set by "SetConf" method
+  // rather than jobContext, this case should throw "TableNotFoundException" exception
+  test("TableOutputFormat.checkOutputSpecs test without setConf called, should throw TableNotFoundException") {
+    val jobContext = getJobContext()
+    val format = jobContext.getOutputFormatClass
+    val jobFormat = format.newInstance
+
+    val hConf = TEST_UTIL.getConfiguration
+    hConf.set(TableOutputFormat.OUTPUT_TABLE, tableNameTest)
+    jobFormat.asInstanceOf[TableOutputFormat[String]].setConf(hConf)
+    Try {
+      jobFormat.checkOutputSpecs(jobContext)
+    } match {
+      case Success(_) => assert(false)
+      case Failure(e: Exception) => {
+        if(e.isInstanceOf[TableNotFoundException])
+          assert(true)
+        else
+          assert(false)
+      }
+    }
+  }
+
+}


[5/6] hbase git commit: Revert "change getConf as first sequence instead of jobContext in TableOutputFormat.checkOutputSpec, add unit tests"

Posted by md...@apache.org.
Revert "change getConf as first sequence instead of jobContext in TableOutputFormat.checkOutputSpec, add unit tests"

This reverts commit b748ea3b0d3a074b976fa0ae59fc591303e805a0.


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

Branch: refs/heads/master
Commit: 82e6fae1de5a5f700c0237dbaa8994f409833370
Parents: 0dcae90
Author: Mike Drob <md...@apache.org>
Authored: Tue May 8 22:20:06 2018 -0500
Committer: Mike Drob <md...@apache.org>
Committed: Tue May 8 22:20:06 2018 -0500

----------------------------------------------------------------------
 .../hbase/mapreduce/TableOutputFormat.java      |   4 +-
 .../hbase/spark/TableOutputFormatSuite.scala    | 130 -------------------
 2 files changed, 2 insertions(+), 132 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/82e6fae1/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
index 78be5af..4eb2654 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
@@ -174,9 +174,9 @@ implements Configurable {
   @Override
   public void checkOutputSpecs(JobContext context) throws IOException,
       InterruptedException {
-    Configuration hConf = getConf();
+    Configuration hConf = context.getConfiguration();
     if (hConf == null) {
-      hConf = context.getConfiguration();
+      hConf = this.conf;
     }
 
     try (Admin admin = ConnectionFactory.createConnection(hConf).getAdmin()) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/82e6fae1/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/TableOutputFormatSuite.scala
----------------------------------------------------------------------
diff --git a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/TableOutputFormatSuite.scala b/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/TableOutputFormatSuite.scala
deleted file mode 100644
index b306f08..0000000
--- a/hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/TableOutputFormatSuite.scala
+++ /dev/null
@@ -1,130 +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.spark
-
-
-import java.text.SimpleDateFormat
-import java.util.{Date, Locale}
-
-import org.apache.hadoop.hbase.mapreduce.TableOutputFormat
-import org.apache.hadoop.hbase.{HBaseTestingUtility, TableName, TableNotFoundException}
-import org.apache.hadoop.hbase.util.Bytes
-import org.apache.hadoop.mapreduce.{Job, TaskAttemptID, TaskType}
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
-import org.apache.spark.{SparkConf, SparkContext}
-import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite}
-
-import scala.util.{Failure, Success, Try}
-
-
-// Unit tests for HBASE-20521: change get configuration(TableOutputFormat.conf) object first sequence from jobContext to getConf
-// this suite contains two tests, one for normal case(getConf return null, use jobContext), create new TableOutputformat object without init TableOutputFormat.conf object,
-// configuration object inside checkOutputSpecs came from jobContext.
-// The other one(getConf return conf object) we manually call "setConf" to init TableOutputFormat.conf, for making it more straight forward, we specify a nonexistent table
-// name in conf object, checkOutputSpecs will then throw TableNotFoundException exception
-class TableOutputFormatSuite extends FunSuite with
-  BeforeAndAfterEach with BeforeAndAfterAll with Logging{
-  @transient var sc: SparkContext = null
-  var TEST_UTIL = new HBaseTestingUtility
-
-  val tableName = "TableOutputFormatTest"
-  val tableNameTest = "NonExistentTable"
-  val columnFamily = "cf"
-
-  override protected def beforeAll(): Unit = {
-    TEST_UTIL.startMiniCluster
-
-    logInfo(" - minicluster started")
-    try {
-      TEST_UTIL.deleteTable(TableName.valueOf(tableName))
-    }
-    catch {
-      case e: Exception => logInfo(" - no table " + tableName + " found")
-    }
-
-    TEST_UTIL.createTable(TableName.valueOf(tableName), Bytes.toBytes(columnFamily))
-    logInfo(" - created table")
-
-    //set "validateOutputSpecs" true anyway, force to validate output spec
-    val sparkConf = new SparkConf()
-      .setMaster("local")
-      .setAppName("test")
-
-    sc = new SparkContext(sparkConf)
-  }
-
-  override protected def afterAll(): Unit = {
-    logInfo(" - delete table: " + tableName)
-    TEST_UTIL.deleteTable(TableName.valueOf(tableName))
-    logInfo(" - shutting down minicluster")
-    TEST_UTIL.shutdownMiniCluster()
-
-    TEST_UTIL.cleanupTestDir()
-    sc.stop()
-  }
-
-  def getJobContext() = {
-    val hConf = TEST_UTIL.getConfiguration
-    hConf.set(TableOutputFormat.OUTPUT_TABLE, tableName)
-    val job = Job.getInstance(hConf)
-    job.setOutputFormatClass(classOf[TableOutputFormat[String]])
-
-    val jobTrackerId = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US).format(new Date())
-    val jobAttemptId = new TaskAttemptID(jobTrackerId, 1, TaskType.MAP, 0, 0)
-    new TaskAttemptContextImpl(job.getConfiguration, jobAttemptId)
-  }
-
-  // Mock up jobContext object and execute actions in "write" function
-  // from "org.apache.spark.internal.io.SparkHadoopMapReduceWriter"
-  // this case should run normally without any exceptions
-  test("TableOutputFormat.checkOutputSpecs test without setConf called, should return true and without exceptions") {
-    val jobContext = getJobContext()
-    val format = jobContext.getOutputFormatClass
-    val jobFormat = format.newInstance
-    Try {
-      jobFormat.checkOutputSpecs(jobContext)
-    } match {
-      case Success(_) => assert(true)
-      case Failure(_) => assert(false)
-    }
-  }
-
-  // Set configuration externally, checkOutputSpec should use configuration object set by "SetConf" method
-  // rather than jobContext, this case should throw "TableNotFoundException" exception
-  test("TableOutputFormat.checkOutputSpecs test without setConf called, should throw TableNotFoundException") {
-    val jobContext = getJobContext()
-    val format = jobContext.getOutputFormatClass
-    val jobFormat = format.newInstance
-
-    val hConf = TEST_UTIL.getConfiguration
-    hConf.set(TableOutputFormat.OUTPUT_TABLE, tableNameTest)
-    jobFormat.asInstanceOf[TableOutputFormat[String]].setConf(hConf)
-    Try {
-      jobFormat.checkOutputSpecs(jobContext)
-    } match {
-      case Success(_) => assert(false)
-      case Failure(e: Exception) => {
-        if(e.isInstanceOf[TableNotFoundException])
-          assert(true)
-        else
-          assert(false)
-      }
-    }
-  }
-
-}


[4/6] hbase git commit: HBASE-20521 change getConf as first sequence instead of jobContext in TableOutputFormat.checkOutputSpec, add unit tests

Posted by md...@apache.org.
HBASE-20521 change getConf as first sequence instead of jobContext in TableOutputFormat.checkOutputSpec, add unit tests


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

Branch: refs/heads/branch-2.0
Commit: c15fd6b00a37ed927c33cb13401dc8dfe234e595
Parents: 18ca1fc
Author: Mike Drob <md...@apache.org>
Authored: Tue May 8 22:18:48 2018 -0500
Committer: Mike Drob <md...@apache.org>
Committed: Tue May 8 22:19:44 2018 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c15fd6b0/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
index 0a1928b..78be5af 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
@@ -174,9 +174,9 @@ implements Configurable {
   @Override
   public void checkOutputSpecs(JobContext context) throws IOException,
       InterruptedException {
-    Configuration hConf = context.getConfiguration();
-    if(hConf == null) {
-      hConf = this.conf;
+    Configuration hConf = getConf();
+    if (hConf == null) {
+      hConf = context.getConfiguration();
     }
 
     try (Admin admin = ConnectionFactory.createConnection(hConf).getAdmin()) {


[2/6] hbase git commit: HBASE-20521 change getConf as first sequence instead of jobContext in TableOutputFormat.checkOutputSpec, add unit tests

Posted by md...@apache.org.
HBASE-20521 change getConf as first sequence instead of jobContext in TableOutputFormat.checkOutputSpec, add unit tests


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

Branch: refs/heads/branch-2
Commit: 421ed6ca513d09e1345a73c5da1289798a25ae47
Parents: d91908b
Author: Mike Drob <md...@apache.org>
Authored: Tue May 8 22:18:48 2018 -0500
Committer: Mike Drob <md...@apache.org>
Committed: Tue May 8 22:18:48 2018 -0500

----------------------------------------------------------------------
 .../org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/421ed6ca/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
index 0a1928b..78be5af 100644
--- a/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
+++ b/hbase-mapreduce/src/main/java/org/apache/hadoop/hbase/mapreduce/TableOutputFormat.java
@@ -174,9 +174,9 @@ implements Configurable {
   @Override
   public void checkOutputSpecs(JobContext context) throws IOException,
       InterruptedException {
-    Configuration hConf = context.getConfiguration();
-    if(hConf == null) {
-      hConf = this.conf;
+    Configuration hConf = getConf();
+    if (hConf == null) {
+      hConf = context.getConfiguration();
     }
 
     try (Admin admin = ConnectionFactory.createConnection(hConf).getAdmin()) {