You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@fluo.apache.org by kt...@apache.org on 2016/09/13 20:38:06 UTC

incubator-fluo git commit: fixes #764 fixed LoaderExecutor exception handling

Repository: incubator-fluo
Updated Branches:
  refs/heads/master 2b1b2a2b0 -> cb78079bf


fixes #764 fixed LoaderExecutor exception handling


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

Branch: refs/heads/master
Commit: cb78079bf805d30c0292b74278561556da177f82
Parents: 2b1b2a2
Author: Keith Turner <kt...@apache.org>
Authored: Fri Sep 9 12:17:02 2016 -0400
Committer: Keith Turner <kt...@apache.org>
Committed: Fri Sep 9 12:17:02 2016 -0400

----------------------------------------------------------------------
 .../core/client/LoaderExecutorAsyncImpl.java    | 18 +++++--
 .../integration/client/LoaderExecutorIT.java    | 52 ++++++++++++++++++++
 2 files changed, 65 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/cb78079b/modules/core/src/main/java/org/apache/fluo/core/client/LoaderExecutorAsyncImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/fluo/core/client/LoaderExecutorAsyncImpl.java b/modules/core/src/main/java/org/apache/fluo/core/client/LoaderExecutorAsyncImpl.java
index d6f5187..b05467a 100644
--- a/modules/core/src/main/java/org/apache/fluo/core/client/LoaderExecutorAsyncImpl.java
+++ b/modules/core/src/main/java/org/apache/fluo/core/client/LoaderExecutorAsyncImpl.java
@@ -48,17 +48,23 @@ public class LoaderExecutorAsyncImpl implements LoaderExecutor {
 
   private final Counter commiting = new Counter();
 
+  private void setException(Throwable t) {
+    if (!exceptionRef.compareAndSet(null, t)) {
+      LoggerFactory.getLogger(LoaderExecutorAsyncImpl.class).debug(
+          "Multiple exceptions occured, not reporting subsequent ones", t);
+    }
+  }
+
   class LoaderCommitObserver implements AsyncCommitObserver, Runnable {
 
     AsyncTransaction txi;
     Loader loader;
-    private boolean done = false;
+    private AtomicBoolean done = new AtomicBoolean(false);
 
     private void close() {
       txi = null;
-      if (!done) {
+      if (done.compareAndSet(false, true)) {
         commiting.decrement();
-        done = true;
       } else {
         // its only expected that this should be called once.. if its called multiple times in
         // indicates an error in asyn code
@@ -81,7 +87,7 @@ public class LoaderExecutorAsyncImpl implements LoaderExecutor {
     @Override
     public void failed(Throwable t) {
       close();
-      exceptionRef.compareAndSet(null, t);
+      setException(t);
     }
 
     @Override
@@ -113,12 +119,14 @@ public class LoaderExecutorAsyncImpl implements LoaderExecutor {
           return env.getAppConfiguration();
         }
       };
+
       try {
         loader.load(txi, context);
         env.getSharedResources().getCommitManager().beginCommit(txi, loader.getClass(), this);
       } catch (Exception e) {
+        setException(e);
+        close();
         LoggerFactory.getLogger(LoaderCommitObserver.class).debug(e.getMessage(), e);
-        exceptionRef.compareAndSet(null, e);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-fluo/blob/cb78079b/modules/integration/src/test/java/org/apache/fluo/integration/client/LoaderExecutorIT.java
----------------------------------------------------------------------
diff --git a/modules/integration/src/test/java/org/apache/fluo/integration/client/LoaderExecutorIT.java b/modules/integration/src/test/java/org/apache/fluo/integration/client/LoaderExecutorIT.java
new file mode 100644
index 0000000..6fdb5c1
--- /dev/null
+++ b/modules/integration/src/test/java/org/apache/fluo/integration/client/LoaderExecutorIT.java
@@ -0,0 +1,52 @@
+/*
+ * 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.fluo.integration.client;
+
+import org.apache.fluo.api.client.Loader;
+import org.apache.fluo.api.client.LoaderExecutor;
+import org.apache.fluo.api.client.TransactionBase;
+import org.apache.fluo.api.data.Column;
+import org.apache.fluo.api.exceptions.AlreadySetException;
+import org.apache.fluo.integration.ITBaseMini;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class LoaderExecutorIT extends ITBaseMini {
+
+  public static class BadLoader implements Loader {
+
+    @Override
+    public void load(TransactionBase tx, Context context) throws Exception {
+      tx.set("r", new Column("f", "q"), "v");
+      // setting same thing should cause exception
+      tx.set("r", new Column("f", "q"), "v2");
+    }
+
+  }
+
+  @Test
+  public void testLoaderFailure() {
+    LoaderExecutor le = client.newLoaderExecutor();
+    le.execute(new BadLoader());
+
+    try {
+      le.close();
+      Assert.fail();
+    } catch (RuntimeException e) {
+      Assert.assertEquals(AlreadySetException.class, e.getCause().getClass());
+    }
+  }
+}