|
@@ -18,6 +18,9 @@
|
|
|
|
|
|
package org.apache.hadoop.ipc;
|
|
|
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE;
|
|
|
+import static org.apache.hadoop.fs.CommonConfigurationKeys.IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE_DEFAULT;
|
|
|
+
|
|
|
import java.io.IOException;
|
|
|
import java.lang.reflect.Constructor;
|
|
|
import java.lang.reflect.InvocationTargetException;
|
|
@@ -63,7 +66,7 @@ public class CallQueueManager<E extends Schedulable>
|
|
|
}
|
|
|
|
|
|
private volatile boolean clientBackOffEnabled;
|
|
|
- private boolean serverFailOverEnabled;
|
|
|
+ private volatile boolean serverFailOverEnabled;
|
|
|
|
|
|
// Atomic refs point to active callQueue
|
|
|
// We have two so we can better control swapping
|
|
@@ -81,18 +84,15 @@ public class CallQueueManager<E extends Schedulable>
|
|
|
namespace, conf);
|
|
|
int[] capacityWeights = parseCapacityWeights(priorityLevels,
|
|
|
namespace, conf);
|
|
|
+ this.serverFailOverEnabled = getServerFailOverEnable(namespace, conf);
|
|
|
BlockingQueue<E> bq = createCallQueueInstance(backingClass,
|
|
|
priorityLevels, maxQueueSize, namespace, capacityWeights, conf);
|
|
|
this.clientBackOffEnabled = clientBackOffEnabled;
|
|
|
- this.serverFailOverEnabled = conf.getBoolean(
|
|
|
- namespace + "." +
|
|
|
- CommonConfigurationKeys.IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE,
|
|
|
- CommonConfigurationKeys.IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE_DEFAULT);
|
|
|
this.putRef = new AtomicReference<BlockingQueue<E>>(bq);
|
|
|
this.takeRef = new AtomicReference<BlockingQueue<E>>(bq);
|
|
|
LOG.info("Using callQueue: {}, queueCapacity: {}, " +
|
|
|
- "scheduler: {}, ipcBackoff: {}.",
|
|
|
- backingClass, maxQueueSize, schedulerClass, clientBackOffEnabled);
|
|
|
+ "scheduler: {}, ipcBackoff: {}, ipcFailOver: {}.",
|
|
|
+ backingClass, maxQueueSize, schedulerClass, clientBackOffEnabled, serverFailOverEnabled);
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting // only!
|
|
@@ -105,6 +105,41 @@ public class CallQueueManager<E extends Schedulable>
|
|
|
this.serverFailOverEnabled = serverFailOverEnabled;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Return boolean value configured by property 'ipc.<port>.callqueue.overflow.trigger.failover'
|
|
|
+ * if it is present. If the config is not present, default config
|
|
|
+ * (without port) is used to derive class i.e 'ipc.callqueue.overflow.trigger.failover',
|
|
|
+ * and derived value is returned if configured. Otherwise, default value
|
|
|
+ * {@link CommonConfigurationKeys#IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE_DEFAULT} is returned.
|
|
|
+ *
|
|
|
+ * @param namespace Namespace "ipc" + "." + Server's listener port.
|
|
|
+ * @param conf Configuration properties.
|
|
|
+ * @return Value returned based on configuration.
|
|
|
+ */
|
|
|
+ private boolean getServerFailOverEnable(String namespace, Configuration conf) {
|
|
|
+ String propertyKey = namespace + "." +
|
|
|
+ CommonConfigurationKeys.IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE;
|
|
|
+
|
|
|
+ if (conf.get(propertyKey) != null) {
|
|
|
+ return conf.getBoolean(propertyKey,
|
|
|
+ CommonConfigurationKeys.IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ String[] nsPort = namespace.split("\\.");
|
|
|
+ if (nsPort.length == 2) {
|
|
|
+ // Only if ns is split with ".", we can separate namespace and port.
|
|
|
+ // In the absence of "ipc.<port>.callqueue.overflow.trigger.failover" property,
|
|
|
+ // we look up "ipc.callqueue.overflow.trigger.failover" property.
|
|
|
+ return conf.getBoolean(nsPort[0] + "."
|
|
|
+ + IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE, IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE_DEFAULT);
|
|
|
+ }
|
|
|
+
|
|
|
+ // Otherwise return default value.
|
|
|
+ LOG.info("{} not specified set default value is {}",
|
|
|
+ IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE, IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE_DEFAULT);
|
|
|
+ return CommonConfigurationKeys.IPC_CALLQUEUE_SERVER_FAILOVER_ENABLE_DEFAULT;
|
|
|
+ }
|
|
|
+
|
|
|
private static <T extends RpcScheduler> T createScheduler(
|
|
|
Class<T> theClass, int priorityLevels, String ns, Configuration conf) {
|
|
|
// Used for custom, configurable scheduler
|
|
@@ -155,9 +190,9 @@ public class CallQueueManager<E extends Schedulable>
|
|
|
// Used for custom, configurable callqueues
|
|
|
try {
|
|
|
Constructor<T> ctor = theClass.getDeclaredConstructor(int.class,
|
|
|
- int.class, String.class, int[].class, Configuration.class);
|
|
|
- return ctor.newInstance(priorityLevels, maxLen, ns,
|
|
|
- capacityWeights, conf);
|
|
|
+ int.class, String.class, int[].class, boolean.class, Configuration.class);
|
|
|
+ return ctor.newInstance(priorityLevels, maxLen, ns, capacityWeights,
|
|
|
+ this.serverFailOverEnabled, conf);
|
|
|
} catch (RuntimeException e) {
|
|
|
throw e;
|
|
|
} catch (InvocationTargetException e) {
|
|
@@ -199,6 +234,20 @@ public class CallQueueManager<E extends Schedulable>
|
|
|
return clientBackOffEnabled;
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ public boolean isServerFailOverEnabled() {
|
|
|
+ return serverFailOverEnabled;
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ public boolean isServerFailOverEnabledByQueue() {
|
|
|
+ BlockingQueue<E> bq = putRef.get();
|
|
|
+ if (bq instanceof FairCallQueue) {
|
|
|
+ return ((FairCallQueue<E>) bq).isServerFailOverEnabled();
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
// Based on policy to determine back off current call
|
|
|
boolean shouldBackOff(Schedulable e) {
|
|
|
return scheduler.shouldBackOff(e);
|
|
@@ -421,6 +470,9 @@ public class CallQueueManager<E extends Schedulable>
|
|
|
RpcScheduler newScheduler = createScheduler(schedulerClass, priorityLevels,
|
|
|
ns, conf);
|
|
|
int[] capacityWeights = parseCapacityWeights(priorityLevels, ns, conf);
|
|
|
+
|
|
|
+ // Update serverFailOverEnabled.
|
|
|
+ this.serverFailOverEnabled = getServerFailOverEnable(ns, conf);
|
|
|
BlockingQueue<E> newQ = createCallQueueInstance(queueClassToUse,
|
|
|
priorityLevels, maxSize, ns, capacityWeights, conf);
|
|
|
|