You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@sqoop.apache.org by bl...@apache.org on 2012/11/12 21:05:10 UTC

git commit: SQOOP-692 Move specific error codes to own error codes class (Jarek Jarcec Cecho)

Updated Branches:
  refs/heads/sqoop2 fa815557c -> 5f95f114f


SQOOP-692 Move specific error codes to own error codes class
(Jarek Jarcec Cecho)


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

Branch: refs/heads/sqoop2
Commit: 5f95f114f69bb0fea86c9d70dde3ae0c7c07a204
Parents: fa81555
Author: Bilung Lee <bl...@apache.org>
Authored: Mon Nov 12 11:43:20 2012 -0800
Committer: Bilung Lee <bl...@apache.org>
Committed: Mon Nov 12 11:43:20 2012 -0800

----------------------------------------------------------------------
 .../main/java/org/apache/sqoop/core/CoreError.java |   47 --------
 .../apache/sqoop/job/MapreduceExecutionError.java  |   91 +++++++++++++++
 .../sqoop/job/etl/HdfsSequenceImportLoader.java    |    8 +-
 .../apache/sqoop/job/etl/HdfsTextImportLoader.java |    8 +-
 .../main/java/org/apache/sqoop/job/io/Data.java    |   22 ++--
 .../java/org/apache/sqoop/job/mr/SqoopMapper.java  |    8 +-
 .../job/mr/SqoopOutputFormatLoadExecutor.java      |   12 +-
 .../java/org/apache/sqoop/job/mr/SqoopSplit.java   |    6 +-
 8 files changed, 123 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/sqoop/blob/5f95f114/core/src/main/java/org/apache/sqoop/core/CoreError.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/sqoop/core/CoreError.java b/core/src/main/java/org/apache/sqoop/core/CoreError.java
index 08034ed..f59d132 100644
--- a/core/src/main/java/org/apache/sqoop/core/CoreError.java
+++ b/core/src/main/java/org/apache/sqoop/core/CoreError.java
@@ -51,53 +51,6 @@ public enum CoreError implements ErrorCode {
   /** The configuration system has not been initialized correctly. */
   CORE_0007("System not initialized"),
 
-  /** Error occurs during job execution. */
-  CORE_0008("Error occurs during job execution"),
-
-  /** The system was unable to load the specified class. */
-  CORE_0009("Unable to load the specified class"),
-
-  /** The system was unable to instantiate the specified class. */
-  CORE_0010("Unable to instantiate the specified class"),
-
-  /** The parameter already exists in the context */
-  CORE_0011("The parameter already exists in the context"),
-
-  /** The type is not supported */
-  CORE_0012("The type is not supported"),
-
-  /** Cannot write to the data writer */
-  CORE_0013("Cannot write to the data writer"),
-
-  /** Cannot read from the data reader */
-  CORE_0014("Cannot read to the data reader"),
-
-  /** Unable to write data due to interrupt */
-  CORE_0015("Unable to write data due to interrupt"),
-
-  /** Unable to read data due to interrupt */
-  CORE_0016("Unable to read data due to interrupt"),
-
-  /** Error occurs during extractor run */
-  CORE_0017("Error occurs during extractor run"),
-
-  /** Error occurs during loader run */
-  CORE_0018("Error occurs during loader run"),
-
-  CORE_0019("Data have not been completely consumed yet"),
-
-  /** The required option has not been set yet */
-  CORE_0020("The required option has not been set yet"),
-
-  /** Error occurs during partitioner run */
-  CORE_0021("Error occurs during partitioner run"),
-
-  /** Unable to parse because it is not properly delimited */
-  CORE_0022("Unable to parse because it is not properly delimited"),
-
-  /** Unknown job type */
-  CORE_0023("Unknown job type"),
-
   ;
 
   private final String message;

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5f95f114/execution/mapreduce/src/main/java/org/apache/sqoop/job/MapreduceExecutionError.java
----------------------------------------------------------------------
diff --git a/execution/mapreduce/src/main/java/org/apache/sqoop/job/MapreduceExecutionError.java b/execution/mapreduce/src/main/java/org/apache/sqoop/job/MapreduceExecutionError.java
new file mode 100644
index 0000000..aa1f733
--- /dev/null
+++ b/execution/mapreduce/src/main/java/org/apache/sqoop/job/MapreduceExecutionError.java
@@ -0,0 +1,91 @@
+/**
+ * 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.sqoop.job;
+
+import org.apache.sqoop.common.ErrorCode;
+
+/**
+ *
+ */
+public enum MapreduceExecutionError implements ErrorCode {
+
+  MAPRED_EXEC_0000("Unknown error"),
+
+  /** Error occurs during job execution. */
+  MAPRED_EXEC_0008("Error occurs during job execution"),
+
+  /** The system was unable to load the specified class. */
+  MAPRED_EXEC_0009("Unable to load the specified class"),
+
+  /** The system was unable to instantiate the specified class. */
+  MAPRED_EXEC_0010("Unable to instantiate the specified class"),
+
+  /** The parameter already exists in the context */
+  MAPRED_EXEC_0011("The parameter already exists in the context"),
+
+  /** The type is not supported */
+  MAPRED_EXEC_0012("The type is not supported"),
+
+  /** Cannot write to the data writer */
+  MAPRED_EXEC_0013("Cannot write to the data writer"),
+
+  /** Cannot read from the data reader */
+  MAPRED_EXEC_0014("Cannot read to the data reader"),
+
+  /** Unable to write data due to interrupt */
+  MAPRED_EXEC_0015("Unable to write data due to interrupt"),
+
+  /** Unable to read data due to interrupt */
+  MAPRED_EXEC_0016("Unable to read data due to interrupt"),
+
+  /** Error occurs during extractor run */
+  MAPRED_EXEC_0017("Error occurs during extractor run"),
+
+  /** Error occurs during loader run */
+  MAPRED_EXEC_0018("Error occurs during loader run"),
+
+  MAPRED_EXEC_0019("Data have not been completely consumed yet"),
+
+  /** The required option has not been set yet */
+  MAPRED_EXEC_0020("The required option has not been set yet"),
+
+  /** Error occurs during partitioner run */
+  MAPRED_EXEC_0021("Error occurs during partitioner run"),
+
+  /** Unable to parse because it is not properly delimited */
+  MAPRED_EXEC_0022("Unable to parse because it is not properly delimited"),
+
+  /** Unknown job type */
+  MAPRED_EXEC_0023("Unknown job type"),
+
+  ;
+
+  private final String message;
+
+  private MapreduceExecutionError(String message) {
+    this.message = message;
+  }
+
+  public String getCode() {
+    return name();
+  }
+
+  public String getMessage() {
+    return message;
+  }
+}

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5f95f114/execution/mapreduce/src/main/java/org/apache/sqoop/job/etl/HdfsSequenceImportLoader.java
----------------------------------------------------------------------
diff --git a/execution/mapreduce/src/main/java/org/apache/sqoop/job/etl/HdfsSequenceImportLoader.java b/execution/mapreduce/src/main/java/org/apache/sqoop/job/etl/HdfsSequenceImportLoader.java
index 1235d1d..29a73b0 100644
--- a/execution/mapreduce/src/main/java/org/apache/sqoop/job/etl/HdfsSequenceImportLoader.java
+++ b/execution/mapreduce/src/main/java/org/apache/sqoop/job/etl/HdfsSequenceImportLoader.java
@@ -29,8 +29,8 @@ import org.apache.hadoop.io.Text;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.sqoop.common.ImmutableContext;
 import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.core.CoreError;
 import org.apache.sqoop.job.JobConstants;
+import org.apache.sqoop.job.MapreduceExecutionError;
 import org.apache.sqoop.job.io.Data;
 import org.apache.sqoop.job.io.DataReader;
 import org.apache.sqoop.utils.ClassUtils;
@@ -59,7 +59,7 @@ public class HdfsSequenceImportLoader extends Loader {
     if (codecname != null) {
       Class<?> clz = ClassUtils.loadClass(codecname);
       if (clz == null) {
-        throw new SqoopException(CoreError.CORE_0009, codecname);
+        throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0009, codecname);
       }
 
       try {
@@ -68,7 +68,7 @@ public class HdfsSequenceImportLoader extends Loader {
           ((Configurable) codec).setConf(conf);
         }
       } catch (Exception e) {
-        throw new SqoopException(CoreError.CORE_0010, codecname, e);
+        throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0010, codecname, e);
       }
     }
 
@@ -100,7 +100,7 @@ public class HdfsSequenceImportLoader extends Loader {
       filewriter.close();
 
     } catch (IOException e) {
-      throw new SqoopException(CoreError.CORE_0018, e);
+      throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0018, e);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5f95f114/execution/mapreduce/src/main/java/org/apache/sqoop/job/etl/HdfsTextImportLoader.java
----------------------------------------------------------------------
diff --git a/execution/mapreduce/src/main/java/org/apache/sqoop/job/etl/HdfsTextImportLoader.java b/execution/mapreduce/src/main/java/org/apache/sqoop/job/etl/HdfsTextImportLoader.java
index 36aa11f..711df0f 100644
--- a/execution/mapreduce/src/main/java/org/apache/sqoop/job/etl/HdfsTextImportLoader.java
+++ b/execution/mapreduce/src/main/java/org/apache/sqoop/job/etl/HdfsTextImportLoader.java
@@ -29,8 +29,8 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.compress.CompressionCodec;
 import org.apache.sqoop.common.ImmutableContext;
 import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.core.CoreError;
 import org.apache.sqoop.job.JobConstants;
+import org.apache.sqoop.job.MapreduceExecutionError;
 import org.apache.sqoop.job.io.Data;
 import org.apache.sqoop.job.io.DataReader;
 import org.apache.sqoop.utils.ClassUtils;
@@ -58,7 +58,7 @@ public class HdfsTextImportLoader extends Loader {
     if (codecname != null) {
       Class<?> clz = ClassUtils.loadClass(codecname);
       if (clz == null) {
-        throw new SqoopException(CoreError.CORE_0009, codecname);
+        throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0009, codecname);
       }
 
       try {
@@ -67,7 +67,7 @@ public class HdfsTextImportLoader extends Loader {
           ((Configurable) codec).setConf(conf);
         }
       } catch (Exception e) {
-        throw new SqoopException(CoreError.CORE_0010, codecname, e);
+        throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0010, codecname, e);
       }
 
       filename += codec.getDefaultExtension();
@@ -95,7 +95,7 @@ public class HdfsTextImportLoader extends Loader {
       filewriter.close();
 
     } catch (IOException e) {
-      throw new SqoopException(CoreError.CORE_0018, e);
+      throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0018, e);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5f95f114/execution/mapreduce/src/main/java/org/apache/sqoop/job/io/Data.java
----------------------------------------------------------------------
diff --git a/execution/mapreduce/src/main/java/org/apache/sqoop/job/io/Data.java b/execution/mapreduce/src/main/java/org/apache/sqoop/job/io/Data.java
index 4ddd132..f6fff0b 100644
--- a/execution/mapreduce/src/main/java/org/apache/sqoop/job/io/Data.java
+++ b/execution/mapreduce/src/main/java/org/apache/sqoop/job/io/Data.java
@@ -29,7 +29,7 @@ import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.WritableComparator;
 import org.apache.hadoop.io.WritableUtils;
 import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.core.CoreError;
+import org.apache.sqoop.job.MapreduceExecutionError;
 
 public class Data implements WritableComparable<Data> {
 
@@ -70,7 +70,7 @@ public class Data implements WritableComparable<Data> {
       this.content = content;
       break;
     default:
-      throw new SqoopException(CoreError.CORE_0012, String.valueOf(type));
+      throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0012, String.valueOf(type));
     }
   }
 
@@ -81,7 +81,7 @@ public class Data implements WritableComparable<Data> {
     case ARRAY_RECORD:
       return parse();
     default:
-      throw new SqoopException(CoreError.CORE_0012, String.valueOf(targetType));
+      throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0012, String.valueOf(targetType));
     }
   }
 
@@ -135,7 +135,7 @@ public class Data implements WritableComparable<Data> {
       }
       return result;
     default:
-      throw new SqoopException(CoreError.CORE_0012, String.valueOf(type));
+      throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0012, String.valueOf(type));
     }
   }
 
@@ -150,7 +150,7 @@ public class Data implements WritableComparable<Data> {
       readArray(in);
       break;
     default:
-      throw new SqoopException(CoreError.CORE_0012, String.valueOf(type));
+      throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0012, String.valueOf(type));
     }
   }
 
@@ -165,7 +165,7 @@ public class Data implements WritableComparable<Data> {
       writeArray(out);
       break;
     default:
-      throw new SqoopException(CoreError.CORE_0012, String.valueOf(type));
+      throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0012, String.valueOf(type));
     }
   }
 
@@ -243,7 +243,7 @@ public class Data implements WritableComparable<Data> {
 
       default:
         throw new IOException(
-          new SqoopException(CoreError.CORE_0012, Integer.toString(type))
+          new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0012, Integer.toString(type))
         );
       }
     }
@@ -301,8 +301,8 @@ public class Data implements WritableComparable<Data> {
 
       } else {
         throw new IOException(
-          new SqoopException(
-              CoreError.CORE_0012, array[i].getClass().getName()
+          new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0012,
+            array[i].getClass().getName()
           )
         );
       }
@@ -345,7 +345,7 @@ public class Data implements WritableComparable<Data> {
       return sb.toString();
 
     default:
-      throw new SqoopException(CoreError.CORE_0012, String.valueOf(type));
+      throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0012, String.valueOf(type));
     }
   }
 
@@ -363,7 +363,7 @@ public class Data implements WritableComparable<Data> {
       return (Object[])content;
 
     default:
-      throw new SqoopException(CoreError.CORE_0012, String.valueOf(type));
+      throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0012, String.valueOf(type));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5f95f114/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopMapper.java
----------------------------------------------------------------------
diff --git a/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopMapper.java b/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopMapper.java
index dbe832a..1c0f3aa 100644
--- a/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopMapper.java
+++ b/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopMapper.java
@@ -25,8 +25,8 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.core.CoreError;
 import org.apache.sqoop.job.JobConstants;
+import org.apache.sqoop.job.MapreduceExecutionError;
 import org.apache.sqoop.job.PrefixContext;
 import org.apache.sqoop.job.etl.Extractor;
 import org.apache.sqoop.job.io.Data;
@@ -67,7 +67,7 @@ public class SqoopMapper
         configJob = ConfigurationUtils.getFrameworkJob(conf);
         break;
       default:
-        throw new SqoopException(CoreError.CORE_0023);
+        throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0023);
     }
 
     SqoopSplit split = context.getCurrentKey();
@@ -77,7 +77,7 @@ public class SqoopMapper
         new MapDataWriter(context));
 
     } catch (Exception e) {
-      throw new SqoopException(CoreError.CORE_0017, e);
+      throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0017, e);
     }
   }
 
@@ -118,7 +118,7 @@ public class SqoopMapper
       try {
         context.write(data, NullWritable.get());
       } catch (Exception e) {
-        throw new SqoopException(CoreError.CORE_0013, e);
+        throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0013, e);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5f95f114/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopOutputFormatLoadExecutor.java
----------------------------------------------------------------------
diff --git a/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopOutputFormatLoadExecutor.java b/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopOutputFormatLoadExecutor.java
index 96e1533..0d636ae 100644
--- a/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopOutputFormatLoadExecutor.java
+++ b/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopOutputFormatLoadExecutor.java
@@ -26,8 +26,8 @@ import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.RecordWriter;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.core.CoreError;
 import org.apache.sqoop.job.JobConstants;
+import org.apache.sqoop.job.MapreduceExecutionError;
 import org.apache.sqoop.job.PrefixContext;
 import org.apache.sqoop.job.etl.Loader;
 import org.apache.sqoop.job.io.Data;
@@ -88,7 +88,7 @@ public class SqoopOutputFormatLoadExecutor {
           data.notify();
 
           // throw exception
-          throw new SqoopException(CoreError.CORE_0015, e);
+          throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0015, e);
         }
       }
     }
@@ -119,7 +119,7 @@ public class SqoopOutputFormatLoadExecutor {
           data.notify();
 
           // throw exception
-          throw new SqoopException(CoreError.CORE_0015, e);
+          throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0015, e);
         }
       }
     }
@@ -170,7 +170,7 @@ public class SqoopOutputFormatLoadExecutor {
           data.notify();
 
           // throw exception
-          throw new SqoopException(CoreError.CORE_0016, e);
+          throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0016, e);
         }
       }
     }
@@ -201,7 +201,7 @@ public class SqoopOutputFormatLoadExecutor {
       try {
         loader.run(frameworkContext, reader);
       } catch (Throwable t) {
-        throw new SqoopException(CoreError.CORE_0018, t);
+        throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0018, t);
       }
 
       synchronized (data) {
@@ -214,7 +214,7 @@ public class SqoopOutputFormatLoadExecutor {
         // if no exception happens yet
         if (exception == null && !writerFinished) {
           // create exception if data are not all consumed
-          exception = new SqoopException(CoreError.CORE_0019);
+          exception = new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0019);
         }
 
         // throw deferred exception if exist

http://git-wip-us.apache.org/repos/asf/sqoop/blob/5f95f114/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopSplit.java
----------------------------------------------------------------------
diff --git a/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopSplit.java b/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopSplit.java
index 7dc9541..dca4c90 100644
--- a/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopSplit.java
+++ b/execution/mapreduce/src/main/java/org/apache/sqoop/job/mr/SqoopSplit.java
@@ -24,7 +24,7 @@ import java.io.IOException;
 import org.apache.hadoop.io.Writable;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.sqoop.common.SqoopException;
-import org.apache.sqoop.core.CoreError;
+import org.apache.sqoop.job.MapreduceExecutionError;
 import org.apache.sqoop.job.etl.Partition;
 import org.apache.sqoop.utils.ClassUtils;
 
@@ -60,12 +60,12 @@ public class SqoopSplit extends InputSplit implements Writable {
     // instantiate Partition object
     Class<?> clz = ClassUtils.loadClass(className);
     if (clz == null) {
-      throw new SqoopException(CoreError.CORE_0009, className);
+      throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0009, className);
     }
     try {
       partition = (Partition) clz.newInstance();
     } catch (Exception e) {
-      throw new SqoopException(CoreError.CORE_0010, className, e);
+      throw new SqoopException(MapreduceExecutionError.MAPRED_EXEC_0010, className, e);
     }
     // read Partition object content
     partition.readFields(in);