You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@streams.apache.org by sb...@apache.org on 2014/09/04 19:12:17 UTC

[29/41] git commit: STREAMS-158 | Failing datum and writing error to metadata rather than throwing exception

STREAMS-158 | Failing datum and writing error to metadata rather than throwing exception


Project: http://git-wip-us.apache.org/repos/asf/incubator-streams/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-streams/commit/16563aa4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-streams/tree/16563aa4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-streams/diff/16563aa4

Branch: refs/heads/pp
Commit: 16563aa41162c5e251daf2a6c726a0221e2cfdc8
Parents: ee6b4d5
Author: mfranklin <mf...@apache.org>
Authored: Mon Aug 25 10:49:53 2014 -0400
Committer: mfranklin <mf...@apache.org>
Committed: Mon Aug 25 10:49:53 2014 -0400

----------------------------------------------------------------------
 .../apache/streams/core/util/DatumUtils.java    | 49 ++++++++++++++++++++
 .../local/tasks/StreamsPersistWriterTask.java   |  2 +
 .../local/tasks/StreamsProcessorTask.java       |  6 ++-
 .../local/tasks/StreamsProviderTask.java        |  2 +
 4 files changed, 57 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/16563aa4/streams-core/src/main/java/org/apache/streams/core/util/DatumUtils.java
----------------------------------------------------------------------
diff --git a/streams-core/src/main/java/org/apache/streams/core/util/DatumUtils.java b/streams-core/src/main/java/org/apache/streams/core/util/DatumUtils.java
new file mode 100644
index 0000000..eedbb07
--- /dev/null
+++ b/streams-core/src/main/java/org/apache/streams/core/util/DatumUtils.java
@@ -0,0 +1,49 @@
+/*
+ * 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.streams.core.util;
+
+import com.google.common.collect.Maps;
+import org.apache.streams.core.StreamsDatum;
+import org.apache.streams.core.StreamsOperation;
+
+import java.util.Map;
+
+/**
+ * Provides common utilities for managing and manipulating StreamsDatums
+ */
+public class DatumUtils {
+
+    /**
+     * Adds an error occurred during a StreamsOperation step to the StreamsDatum's metadata.  By convention, errors are
+     * placed in the metadata under the "errors" and are organized by class simple name where the failure occurred.
+     *
+     * @param datum the datum on which the operation step errored
+     * @param e the error encountered
+     * @param operationClass the class of the operation
+     */
+    @SuppressWarnings("all")
+    public static void addErrorToMetadata(StreamsDatum datum, Throwable e, Class<? extends StreamsOperation> operationClass) {
+        if(!datum.getMetadata().containsKey("errors")) {
+            datum.getMetadata().put("errors", Maps.<String, Throwable>newHashMap());
+        }
+        Map<String, Throwable> errors = (Map)datum.getMetadata().get("errors");
+        errors.put(operationClass.getCanonicalName(), e);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/16563aa4/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsPersistWriterTask.java
----------------------------------------------------------------------
diff --git a/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsPersistWriterTask.java b/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsPersistWriterTask.java
index 4ec5fe9..59c438e 100644
--- a/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsPersistWriterTask.java
+++ b/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsPersistWriterTask.java
@@ -19,6 +19,7 @@
 package org.apache.streams.local.tasks;
 
 import org.apache.streams.core.*;
+import org.apache.streams.core.util.DatumUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -99,6 +100,7 @@ public class StreamsPersistWriterTask extends BaseStreamsTask implements DatumSt
                         LOGGER.error("Error writing to persist writer {}", this.writer.getClass().getSimpleName(), e);
                         this.keepRunning.set(false);
                         statusCounter.incrementStatus(DatumStatus.FAIL);
+                        DatumUtils.addErrorToMetadata(datum, e, this.writer.getClass());
                     }
                 }
                 else {

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/16563aa4/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsProcessorTask.java
----------------------------------------------------------------------
diff --git a/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsProcessorTask.java b/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsProcessorTask.java
index 30139fe..beec0c2 100644
--- a/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsProcessorTask.java
+++ b/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsProcessorTask.java
@@ -18,7 +18,9 @@
 
 package org.apache.streams.local.tasks;
 
+import com.google.common.collect.Maps;
 import org.apache.streams.core.*;
+import org.apache.streams.core.util.DatumUtils;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -108,9 +110,9 @@ public class StreamsProcessorTask extends BaseStreamsTask implements DatumStatus
                         }
                     } catch (Throwable e) {
                         LOGGER.error("Throwable Streams Processor {}", e);
-                        e.printStackTrace();
                         statusCounter.incrementStatus(DatumStatus.FAIL);
-                        throw new RuntimeException(e);
+                        //Add the error to the metadata, but keep processing
+                        DatumUtils.addErrorToMetadata(datum, e, this.processor.getClass());
                     }
                 }
                 else {

http://git-wip-us.apache.org/repos/asf/incubator-streams/blob/16563aa4/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsProviderTask.java
----------------------------------------------------------------------
diff --git a/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsProviderTask.java b/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsProviderTask.java
index fe7ea95..8809d5a 100644
--- a/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsProviderTask.java
+++ b/streams-runtimes/streams-runtime-local/src/main/java/org/apache/streams/local/tasks/StreamsProviderTask.java
@@ -19,6 +19,7 @@
 package org.apache.streams.local.tasks;
 
 import org.apache.streams.core.*;
+import org.apache.streams.core.util.DatumUtils;
 import org.joda.time.DateTime;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -203,6 +204,7 @@ public class StreamsProviderTask extends BaseStreamsTask implements DatumStatusC
                     statusCounter.incrementStatus(DatumStatus.SUCCESS);
                 } catch( Exception e ) {
                     statusCounter.incrementStatus(DatumStatus.FAIL);
+                    DatumUtils.addErrorToMetadata(datum, e, this.provider.getClass());
                 }
             }
             else {