This is an automated email from the ASF dual-hosted git repository.

nihaljain pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hbase-connectors.git


The following commit(s) were added to refs/heads/master by this push:
     new 065a74f  HBASE-28534 Fix Kerberos authentication failure in local mode 
(#128)
065a74f is described below

commit 065a74f841430938ce0a94e4cafc5ec6446b04e3
Author: Junegunn Choi <[email protected]>
AuthorDate: Sat May 24 01:39:37 2025 +0900

    HBASE-28534 Fix Kerberos authentication failure in local mode (#128)
    
    Signed-off-by: Istvan Toth <[email protected]>
    Signed-off-by: Nihal Jain <[email protected]>
---
 .../org/apache/hadoop/hbase/spark/HBaseContext.scala     | 16 ----------------
 .../org/apache/hadoop/hbase/spark/NewHBaseRDD.scala      |  1 -
 .../hbase/spark/datasources/HBaseTableScanRDD.scala      |  2 --
 3 files changed, 19 deletions(-)

diff --git 
a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
 
b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
index c055bbc..6f33ba3 100644
--- 
a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
+++ 
b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/HBaseContext.scala
@@ -65,7 +65,6 @@ class HBaseContext(
     with Logging {
 
   @transient var tmpHdfsConfiguration: Configuration = config
-  @transient var appliedCredentials = false
   @transient val job = Job.getInstance(config)
   TableMapReduceUtil.initCredentials(job)
   val broadcastedConf = sc.broadcast(new SerializableWritable(config))
@@ -223,16 +222,6 @@ class HBaseContext(
           }))
   }
 
-  def applyCreds[T]() {
-    if (!appliedCredentials) {
-      appliedCredentials = true
-
-      @transient val ugi = UserGroupInformation.getCurrentUser
-      // specify that this is a proxy user
-      ugi.setAuthenticationMethod(AuthenticationMethod.PROXY)
-    }
-  }
-
   /**
    * A simple abstraction over the HBaseContext.streamMapPartition method.
    *
@@ -470,9 +459,6 @@ class HBaseContext(
       f: (Iterator[T], Connection) => Unit) = {
 
     val config = getConf(configBroadcast)
-
-    applyCreds
-    // specify that this is a proxy user
     val smartConn = HBaseConnectionCache.getConnection(config)
     try {
       f(it, smartConn.connection)
@@ -511,8 +497,6 @@ class HBaseContext(
       mp: (Iterator[K], Connection) => Iterator[U]): Iterator[U] = {
 
     val config = getConf(configBroadcast)
-    applyCreds
-
     val smartConn = HBaseConnectionCache.getConnection(config)
     try {
       mp(it, smartConn.connection)
diff --git 
a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/NewHBaseRDD.scala
 
b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/NewHBaseRDD.scala
index aeb502d..de536fd 100644
--- 
a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/NewHBaseRDD.scala
+++ 
b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/NewHBaseRDD.scala
@@ -34,7 +34,6 @@ class NewHBaseRDD[K, V](
     extends NewHadoopRDD(sc, inputFormatClass, keyClass, valueClass, __conf) {
 
   override def compute(theSplit: Partition, context: TaskContext): 
InterruptibleIterator[(K, V)] = {
-    hBaseContext.applyCreds()
     super.compute(theSplit, context)
   }
 }
diff --git 
a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
 
b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
index 0d7ec0d..7ceb7a4 100644
--- 
a/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
+++ 
b/spark/hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseTableScanRDD.scala
@@ -144,7 +144,6 @@ class HBaseTableScanRDD(
               rowkeySet.add(y.mkString("Array(", ", ", ")"))
             }
         }
-        hbaseContext.applyCreds()
         val tmp = tbr.get(gets)
         rddResources.addResource(tmp)
         toResultIterator(tmp)
@@ -250,7 +249,6 @@ class HBaseTableScanRDD(
     val rIts = scans.par
       .map {
         scan =>
-          hbaseContext.applyCreds()
           val scanner = tableResource.getScanner(scan)
           rddResources.addResource(scanner)
           scanner

Reply via email to