Permalink
Browse files

Merge branch 'S4-44' into dev

  • Loading branch information...
2 parents dd96e87 + a0574b5 commit 1f6ddce9e95fee6973f1c37a53097a112d6fb34b @matthieumorel matthieumorel committed Mar 1, 2012
Showing with 920 additions and 568 deletions.
  1. +49 −28 s4-core/src/main/java/org/apache/s4/ft/CheckpointingCoordinator.java
  2. +43 −0 s4-core/src/main/java/org/apache/s4/ft/FetchTask.java
  3. +0 −37 s4-core/src/main/java/org/apache/s4/ft/InitiateCheckpointingEvent.java
  4. +126 −129 s4-core/src/main/java/org/apache/s4/ft/SafeKeeper.java
  5. +5 −5 s4-core/src/main/java/org/apache/s4/ft/SaveStateTask.java
  6. +8 −1 s4-core/src/main/java/org/apache/s4/ft/SerializeTask.java
  7. +44 −86 s4-core/src/main/java/org/apache/s4/processor/AbstractPE.java
  8. +59 −101 s4-core/src/main/java/org/apache/s4/processor/PEContainer.java
  9. +16 −14 s4-core/src/main/java/org/apache/s4/util/MetricsName.java
  10. +7 −3 s4-core/src/test/java/org/apache/s4/ft/RecoveryTest.java
  11. +121 −122 s4-core/src/test/java/org/apache/s4/ft/StatefulTestPE.java
  12. +5 −5 s4-core/src/test/java/org/apache/s4/ft/TestUtils.java
  13. +53 −0 s4-core/src/test/java/org/apache/s4/ft/rectimeout/BrokenStorage.java
  14. +130 −0 s4-core/src/test/java/org/apache/s4/ft/rectimeout/RecoveryTimeoutTest.java
  15. +26 −0 s4-core/src/test/java/org/apache/s4/ft/rectimeout/app_conf.xml
  16. +196 −0 s4-core/src/test/java/org/apache/s4/ft/rectimeout/s4_core_conf_broken_backend.xml
  17. +6 −0 s4-core/src/test/java/org/apache/s4/ft/rectimeout/wall_clock.xml
  18. +13 −20 s4-core/src/test/java/org/apache/s4/ft/wordcount/FTWordCountTest.java
  19. +8 −9 s4-core/src/test/java/org/apache/s4/wordcount/WordClassifier.java
  20. +5 −8 s4-core/src/test/java/org/apache/s4/wordcount/WordCountTest.java
@@ -17,6 +17,7 @@
*/
package org.apache.s4.ft;
+import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
@@ -27,8 +28,8 @@
/**
* Prevents event processing thread and serialization thread to overlap on the same PE instance, which would cause consistency issues during recovery.
- *
- * How it works:
+ *
+ * How it works (for each prototype):
* - we keep track of the PE being serialized and the PE being processed
* - access to the PE is guarded by the instance of this class
* - if the event processing thread receives an event that is handled by a PE currently being serialized, it waits until serialization is complete
@@ -41,30 +42,35 @@
private static final Logger logger = Logger
.getLogger(CheckpointingCoordinator.class);
- AbstractPE processing = null;
- AbstractPE serializing = null;
+ private static class PrototypeSynchro {
+ Lock lock = new ReentrantLock();
+ Condition processingFinished = lock.newCondition();
+ Condition serializingFinished = lock.newCondition();
+ AbstractPE processing = null;
+ AbstractPE serializing = null;
+ }
+
+ ConcurrentHashMap<String, PrototypeSynchro> synchros = new ConcurrentHashMap<String, CheckpointingCoordinator.PrototypeSynchro>();
long maxSerializationLockDuration;
- Lock lock = new ReentrantLock();
- Condition processingFinished = lock.newCondition();
- Condition serializingFinished = lock.newCondition();
public CheckpointingCoordinator(long maxSerializationLockDuration) {
super();
this.maxSerializationLockDuration = maxSerializationLockDuration;
}
public void acquireForProcessing(AbstractPE pe) {
- lock.lock();
+ PrototypeSynchro sync = getPrototypeSynchro(pe);
+ sync.lock.lock();
try {
- if (serializing == pe) {
+ if (sync.serializing == pe) {
try {
if (logger.isTraceEnabled()) {
logger.trace("processing must wait for serialization to finish for PE "
+ pe.getId() + "/" + pe.getKeyValueString());
}
- serializingFinished.await(maxSerializationLockDuration,
+ sync.serializingFinished.await(maxSerializationLockDuration,
TimeUnit.MILLISECONDS);
acquireForProcessing(pe);
} catch (InterruptedException e) {
@@ -75,61 +81,76 @@ public void acquireForProcessing(AbstractPE pe) {
+ "/"
+ pe.getKeyValueString()
+ "]\nProceeding anyway, but checkpoint may contain inconsistent value");
- serializing = null;
+ sync.serializing = null;
}
}
- processing = pe;
+ sync.processing = pe;
} finally {
- lock.unlock();
+ sync.lock.unlock();
}
}
public void releaseFromProcessing(AbstractPE pe) {
- lock.lock();
+ PrototypeSynchro sync = getPrototypeSynchro(pe);
+ sync.lock.lock();
try {
- if (processing == pe) {
- processing = null;
- processingFinished.signal();
+ if (sync.processing == pe) {
+ sync.processing = null;
+ sync.processingFinished.signal();
} else {
logger.warn("Cannot release from processing thread a PE that is not already in processing state");
}
} finally {
- lock.unlock();
+ sync.lock.unlock();
}
}
public void acquireForSerialization(AbstractPE pe) {
- lock.lock();
+ PrototypeSynchro sync = getPrototypeSynchro(pe);
+ sync.lock.lock();
try {
- if (processing == pe) {
+ if (sync.processing == pe) {
try {
if (logger.isTraceEnabled()) {
logger.trace("serialization must wait for processing to finish for PE "
+ pe.getId() + "/" + pe.getKeyValueString());
}
- processingFinished.await(maxSerializationLockDuration, TimeUnit.MILLISECONDS);
+ sync.processingFinished.await(maxSerializationLockDuration, TimeUnit.MILLISECONDS);
acquireForSerialization(pe);
} catch (InterruptedException e) {
// we still need to make sure it is now safe to serialize
acquireForSerialization(pe);
}
}
- serializing = pe;
+ sync.serializing = pe;
} finally {
- lock.unlock();
+ sync.lock.unlock();
+ }
+ }
+
+ private PrototypeSynchro getPrototypeSynchro(AbstractPE pe) {
+ PrototypeSynchro sync = synchros.get(pe.getId());
+ if (sync==null) {
+ sync = new PrototypeSynchro();
+ PrototypeSynchro existing = synchros.putIfAbsent(pe.getId(), sync);
+ if (existing !=null) {
+ sync = existing;
+ }
}
+ return sync;
}
public void releaseFromSerialization(AbstractPE pe)
throws InterruptedException {
- lock.lock();
+ PrototypeSynchro sync = synchros.get(pe.getId());
+ sync.lock.lock();
try {
- if (serializing == pe) {
- serializing = null;
- serializingFinished.signal();
+ if (sync.serializing == pe) {
+ sync.serializing = null;
+ sync.serializingFinished.signal();
}
} finally {
- lock.unlock();
+ sync.lock.unlock();
}
}
}
@@ -0,0 +1,43 @@
+package org.apache.s4.ft;
+
+import static org.apache.s4.util.MetricsName.S4_CORE_METRICS;
+
+import java.util.concurrent.Callable;
+
+import org.apache.s4.util.MetricsName;
+
+/**
+ * Encapsulates a fetching operation.
+ *
+ */
+public class FetchTask implements Callable<byte[]>{
+
+ StateStorage stateStorage;
+ SafeKeeper safeKeeper;
+ SafeKeeperId safeKeeperId;
+
+ public FetchTask(StateStorage stateStorage, SafeKeeper safeKeeper,
+ SafeKeeperId safeKeeperId) {
+ super();
+ this.stateStorage = stateStorage;
+ this.safeKeeper = safeKeeper;
+ this.safeKeeperId = safeKeeperId;
+ }
+
+ @Override
+ public byte[] call() throws Exception {
+ try {
+ byte[] result = stateStorage.fetchState(safeKeeperId);
+ if (safeKeeper.monitor!=null) {
+ safeKeeper.monitor.increment(MetricsName.checkpointing_fetching_success.toString(), 1, S4_CORE_METRICS.toString());
+ }
+ return result;
+ } catch (Exception e) {
+ if (safeKeeper.monitor!=null) {
+ safeKeeper.monitor.increment(MetricsName.checkpointing_fetching_failed.toString(), 1, S4_CORE_METRICS.toString());
+ }
+ throw e;
+ }
+ }
+
+}
@@ -1,37 +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.s4.ft;
-
-/**
- *
- * Event that triggers a checkpoint.
- *
- */
-public class InitiateCheckpointingEvent extends CheckpointingEvent {
-
- public InitiateCheckpointingEvent() {
- // as required by default kryo serializer
- }
-
- public InitiateCheckpointingEvent(SafeKeeperId safeKeeperId) {
- super(safeKeeperId);
- }
-
-
-
-}
Oops, something went wrong.

0 comments on commit 1f6ddce

Please sign in to comment.