|
@@ -19,8 +19,12 @@
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
|
|
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.conf;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
|
+import org.apache.commons.io.serialization.ValidatingObjectInputStream;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.LogFactory;
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
|
+import org.apache.zookeeper.KeeperException.NodeExistsException;
|
|
|
|
+import org.slf4j.Logger;
|
|
|
|
+import org.slf4j.LoggerFactory;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.util.curator.ZKCuratorManager;
|
|
import org.apache.hadoop.util.curator.ZKCuratorManager;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
@@ -33,7 +37,6 @@ import org.apache.zookeeper.data.ACL;
|
|
|
|
|
|
import java.io.ByteArrayInputStream;
|
|
import java.io.ByteArrayInputStream;
|
|
import java.io.ByteArrayOutputStream;
|
|
import java.io.ByteArrayOutputStream;
|
|
-import java.io.ObjectInputStream;
|
|
|
|
import java.io.ObjectOutputStream;
|
|
import java.io.ObjectOutputStream;
|
|
import java.util.HashMap;
|
|
import java.util.HashMap;
|
|
import java.util.LinkedList;
|
|
import java.util.LinkedList;
|
|
@@ -229,7 +232,8 @@ public class ZKConfigurationStore extends YarnConfigurationStore {
|
|
|
|
|
|
private static Object deserializeObject(byte[] bytes) throws Exception {
|
|
private static Object deserializeObject(byte[] bytes) throws Exception {
|
|
try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
|
|
try (ByteArrayInputStream bais = new ByteArrayInputStream(bytes);
|
|
- ObjectInputStream ois = new ObjectInputStream(bais);) {
|
|
|
|
|
|
+ ValidatingObjectInputStream ois = new ValidatingObjectInputStream(bais);) {
|
|
|
|
+ ois.accept(LinkedList.class, LogMutation.class, HashMap.class, String.class);
|
|
return ois.readObject();
|
|
return ois.readObject();
|
|
}
|
|
}
|
|
}
|
|
}
|