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

abstractdog pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new f753d25f600 HIVE-28135: Remove HiveIOExceptionHandlerUtil (#5163)
f753d25f600 is described below

commit f753d25f6003586d596382a3cbda78ff593c5245
Author: Bodor Laszlo <[email protected]>
AuthorDate: Mon Sep 15 08:36:18 2025 +0200

    HIVE-28135: Remove HiveIOExceptionHandlerUtil (#5163)
---
 .../java/org/apache/hadoop/hive/conf/HiveConf.java |   4 -
 .../hive/ql/io/BucketizedHiveRecordReader.java     |   8 +-
 .../hive/ql/io/HiveContextAwareRecordReader.java   | 102 ++++++++---------
 .../apache/hadoop/hive/ql/io/HiveInputFormat.java  |   4 +-
 .../hadoop/hive/ql/io/SymlinkTextInputFormat.java  |  10 +-
 .../test/results/clientnegative/serde_regex2.q.out |  24 ++--
 .../hadoop/hive/io/HiveIOExceptionHandler.java     |  52 ---------
 .../hive/io/HiveIOExceptionHandlerChain.java       | 124 ---------------------
 .../hadoop/hive/io/HiveIOExceptionHandlerUtil.java |  88 ---------------
 .../hive/io/HiveIOExceptionNextHandleResult.java   |  55 ---------
 .../hadoop/hive/shims/HadoopShimsSecure.java       |  20 +---
 11 files changed, 63 insertions(+), 428 deletions(-)

diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java 
b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index a9330289977..e0e1bdbcb5c 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -3736,10 +3736,6 @@ public static enum ConfVars {
     HIVE_REWORK_MAPREDWORK("hive.rework.mapredwork", false,
         "should rework the mapred work or not.\n" +
         "This is first introduced by SymlinkTextInputFormat to replace symlink 
files with real paths at compile time."),
-    HIVE_IO_EXCEPTION_HANDLERS("hive.io.exception.handlers", "",
-        "A list of io exception handler class names. This is used\n" +
-        "to construct a list exception handlers to handle exceptions thrown\n" 
+
-        "by record readers"),
 
     // logging configuration
     HIVE_LOG4J_FILE("hive.log4j.file", "",
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/io/BucketizedHiveRecordReader.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/BucketizedHiveRecordReader.java
index bff0933de11..416ac22006c 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/BucketizedHiveRecordReader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/BucketizedHiveRecordReader.java
@@ -20,7 +20,6 @@
 
 import java.io.IOException;
 
-import org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.mapred.FileSplit;
@@ -118,12 +117,7 @@ protected boolean initNextRecordReader() throws 
IOException {
     }
 
     // get a record reader for the idx-th chunk
-    try {
-      recordReader = inputFormat.getRecordReader(split.getSplit(idx), jobConf,
-          reporter);
-    } catch (Exception e) {
-      recordReader = 
HiveIOExceptionHandlerUtil.handleRecordReaderCreationException(e, jobConf);
-    }
+    recordReader = inputFormat.getRecordReader(split.getSplit(idx), jobConf, 
reporter);
 
     // if we're performing a binary search, we need to restart it
     if (isSorted) {
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java
index ef1f3e075cb..ec19dcab737 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveContextAwareRecordReader.java
@@ -29,7 +29,6 @@
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.exec.FooterBuffer;
 import org.apache.hadoop.hive.ql.io.IOContext.Comparison;
@@ -320,65 +319,60 @@ public boolean doNext(K key, V value) throws IOException {
       }
     }
 
-    try {
-
-      /**
-       * When start reading new file, check header, footer rows.
-       * If file contains header, skip header lines before reading the records.
-       * If file contains footer, used a FooterBuffer to remove footer lines
-       * at the end of the table file.
-       **/
-      if (this.ioCxtRef.getCurrentBlockStart() == 0) {
-
-        // Check if the table file has header to skip.
-        footerBuffer = null;
-        Path filePath = this.ioCxtRef.getInputPath();
-        PartitionDesc part = null;
-        try {
-          if (pathToPartitionInfo == null) {
-            pathToPartitionInfo = Utilities
-              .getMapWork(jobConf).getPathToPartitionInfo();
-          }
-          part = HiveFileFormatUtils
-              .getFromPathRecursively(pathToPartitionInfo,
-                  filePath, IOPrepareCache.get().getPartitionDescMap());
-        } catch (AssertionError ae) {
-          LOG.info("Cannot get partition description from " + 
this.ioCxtRef.getInputPath()
-              + "because " + ae.getMessage());
-          part = null;
-        } catch (Exception e) {
-          LOG.info("Cannot get partition description from " + 
this.ioCxtRef.getInputPath()
-              + "because " + e.getMessage());
-          part = null;
-        }
-        TableDesc table = (part == null) ? null : part.getTableDesc();
-        // In TextFormat, skipping is already taken care of as part of 
SkippingTextInputFormat.
-        // This code will be also called from LLAP when pipeline is 
non-vectorized and cannot create wrapper.
-        if (table != null && 
!TextInputFormat.class.isAssignableFrom(part.getInputFileFormatClass())) {
-          headerCount = Utilities.getHeaderCount(table);
-          footerCount = Utilities.getFooterCount(table, jobConf);
+    /**
+     * When start reading new file, check header, footer rows.
+     * If file contains header, skip header lines before reading the records.
+     * If file contains footer, used a FooterBuffer to remove footer lines
+     * at the end of the table file.
+     **/
+    if (this.ioCxtRef.getCurrentBlockStart() == 0) {
+
+      // Check if the table file has header to skip.
+      footerBuffer = null;
+      Path filePath = this.ioCxtRef.getInputPath();
+      PartitionDesc part = null;
+      try {
+        if (pathToPartitionInfo == null) {
+          pathToPartitionInfo = Utilities
+            .getMapWork(jobConf).getPathToPartitionInfo();
         }
+        part = HiveFileFormatUtils
+            .getFromPathRecursively(pathToPartitionInfo,
+                filePath, IOPrepareCache.get().getPartitionDescMap());
+      } catch (AssertionError ae) {
+        LOG.info("Cannot get partition description from " + 
this.ioCxtRef.getInputPath()
+            + "because " + ae.getMessage());
+        part = null;
+      } catch (Exception e) {
+        LOG.info("Cannot get partition description from " + 
this.ioCxtRef.getInputPath()
+            + "because " + e.getMessage());
+        part = null;
+      }
+      TableDesc table = (part == null) ? null : part.getTableDesc();
+      // In TextFormat, skipping is already taken care of as part of 
SkippingTextInputFormat.
+      // This code will be also called from LLAP when pipeline is 
non-vectorized and cannot create wrapper.
+      if (table != null && 
!TextInputFormat.class.isAssignableFrom(part.getInputFileFormatClass())) {
+        headerCount = Utilities.getHeaderCount(table);
+        footerCount = Utilities.getFooterCount(table, jobConf);
+      }
 
-        // If input contains header, skip header.
-        if (!Utilities.skipHeader(recordReader, headerCount, key, value)) {
+      // If input contains header, skip header.
+      if (!Utilities.skipHeader(recordReader, headerCount, key, value)) {
+        return false;
+      }
+      if (footerCount > 0) {
+        footerBuffer = new FooterBuffer();
+        if (!footerBuffer.initializeBuffer(jobConf, recordReader, footerCount, 
key, value)) {
           return false;
         }
-        if (footerCount > 0) {
-          footerBuffer = new FooterBuffer();
-          if (!footerBuffer.initializeBuffer(jobConf, recordReader, 
footerCount, key, value)) {
-            return false;
-          }
-        }
       }
-      if (footerBuffer == null) {
+    }
+    if (footerBuffer == null) {
 
-        // Table files don't have footer rows.
-        return recordReader.next(key,  value);
-      } else {
-        return footerBuffer.updateBuffer(jobConf, recordReader, key, value);
-      }
-    } catch (Exception e) {
-      return HiveIOExceptionHandlerUtil.handleRecordReaderNextException(e, 
jobConf);
+      // Table files don't have footer rows.
+      return recordReader.next(key,  value);
+    } else {
+      return footerBuffer.updateBuffer(jobConf, recordReader, key, value);
     }
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
index 6e43e9a7994..076fbf7d392 100755
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/HiveInputFormat.java
@@ -34,7 +34,6 @@
 import org.apache.hadoop.hive.common.type.TimestampTZUtil;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil;
 import org.apache.hadoop.hive.llap.io.api.LlapIo;
 import org.apache.hadoop.hive.llap.io.api.LlapProxy;
 import org.apache.hadoop.hive.ql.exec.LimitOperator;
@@ -475,8 +474,7 @@ public RecordReader getRecordReader(InputSplit split, 
JobConf job,
         LOG.info("Ignoring exception while getting record reader as limit is 
reached", rootCause);
         innerReader = new NullRowsRecordReader(job, split);
       } else {
-        innerReader = HiveIOExceptionHandlerUtil
-            .handleRecordReaderCreationException(e, job);
+        throw new IOException("Exception caught while creating the inner 
reader", e);
       }
     }
     HiveRecordReader<K,V> rr = new HiveRecordReader(innerReader, job);
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/io/SymlinkTextInputFormat.java 
b/ql/src/java/org/apache/hadoop/hive/ql/io/SymlinkTextInputFormat.java
index 3f8f75f5ccd..013826e9560 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/io/SymlinkTextInputFormat.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/io/SymlinkTextInputFormat.java
@@ -31,7 +31,6 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.common.FileUtils;
-import org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.FileInputFormat;
@@ -102,14 +101,7 @@ public RecordReader<LongWritable, Text> getRecordReader(
     // The target data is in TextInputFormat.
     TextInputFormat inputFormat = new TextInputFormat();
     inputFormat.configure(job);
-    RecordReader innerReader = null;
-    try {
-      innerReader = inputFormat.getRecordReader(targetSplit, job,
-          reporter);
-    } catch (Exception e) {
-      innerReader = HiveIOExceptionHandlerUtil
-          .handleRecordReaderCreationException(e, job);
-    }
+    RecordReader innerReader = inputFormat.getRecordReader(targetSplit, job, 
reporter);
     HiveRecordReader rr = new HiveRecordReader(innerReader, job);
     rr.initIOContext((FileSplit)targetSplit, job, TextInputFormat.class, 
innerReader);
     return rr;
diff --git a/ql/src/test/results/clientnegative/serde_regex2.q.out 
b/ql/src/test/results/clientnegative/serde_regex2.q.out
index 9af29ff8394..32574c39806 100644
--- a/ql/src/test/results/clientnegative/serde_regex2.q.out
+++ b/ql/src/test/results/clientnegative/serde_regex2.q.out
@@ -61,21 +61,17 @@ PREHOOK: type: QUERY
 PREHOOK: Input: default@serde_regex
 #### A masked pattern was here ####
 Status: Failed
-Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task 
failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error 
while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: 
org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: 
java.io.IOException: org.apache.hadoop.hive.serde2.SerDeException: Number of 
matching groups doesn't match the number of columns
+Vertex failed, vertexName=Map 1, vertexId=vertex_#ID#, diagnostics=[Task 
failed, taskId=task_#ID#, diagnostics=[TaskAttempt 0 failed, info=[Error: Error 
while running task ( failure ) : attempt_#ID#:java.lang.RuntimeException: 
org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: 
org.apache.hadoop.hive.serde2.SerDeException: Number of matching groups doesn't 
match the number of columns
 #### A masked pattern was here ####
-Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: 
java.io.IOException: java.io.IOException: 
org.apache.hadoop.hive.serde2.SerDeException: Number of matching groups doesn't 
match the number of columns
-#### A masked pattern was here ####
-Caused by: java.io.IOException: java.io.IOException: 
org.apache.hadoop.hive.serde2.SerDeException: Number of matching groups doesn't 
match the number of columns
+Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: 
java.io.IOException: org.apache.hadoop.hive.serde2.SerDeException: Number of 
matching groups doesn't match the number of columns
 #### A masked pattern was here ####
 Caused by: java.io.IOException: org.apache.hadoop.hive.serde2.SerDeException: 
Number of matching groups doesn't match the number of columns
 #### A masked pattern was here ####
 Caused by: org.apache.hadoop.hive.serde2.SerDeException: Number of matching 
groups doesn't match the number of columns
 #### A masked pattern was here ####
-], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : 
attempt_#ID#:java.lang.RuntimeException: 
org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: 
java.io.IOException: org.apache.hadoop.hive.serde2.SerDeException: Number of 
matching groups doesn't match the number of columns
-#### A masked pattern was here ####
-Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: 
java.io.IOException: java.io.IOException: 
org.apache.hadoop.hive.serde2.SerDeException: Number of matching groups doesn't 
match the number of columns
+], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : 
attempt_#ID#:java.lang.RuntimeException: 
org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: 
org.apache.hadoop.hive.serde2.SerDeException: Number of matching groups doesn't 
match the number of columns
 #### A masked pattern was here ####
-Caused by: java.io.IOException: java.io.IOException: 
org.apache.hadoop.hive.serde2.SerDeException: Number of matching groups doesn't 
match the number of columns
+Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: 
java.io.IOException: org.apache.hadoop.hive.serde2.SerDeException: Number of 
matching groups doesn't match the number of columns
 #### A masked pattern was here ####
 Caused by: java.io.IOException: org.apache.hadoop.hive.serde2.SerDeException: 
Number of matching groups doesn't match the number of columns
 #### A masked pattern was here ####
@@ -84,21 +80,17 @@ Caused by: org.apache.hadoop.hive.serde2.SerDeException: 
Number of matching grou
 ]], Vertex did not succeed due to OWN_TASK_FAILURE, failedTasks:1 
killedTasks:0, Vertex vertex_#ID# [Map 1] killed/failed due to:OWN_TASK_FAILURE]
 [Masked Vertex killed due to OTHER_VERTEX_FAILURE]
 DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:1
-FAILED: Execution Error, return code 2 from 
org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, 
vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, 
diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( 
failure ) : attempt_#ID#:java.lang.RuntimeException: 
org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: 
java.io.IOException: org.apache.hadoop.hive.serde2.SerDeException: Number of 
matching groups doesn't match [...]
+FAILED: Execution Error, return code 2 from 
org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 1, 
vertexId=vertex_#ID#, diagnostics=[Task failed, taskId=task_#ID#, 
diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( 
failure ) : attempt_#ID#:java.lang.RuntimeException: 
org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: 
org.apache.hadoop.hive.serde2.SerDeException: Number of matching groups doesn't 
match the number of columns
 #### A masked pattern was here ####
-Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: 
java.io.IOException: java.io.IOException: 
org.apache.hadoop.hive.serde2.SerDeException: Number of matching groups doesn't 
match the number of columns
-#### A masked pattern was here ####
-Caused by: java.io.IOException: java.io.IOException: 
org.apache.hadoop.hive.serde2.SerDeException: Number of matching groups doesn't 
match the number of columns
+Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: 
java.io.IOException: org.apache.hadoop.hive.serde2.SerDeException: Number of 
matching groups doesn't match the number of columns
 #### A masked pattern was here ####
 Caused by: java.io.IOException: org.apache.hadoop.hive.serde2.SerDeException: 
Number of matching groups doesn't match the number of columns
 #### A masked pattern was here ####
 Caused by: org.apache.hadoop.hive.serde2.SerDeException: Number of matching 
groups doesn't match the number of columns
 #### A masked pattern was here ####
-], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : 
attempt_#ID#:java.lang.RuntimeException: 
org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: 
java.io.IOException: org.apache.hadoop.hive.serde2.SerDeException: Number of 
matching groups doesn't match the number of columns
-#### A masked pattern was here ####
-Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: 
java.io.IOException: java.io.IOException: 
org.apache.hadoop.hive.serde2.SerDeException: Number of matching groups doesn't 
match the number of columns
+], TaskAttempt 1 failed, info=[Error: Error while running task ( failure ) : 
attempt_#ID#:java.lang.RuntimeException: 
org.apache.hadoop.hive.ql.metadata.HiveException: java.io.IOException: 
org.apache.hadoop.hive.serde2.SerDeException: Number of matching groups doesn't 
match the number of columns
 #### A masked pattern was here ####
-Caused by: java.io.IOException: java.io.IOException: 
org.apache.hadoop.hive.serde2.SerDeException: Number of matching groups doesn't 
match the number of columns
+Caused by: org.apache.hadoop.hive.ql.metadata.HiveException: 
java.io.IOException: org.apache.hadoop.hive.serde2.SerDeException: Number of 
matching groups doesn't match the number of columns
 #### A masked pattern was here ####
 Caused by: java.io.IOException: org.apache.hadoop.hive.serde2.SerDeException: 
Number of matching groups doesn't match the number of columns
 #### A masked pattern was here ####
diff --git 
a/shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionHandler.java
 
b/shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionHandler.java
deleted file mode 100644
index b9ef6189b30..00000000000
--- 
a/shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionHandler.java
+++ /dev/null
@@ -1,52 +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.hive.io;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.RecordReader;
-
-/**
- * HiveIOExceptionHandler defines an interface that all io exception handler in
- * Hive should implement. Different IO exception handlers can implement
- * different logics based on the exception input into it.
- */
-public interface HiveIOExceptionHandler {
-
-  /**
-   * process exceptions raised when creating a record reader.
-   *
-   * @param e
-   * @return RecordReader
-   */
-  public RecordReader<?, ?> handleRecordReaderCreationException(Exception e)
-      throws IOException;
-
-  /**
-   * process exceptions thrown when calling rr's next
-   *
-   * @param e
-   * @param result
-   * @throws IOException
-   */
-  public void handleRecorReaderNextException(Exception e,
-      HiveIOExceptionNextHandleResult result) throws IOException;
-
-}
diff --git 
a/shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionHandlerChain.java
 
b/shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionHandlerChain.java
deleted file mode 100644
index 9dc0a1a0347..00000000000
--- 
a/shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionHandlerChain.java
+++ /dev/null
@@ -1,124 +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.hive.io;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.mapred.InputFormat;
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.apache.hadoop.util.ReflectionUtils;
-
-/**
- * An exception handler chain that process the input exception by going through
- * all exceptions defined in this chain one by one until either one exception
- * handler returns true or it reaches the end of the chain. If it reaches the
- * end of the chain, and still no exception handler returns true, throw the
- * exception to the caller.
- */
-public class HiveIOExceptionHandlerChain {
-
-  public static final String HIVE_IO_EXCEPTION_HANDLE_CHAIN = 
"hive.io.exception.handlers";
-
-  @SuppressWarnings("unchecked")
-  public static HiveIOExceptionHandlerChain getHiveIOExceptionHandlerChain(
-      JobConf conf) {
-    HiveIOExceptionHandlerChain chain = new HiveIOExceptionHandlerChain();
-    String exceptionHandlerStr = conf.get(HIVE_IO_EXCEPTION_HANDLE_CHAIN);
-    List<HiveIOExceptionHandler> handlerChain = new 
ArrayList<HiveIOExceptionHandler>();
-    if (exceptionHandlerStr != null && !exceptionHandlerStr.trim().equals("")) 
{
-      String[] handlerArr = exceptionHandlerStr.split(",");
-      if (handlerArr != null && handlerArr.length > 0) {
-        for (String handlerStr : handlerArr) {
-          if (!handlerStr.trim().equals("")) {
-            try {
-              Class<? extends HiveIOExceptionHandler> handlerCls =
-                (Class<? extends HiveIOExceptionHandler>) 
Class.forName(handlerStr);
-              HiveIOExceptionHandler handler = 
ReflectionUtils.newInstance(handlerCls, null);
-              handlerChain.add(handler);
-            } catch (Exception e) {
-            }
-          }
-        }
-      }
-    }
-
-    chain.setHandlerChain(handlerChain);
-    return chain;
-  }
-
-  private List<HiveIOExceptionHandler> handlerChain;
-
-  /**
-   * @return the exception handler chain defined
-   */
-  protected List<HiveIOExceptionHandler> getHandlerChain() {
-    return handlerChain;
-  }
-
-  /**
-   * set the exception handler chain
-   * @param handlerChain
-   */
-  protected void setHandlerChain(List<HiveIOExceptionHandler> handlerChain) {
-    this.handlerChain = handlerChain;
-  }
-
-  public RecordReader<?,?>  handleRecordReaderCreationException(Exception e) 
throws IOException {
-    RecordReader<?, ?> ret = null;
-
-    if (handlerChain != null && handlerChain.size() > 0) {
-      for (HiveIOExceptionHandler handler : handlerChain) {
-        ret = handler.handleRecordReaderCreationException(e);
-        if (ret != null) {
-          return ret;
-        }
-      }
-    }
-
-    //re-throw the exception as an IOException
-    throw new IOException(e);
-  }
-
-  /**
-   * This is to handle exception when doing next operations. Here we use a
-   * HiveIOExceptionNextHandleResult to store the results of each handler. If
-   * the exception is handled by one handler, the handler should set
-   * HiveIOExceptionNextHandleResult to be handled, and also set the handle
-   * result. The handle result is used to return the reader's next to determine
-   * if need to open a new file for read or not.
-   */
-  public boolean handleRecordReaderNextException(Exception e)
-      throws IOException {
-    HiveIOExceptionNextHandleResult result = new 
HiveIOExceptionNextHandleResult();
-    if (handlerChain != null && handlerChain.size() > 0) {
-      for (HiveIOExceptionHandler handler : handlerChain) {
-        handler.handleRecorReaderNextException(e, result);
-        if (result.getHandled()) {
-          return result.getHandleResult();
-        }
-      }
-    }
-
-    //re-throw the exception as an IOException
-    throw new IOException(e);
-  }
-
-}
diff --git 
a/shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionHandlerUtil.java
 
b/shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionHandlerUtil.java
deleted file mode 100644
index c9338ef3f21..00000000000
--- 
a/shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionHandlerUtil.java
+++ /dev/null
@@ -1,88 +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.hive.io;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.RecordReader;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class HiveIOExceptionHandlerUtil {
-
-  private static final Logger LOG = 
LoggerFactory.getLogger(HiveIOExceptionHandlerUtil.class.getName());
-
-  private static final ThreadLocal<HiveIOExceptionHandlerChain> 
handlerChainInstance =
-    new ThreadLocal<HiveIOExceptionHandlerChain>();
-
-  private static synchronized HiveIOExceptionHandlerChain get(JobConf job) {
-    HiveIOExceptionHandlerChain cache = 
HiveIOExceptionHandlerUtil.handlerChainInstance
-        .get();
-    if (cache == null) {
-      HiveIOExceptionHandlerChain toSet = HiveIOExceptionHandlerChain
-          .getHiveIOExceptionHandlerChain(job);
-      handlerChainInstance.set(toSet);
-      cache = HiveIOExceptionHandlerUtil.handlerChainInstance.get();
-    }
-    return cache;
-  }
-
-  /**
-   * Handle exception thrown when creating record reader. In case that there is
-   * an exception raised when construction the record reader and one handler 
can
-   * handle this exception, it should return an record reader, which is either 
a
-   * dummy empty record reader or a specific record reader that do some magic.
-   *
-   * @param e
-   * @param job
-   * @return RecordReader
-   * @throws IOException
-   */
-  public static RecordReader handleRecordReaderCreationException(Exception e,
-      JobConf job) throws IOException {
-    LOG.error("RecordReader#init() threw an exception: ", e);
-    HiveIOExceptionHandlerChain ioExpectionHandlerChain = get(job);
-    if (ioExpectionHandlerChain != null) {
-      return ioExpectionHandlerChain.handleRecordReaderCreationException(e);
-    }
-    throw new IOException(e);
-  }
-
-  /**
-   * Handle exception thrown when calling record reader's next. If this
-   * exception is handled by one handler, will just return true. Otherwise,
-   * either re-throw this exception in one handler or at the end of the handler
-   * chain.
-   *
-   * @param e
-   * @param job
-   * @return true on success
-   * @throws IOException
-   */
-  public static boolean handleRecordReaderNextException(Exception e, JobConf 
job)
-      throws IOException {
-    LOG.error("RecordReader#next() threw an exception: ", e);
-    HiveIOExceptionHandlerChain ioExpectionHandlerChain = get(job);
-    if (ioExpectionHandlerChain != null) {
-      return ioExpectionHandlerChain.handleRecordReaderNextException(e);
-    }
-    throw new IOException(e);
-  }
-
-}
diff --git 
a/shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionNextHandleResult.java
 
b/shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionNextHandleResult.java
deleted file mode 100644
index b8d50cbcc50..00000000000
--- 
a/shims/common/src/main/java/org/apache/hadoop/hive/io/HiveIOExceptionNextHandleResult.java
+++ /dev/null
@@ -1,55 +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.hive.io;
-
-/**
- * A container to store handling results for exceptions produced in record
- * reader's next(). It basically contains 2 fields, one is to store if it is
- * handled or not, another field to store the result.
- */
-public class HiveIOExceptionNextHandleResult {
-
-  // this exception has been handled
-  private boolean handled;
-
-  //the handling results
-  private boolean handleResult;
-
-  public boolean getHandled() {
-    return handled;
-  }
-
-  public void setHandled(boolean handled) {
-    this.handled = handled;
-  }
-
-  public boolean getHandleResult() {
-    return handleResult;
-  }
-
-  public void setHandleResult(boolean handleResult) {
-    this.handleResult = handleResult;
-  }
-
-  public void clear() {
-    handled = false;
-    handleResult = false;
-  }
-
-}
diff --git 
a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
 
b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
index 2a92304fc11..0977cc88c22 100644
--- 
a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
+++ 
b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
@@ -19,9 +19,7 @@
 
 import java.io.DataInput;
 import java.io.DataOutput;
-import java.io.EOFException;
 import java.io.IOException;
-import java.io.InputStream;
 import java.lang.reflect.Constructor;
 import java.net.URI;
 import java.security.AccessControlException;
@@ -30,19 +28,15 @@
 import java.util.HashSet;
 import java.util.Set;
 
-import org.apache.commons.lang3.ArrayUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.DefaultFileAccess;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FsShell;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.hive.io.HiveIOExceptionHandlerUtil;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.ClusterStatus;
 import org.apache.hadoop.mapred.FileInputFormat;
 import org.apache.hadoop.mapred.InputSplit;
@@ -137,7 +131,7 @@ public static class CombineFileRecordReader<K, V> 
implements RecordReader<K, V>
     public boolean next(K key, V value) throws IOException {
 
       while ((curReader == null)
-          || !doNextWithExceptionHandler((K) ((CombineHiveKey) key).getKey(),
+          || !doNext((K) ((CombineHiveKey) key).getKey(),
               value)) {
         if (!initNextRecordReader(key)) {
           return false;
@@ -221,13 +215,8 @@ private CombineFileRecordReader(JobConf job, 
CombineFileSplit split, Reporter re
      * @return
      * @throws IOException
      */
-    private boolean doNextWithExceptionHandler(K key, V value) throws 
IOException {
-      try {
-        return curReader.next(key, value);
-      } catch (Exception e) {
-        return HiveIOExceptionHandlerUtil
-            .handleRecordReaderNextException(e, jc);
-      }
+    private boolean doNext(K key, V value) throws IOException {
+      return curReader.next(key, value);
     }
 
     /**
@@ -265,8 +254,7 @@ protected boolean initNextRecordReader(K key) throws 
IOException {
         jc.setLong("map.input.start", split.getOffset(idx));
         jc.setLong("map.input.length", split.getLength(idx));
       } catch (Exception e) {
-        curReader = 
HiveIOExceptionHandlerUtil.handleRecordReaderCreationException(
-            e, jc);
+        throw new IOException(e);
       }
       idx++;
       return true;


Reply via email to