HADOOP-13678 Update jackson from 1.9.13 to 2.x in hadoop-tools. Contributed by Akira Ajisaka.
This commit is contained in:
parent
4d2f380d78
commit
2cc841f16e
@ -181,5 +181,9 @@
|
||||
<version>2.4.0</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
@ -18,6 +18,9 @@
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.web.oauth2;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectReader;
|
||||
import com.squareup.okhttp.OkHttpClient;
|
||||
import com.squareup.okhttp.Request;
|
||||
import com.squareup.okhttp.RequestBody;
|
||||
@ -29,8 +32,6 @@
|
||||
import org.apache.hadoop.hdfs.web.URLConnectionFactory;
|
||||
import org.apache.hadoop.util.Timer;
|
||||
import org.apache.http.HttpStatus;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.map.ObjectReader;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
@ -217,6 +217,10 @@
|
||||
<artifactId>mockito-all</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
|
||||
</dependencies>
|
||||
</project>
|
||||
|
@ -40,6 +40,10 @@
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonParseException;
|
||||
import com.fasterxml.jackson.core.JsonParser;
|
||||
import com.fasterxml.jackson.databind.JsonMappingException;
|
||||
import com.fasterxml.jackson.databind.JsonNode;
|
||||
import org.apache.commons.lang.StringUtils;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
@ -63,14 +67,10 @@
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.codehaus.jackson.JsonNode;
|
||||
import org.codehaus.jackson.JsonParseException;
|
||||
import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.map.JsonMappingException;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.microsoft.azure.storage.StorageException;
|
||||
|
||||
@ -193,8 +193,8 @@ public FolderRenamePending(Path redoFile, NativeAzureFileSystem fs)
|
||||
if (oldFolderName == null || newFolderName == null) {
|
||||
this.committed = false;
|
||||
} else {
|
||||
this.srcKey = oldFolderName.getTextValue();
|
||||
this.dstKey = newFolderName.getTextValue();
|
||||
this.srcKey = oldFolderName.textValue();
|
||||
this.dstKey = newFolderName.textValue();
|
||||
if (this.srcKey == null || this.dstKey == null) {
|
||||
this.committed = false;
|
||||
} else {
|
||||
@ -203,7 +203,7 @@ public FolderRenamePending(Path redoFile, NativeAzureFileSystem fs)
|
||||
this.committed = false;
|
||||
} else {
|
||||
for (int i = 0; i < fileList.size(); i++) {
|
||||
fileStrList.add(fileList.get(i).getTextValue());
|
||||
fileStrList.add(fileList.get(i).textValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -123,16 +123,6 @@
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>jackson-mapper-asl</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>jackson-core-asl</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-httpclient</groupId>
|
||||
<artifactId>commons-httpclient</artifactId>
|
||||
@ -150,5 +140,13 @@
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-annotations</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
@ -18,7 +18,7 @@
|
||||
|
||||
package org.apache.hadoop.fs.swift.auth;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonProperty;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
|
||||
/**
|
||||
* Class that represents authentication request to Openstack Keystone.
|
||||
|
@ -18,7 +18,7 @@
|
||||
|
||||
package org.apache.hadoop.fs.swift.auth.entities;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
|
||||
/**
|
||||
* Access token representation of Openstack Keystone authentication.
|
||||
|
@ -18,7 +18,7 @@
|
||||
|
||||
package org.apache.hadoop.fs.swift.auth.entities;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
|
@ -18,7 +18,7 @@
|
||||
|
||||
package org.apache.hadoop.fs.swift.auth.entities;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
|
||||
import java.net.URI;
|
||||
|
||||
|
@ -18,7 +18,7 @@
|
||||
|
||||
package org.apache.hadoop.fs.swift.auth.entities;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonIgnoreProperties;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
|
||||
/**
|
||||
* Tenant is abstraction in Openstack which describes all account
|
||||
|
@ -18,8 +18,8 @@
|
||||
|
||||
package org.apache.hadoop.fs.swift.auth.entities;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
|
||||
import org.apache.hadoop.fs.swift.auth.Roles;
|
||||
import org.codehaus.jackson.annotate.JsonIgnoreProperties;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
|
@ -17,6 +17,8 @@
|
||||
*/
|
||||
package org.apache.hadoop.fs.swift.snative;
|
||||
|
||||
import com.fasterxml.jackson.databind.type.CollectionType;
|
||||
|
||||
import org.apache.commons.httpclient.Header;
|
||||
import org.apache.commons.httpclient.HttpStatus;
|
||||
import org.apache.commons.logging.Log;
|
||||
@ -36,7 +38,6 @@
|
||||
import org.apache.hadoop.fs.swift.util.JSONUtil;
|
||||
import org.apache.hadoop.fs.swift.util.SwiftObjectPath;
|
||||
import org.apache.hadoop.fs.swift.util.SwiftUtils;
|
||||
import org.codehaus.jackson.map.type.CollectionType;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.FileNotFoundException;
|
||||
|
@ -18,12 +18,12 @@
|
||||
|
||||
package org.apache.hadoop.fs.swift.util;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonGenerationException;
|
||||
import com.fasterxml.jackson.core.type.TypeReference;
|
||||
import com.fasterxml.jackson.databind.JsonMappingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.type.CollectionType;
|
||||
import org.apache.hadoop.fs.swift.exceptions.SwiftJsonMarshallingException;
|
||||
import org.codehaus.jackson.JsonGenerationException;
|
||||
import org.codehaus.jackson.map.JsonMappingException;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.map.type.CollectionType;
|
||||
import org.codehaus.jackson.type.TypeReference;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.StringWriter;
|
||||
@ -54,9 +54,7 @@ public static String toJSON(Object object) throws
|
||||
try {
|
||||
jsonMapper.writeValue(json, object);
|
||||
return json.toString();
|
||||
} catch (JsonGenerationException e) {
|
||||
throw new SwiftJsonMarshallingException(e.toString(), e);
|
||||
} catch (JsonMappingException e) {
|
||||
} catch (JsonGenerationException | JsonMappingException e) {
|
||||
throw new SwiftJsonMarshallingException(e.toString(), e);
|
||||
}
|
||||
}
|
||||
@ -96,9 +94,7 @@ public static <T> T toObject(String value,
|
||||
throws IOException {
|
||||
try {
|
||||
return (T)jsonMapper.readValue(value, typeReference);
|
||||
} catch (JsonGenerationException e) {
|
||||
throw new SwiftJsonMarshallingException("Error generating response", e);
|
||||
} catch (JsonMappingException e) {
|
||||
} catch (JsonGenerationException | JsonMappingException e) {
|
||||
throw new SwiftJsonMarshallingException("Error generating response", e);
|
||||
}
|
||||
}
|
||||
@ -115,11 +111,7 @@ public static <T> T toObject(String value,
|
||||
throws IOException {
|
||||
try {
|
||||
return (T)jsonMapper.readValue(value, collectionType);
|
||||
} catch (JsonGenerationException e) {
|
||||
throw new SwiftJsonMarshallingException(e.toString()
|
||||
+ " source: " + value,
|
||||
e);
|
||||
} catch (JsonMappingException e) {
|
||||
} catch (JsonGenerationException | JsonMappingException e) {
|
||||
throw new SwiftJsonMarshallingException(e.toString()
|
||||
+ " source: " + value,
|
||||
e);
|
||||
|
@ -94,6 +94,15 @@
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
|
@ -21,6 +21,12 @@
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonEncoding;
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.core.Version;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.conf.Configured;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@ -36,13 +42,6 @@
|
||||
import org.apache.hadoop.tools.rumen.serializers.*;
|
||||
import org.apache.hadoop.tools.rumen.state.*;
|
||||
|
||||
import org.codehaus.jackson.JsonEncoding;
|
||||
import org.codehaus.jackson.JsonFactory;
|
||||
import org.codehaus.jackson.JsonGenerator;
|
||||
import org.codehaus.jackson.Version;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.map.module.SimpleModule;
|
||||
|
||||
public class Anonymizer extends Configured implements Tool {
|
||||
private boolean anonymizeTrace = false;
|
||||
private Path inputTracePath = null;
|
||||
@ -88,8 +87,8 @@ private void initialize(String[] args) throws Exception {
|
||||
|
||||
outMapper = new ObjectMapper();
|
||||
// define a module
|
||||
SimpleModule module = new SimpleModule("Anonymization Serializer",
|
||||
new Version(0, 1, 1, "FINAL"));
|
||||
SimpleModule module = new SimpleModule(
|
||||
"Anonymization Serializer", new Version(0, 1, 1, "FINAL", "", ""));
|
||||
// add various serializers to the module
|
||||
// use the default (as-is) serializer for default data types
|
||||
module.addSerializer(DataType.class, new DefaultRumenSerializer());
|
||||
@ -106,7 +105,7 @@ private void initialize(String[] args) throws Exception {
|
||||
// register the module with the object-mapper
|
||||
outMapper.registerModule(module);
|
||||
|
||||
outFactory = outMapper.getJsonFactory();
|
||||
outFactory = outMapper.getFactory();
|
||||
}
|
||||
|
||||
// anonymize the job trace file
|
||||
@ -191,8 +190,8 @@ private JsonGenerator createJsonGenerator(Configuration conf, Path path)
|
||||
output = outFS.create(path);
|
||||
}
|
||||
|
||||
JsonGenerator outGen = outFactory.createJsonGenerator(output,
|
||||
JsonEncoding.UTF8);
|
||||
JsonGenerator outGen =
|
||||
outFactory.createGenerator(output, JsonEncoding.UTF8);
|
||||
outGen.useDefaultPrettyPrinter();
|
||||
|
||||
return outGen;
|
||||
|
@ -35,6 +35,7 @@
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
||||
@ -57,8 +58,6 @@
|
||||
import org.apache.hadoop.io.compress.CodecPool;
|
||||
import org.apache.hadoop.io.compress.Decompressor;
|
||||
|
||||
import org.codehaus.jackson.JsonProcessingException;
|
||||
|
||||
/**
|
||||
* This is the main class for rumen log mining functionality.
|
||||
*
|
||||
|
@ -18,15 +18,14 @@
|
||||
package org.apache.hadoop.tools.rumen;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonParser;
|
||||
import com.fasterxml.jackson.databind.JsonMappingException;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.map.DeserializationConfig;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
/**
|
||||
* A simple wrapper for parsing JSON-encoded data using ObjectMapper.
|
||||
@ -50,11 +49,9 @@ class JsonObjectMapperParser<T> implements Closeable {
|
||||
public JsonObjectMapperParser(Path path, Class<? extends T> clazz,
|
||||
Configuration conf) throws IOException {
|
||||
mapper = new ObjectMapper();
|
||||
mapper.configure(
|
||||
DeserializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
|
||||
this.clazz = clazz;
|
||||
InputStream input = new PossiblyDecompressedInputStream(path, conf);
|
||||
jsonParser = mapper.getJsonFactory().createJsonParser(input);
|
||||
jsonParser = mapper.getFactory().createParser(input);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -66,10 +63,8 @@ public JsonObjectMapperParser(Path path, Class<? extends T> clazz,
|
||||
public JsonObjectMapperParser(InputStream input, Class<? extends T> clazz)
|
||||
throws IOException {
|
||||
mapper = new ObjectMapper();
|
||||
mapper.configure(
|
||||
DeserializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
|
||||
this.clazz = clazz;
|
||||
jsonParser = mapper.getJsonFactory().createJsonParser(input);
|
||||
jsonParser = mapper.getFactory().createParser(input);
|
||||
}
|
||||
|
||||
/**
|
||||
@ -82,7 +77,7 @@ public JsonObjectMapperParser(InputStream input, Class<? extends T> clazz)
|
||||
public T getNext() throws IOException {
|
||||
try {
|
||||
return mapper.readValue(jsonParser, clazz);
|
||||
} catch (EOFException e) {
|
||||
} catch (JsonMappingException e) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
@ -21,16 +21,15 @@
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonEncoding;
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.core.Version;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import org.apache.hadoop.mapreduce.ID;
|
||||
import org.apache.hadoop.tools.rumen.datatypes.DataType;
|
||||
import org.apache.hadoop.tools.rumen.serializers.DefaultRumenSerializer;
|
||||
import org.apache.hadoop.tools.rumen.serializers.ObjectStringSerializer;
|
||||
import org.codehaus.jackson.JsonEncoding;
|
||||
import org.codehaus.jackson.JsonGenerator;
|
||||
import org.codehaus.jackson.Version;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.map.SerializationConfig;
|
||||
import org.codehaus.jackson.map.module.SimpleModule;
|
||||
|
||||
/**
|
||||
* Simple wrapper around {@link JsonGenerator} to write objects in JSON format.
|
||||
@ -41,12 +40,10 @@ public class JsonObjectMapperWriter<T> implements Closeable {
|
||||
|
||||
public JsonObjectMapperWriter(OutputStream output, boolean prettyPrint) throws IOException {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(
|
||||
SerializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
|
||||
|
||||
// define a module
|
||||
SimpleModule module = new SimpleModule("Default Serializer",
|
||||
new Version(0, 1, 1, "FINAL"));
|
||||
SimpleModule module = new SimpleModule(
|
||||
"Default Serializer", new Version(0, 1, 1, "FINAL", "", ""));
|
||||
// add various serializers to the module
|
||||
// add default (all-pass) serializer for all rumen specific data types
|
||||
module.addSerializer(DataType.class, new DefaultRumenSerializer());
|
||||
@ -56,9 +53,7 @@ public JsonObjectMapperWriter(OutputStream output, boolean prettyPrint) throws I
|
||||
// register the module with the object-mapper
|
||||
mapper.registerModule(module);
|
||||
|
||||
mapper.getJsonFactory();
|
||||
writer = mapper.getJsonFactory().createJsonGenerator(
|
||||
output, JsonEncoding.UTF8);
|
||||
writer = mapper.getFactory().createGenerator(output, JsonEncoding.UTF8);
|
||||
if (prettyPrint) {
|
||||
writer.useDefaultPrettyPrinter();
|
||||
}
|
||||
|
@ -27,9 +27,9 @@
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonAnySetter;
|
||||
import org.apache.hadoop.mapreduce.JobID;
|
||||
import org.apache.hadoop.tools.rumen.datatypes.*;
|
||||
import org.codehaus.jackson.annotate.JsonAnySetter;
|
||||
|
||||
/**
|
||||
* A {@link LoggedDiscreteCDF} is a representation of an hadoop job, with the
|
||||
|
@ -25,8 +25,8 @@
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonAnySetter;
|
||||
import org.apache.hadoop.tools.rumen.datatypes.NodeName;
|
||||
import org.codehaus.jackson.annotate.JsonAnySetter;
|
||||
|
||||
/**
|
||||
* A {@link LoggedLocation} is a representation of a point in an hierarchical
|
||||
|
@ -29,8 +29,8 @@
|
||||
import java.util.ArrayList;
|
||||
import java.util.Comparator;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonAnySetter;
|
||||
import org.apache.hadoop.tools.rumen.datatypes.NodeName;
|
||||
import org.codehaus.jackson.annotate.JsonAnySetter;
|
||||
|
||||
/**
|
||||
* A {@link LoggedNetworkTopology} represents a tree that in turn represents a
|
||||
|
@ -17,11 +17,11 @@
|
||||
*/
|
||||
package org.apache.hadoop.tools.rumen;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonAnySetter;
|
||||
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonAnySetter;
|
||||
|
||||
/**
|
||||
* A {@link LoggedSingleRelativeRanking} represents an X-Y coordinate of a
|
||||
* single point in a discrete CDF.
|
||||
|
@ -23,13 +23,13 @@
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonAnySetter;
|
||||
import org.apache.hadoop.mapreduce.TaskID;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JhCounter;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JhCounterGroup;
|
||||
import org.apache.hadoop.mapreduce.jobhistory.JhCounters;
|
||||
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.codehaus.jackson.annotate.JsonAnySetter;
|
||||
|
||||
/**
|
||||
* A {@link LoggedTask} represents a [hadoop] task that is part of a hadoop job.
|
||||
|
@ -23,8 +23,8 @@
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonAnySetter;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.codehaus.jackson.annotate.JsonAnySetter;
|
||||
|
||||
// HACK ALERT!!! This "should" have have two subclasses, which might be called
|
||||
// LoggedMapTaskAttempt and LoggedReduceTaskAttempt, but
|
||||
|
@ -17,12 +17,12 @@
|
||||
*/
|
||||
package org.apache.hadoop.tools.rumen.datatypes;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.tools.rumen.ParsedHost;
|
||||
import org.apache.hadoop.tools.rumen.anonymization.WordList;
|
||||
import org.apache.hadoop.tools.rumen.state.State;
|
||||
import org.apache.hadoop.tools.rumen.state.StatePool;
|
||||
import org.codehaus.jackson.annotate.JsonIgnore;
|
||||
|
||||
/**
|
||||
* Represents the cluster host.
|
||||
|
@ -17,12 +17,12 @@
|
||||
*/
|
||||
package org.apache.hadoop.tools.rumen.serializers;
|
||||
|
||||
import java.io.IOException;
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.JsonSerializer;
|
||||
import com.fasterxml.jackson.databind.SerializerProvider;
|
||||
|
||||
import org.codehaus.jackson.JsonGenerator;
|
||||
import org.codehaus.jackson.JsonProcessingException;
|
||||
import org.codehaus.jackson.map.JsonSerializer;
|
||||
import org.codehaus.jackson.map.SerializerProvider;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A JSON serializer for Strings.
|
||||
|
@ -19,13 +19,13 @@
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.JsonSerializer;
|
||||
import com.fasterxml.jackson.databind.SerializerProvider;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.tools.rumen.datatypes.AnonymizableDataType;
|
||||
import org.apache.hadoop.tools.rumen.state.StatePool;
|
||||
import org.codehaus.jackson.JsonGenerator;
|
||||
import org.codehaus.jackson.JsonProcessingException;
|
||||
import org.codehaus.jackson.map.JsonSerializer;
|
||||
import org.codehaus.jackson.map.SerializerProvider;
|
||||
|
||||
/**
|
||||
* Default Rumen JSON serializer.
|
||||
|
@ -19,11 +19,12 @@
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.JsonSerializer;
|
||||
import com.fasterxml.jackson.databind.SerializerProvider;
|
||||
|
||||
import org.apache.hadoop.tools.rumen.datatypes.DataType;
|
||||
import org.codehaus.jackson.JsonGenerator;
|
||||
import org.codehaus.jackson.JsonProcessingException;
|
||||
import org.codehaus.jackson.map.JsonSerializer;
|
||||
import org.codehaus.jackson.map.SerializerProvider;
|
||||
|
||||
/**
|
||||
* Default Rumen JSON serializer.
|
||||
|
@ -17,12 +17,12 @@
|
||||
*/
|
||||
package org.apache.hadoop.tools.rumen.serializers;
|
||||
|
||||
import java.io.IOException;
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.JsonSerializer;
|
||||
import com.fasterxml.jackson.databind.SerializerProvider;
|
||||
|
||||
import org.codehaus.jackson.JsonGenerator;
|
||||
import org.codehaus.jackson.JsonProcessingException;
|
||||
import org.codehaus.jackson.map.JsonSerializer;
|
||||
import org.codehaus.jackson.map.SerializerProvider;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Rumen JSON serializer for serializing object using toSring() API.
|
||||
|
@ -17,7 +17,7 @@
|
||||
*/
|
||||
package org.apache.hadoop.tools.rumen.state;
|
||||
|
||||
import org.codehaus.jackson.annotate.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
|
||||
/**
|
||||
* Represents a state. This state is managed by {@link StatePool}.
|
||||
|
@ -19,13 +19,13 @@
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonParser;
|
||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||
import com.fasterxml.jackson.databind.DeserializationContext;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode;
|
||||
import org.apache.hadoop.tools.rumen.state.StatePool.StatePair;
|
||||
import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.JsonProcessingException;
|
||||
import org.codehaus.jackson.map.DeserializationContext;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.map.deser.std.StdDeserializer;
|
||||
import org.codehaus.jackson.node.ObjectNode;
|
||||
|
||||
/**
|
||||
* Rumen JSON deserializer for deserializing the {@link State} object.
|
||||
@ -46,7 +46,7 @@ public StatePair deserialize(JsonParser parser,
|
||||
|
||||
try {
|
||||
stateClass =
|
||||
Class.forName(statePairObject.get("className").getTextValue().trim());
|
||||
Class.forName(statePairObject.get("className").textValue().trim());
|
||||
} catch (ClassNotFoundException cnfe) {
|
||||
throw new RuntimeException("Invalid classname!", cnfe);
|
||||
}
|
||||
|
@ -27,6 +27,14 @@
|
||||
import java.util.Calendar;
|
||||
import java.util.HashMap;
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.core.JsonEncoding;
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.core.JsonParser;
|
||||
import com.fasterxml.jackson.core.Version;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
@ -35,16 +43,6 @@
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.tools.rumen.Anonymizer;
|
||||
import org.apache.hadoop.tools.rumen.datatypes.DataType;
|
||||
import org.codehaus.jackson.JsonEncoding;
|
||||
import org.codehaus.jackson.JsonFactory;
|
||||
import org.codehaus.jackson.JsonGenerator;
|
||||
import org.codehaus.jackson.JsonParser;
|
||||
import org.codehaus.jackson.Version;
|
||||
import org.codehaus.jackson.annotate.JsonIgnore;
|
||||
import org.codehaus.jackson.map.DeserializationConfig;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.map.SerializationConfig;
|
||||
import org.codehaus.jackson.map.module.SimpleModule;
|
||||
|
||||
/**
|
||||
* A pool of states. States used by {@link DataType}'s can be managed the
|
||||
@ -212,20 +210,16 @@ private void reloadState(Path stateFile, Configuration conf)
|
||||
|
||||
private void read(DataInput in) throws IOException {
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
mapper.configure(
|
||||
DeserializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
|
||||
|
||||
// define a module
|
||||
SimpleModule module = new SimpleModule("State Serializer",
|
||||
new Version(0, 1, 1, "FINAL"));
|
||||
new Version(0, 1, 1, "FINAL", "", ""));
|
||||
// add the state deserializer
|
||||
module.addDeserializer(StatePair.class, new StateDeserializer());
|
||||
|
||||
// register the module with the object-mapper
|
||||
mapper.registerModule(module);
|
||||
|
||||
JsonParser parser =
|
||||
mapper.getJsonFactory().createJsonParser((DataInputStream)in);
|
||||
JsonParser parser = mapper.getFactory().createParser((DataInputStream)in);
|
||||
StatePool statePool = mapper.readValue(parser, StatePool.class);
|
||||
this.setStates(statePool.getStates());
|
||||
parser.close();
|
||||
@ -283,20 +277,18 @@ private void write(DataOutput out) throws IOException {
|
||||
// This is just a JSON experiment
|
||||
System.out.println("Dumping the StatePool's in JSON format.");
|
||||
ObjectMapper outMapper = new ObjectMapper();
|
||||
outMapper.configure(
|
||||
SerializationConfig.Feature.CAN_OVERRIDE_ACCESS_MODIFIERS, true);
|
||||
// define a module
|
||||
SimpleModule module = new SimpleModule("State Serializer",
|
||||
new Version(0, 1, 1, "FINAL"));
|
||||
new Version(0, 1, 1, "FINAL", "", ""));
|
||||
// add the state serializer
|
||||
//module.addSerializer(State.class, new StateSerializer());
|
||||
|
||||
// register the module with the object-mapper
|
||||
outMapper.registerModule(module);
|
||||
|
||||
JsonFactory outFactory = outMapper.getJsonFactory();
|
||||
JsonGenerator jGen =
|
||||
outFactory.createJsonGenerator((DataOutputStream)out, JsonEncoding.UTF8);
|
||||
JsonFactory outFactory = outMapper.getFactory();
|
||||
JsonGenerator jGen =
|
||||
outFactory.createGenerator((DataOutputStream)out, JsonEncoding.UTF8);
|
||||
jGen.useDefaultPrettyPrinter();
|
||||
|
||||
jGen.writeObject(this);
|
||||
|
@ -21,16 +21,17 @@
|
||||
|
||||
import java.util.List;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonEncoding;
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import com.fasterxml.jackson.core.JsonGenerator;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.codehaus.jackson.JsonEncoding;
|
||||
import org.codehaus.jackson.JsonGenerator;
|
||||
import org.codehaus.jackson.JsonFactory;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
@ -139,9 +140,9 @@ public static void main(String[] args) throws IOException {
|
||||
Path goldFilePath = new Path(filePath.getParent(), "gold"+testName);
|
||||
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
JsonFactory factory = mapper.getJsonFactory();
|
||||
JsonFactory factory = mapper.getFactory();
|
||||
FSDataOutputStream ostream = lfs.create(goldFilePath, true);
|
||||
JsonGenerator gen = factory.createJsonGenerator(ostream,
|
||||
JsonGenerator gen = factory.createGenerator(ostream,
|
||||
JsonEncoding.UTF8);
|
||||
gen.useDefaultPrettyPrinter();
|
||||
|
||||
|
@ -70,6 +70,10 @@
|
||||
<artifactId>jetty-util</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
|
@ -34,6 +34,9 @@
|
||||
import java.util.TreeMap;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.fasterxml.jackson.databind.ObjectWriter;
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
import org.apache.commons.cli.CommandLineParser;
|
||||
import org.apache.commons.cli.GnuParser;
|
||||
@ -41,9 +44,6 @@
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.yarn.sls.utils.SLSUtils;
|
||||
import org.codehaus.jackson.JsonFactory;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.map.ObjectWriter;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
@ -127,7 +127,7 @@ private static void generateSLSLoadFile(String inputFile, String outputFile)
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
ObjectWriter writer = mapper.writerWithDefaultPrettyPrinter();
|
||||
Iterator<Map> i = mapper.readValues(
|
||||
new JsonFactory().createJsonParser(input), Map.class);
|
||||
new JsonFactory().createParser(input), Map.class);
|
||||
while (i.hasNext()) {
|
||||
Map m = i.next();
|
||||
output.write(writer.writeValueAsString(createSLSJob(m)) + EOL);
|
||||
|
@ -33,6 +33,9 @@
|
||||
import java.util.Random;
|
||||
import java.util.Set;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
import org.apache.commons.cli.CommandLineParser;
|
||||
import org.apache.commons.cli.GnuParser;
|
||||
@ -66,8 +69,6 @@
|
||||
import org.apache.hadoop.yarn.sls.utils.SLSUtils;
|
||||
import org.apache.hadoop.yarn.util.resource.Resources;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.codehaus.jackson.JsonFactory;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
@ -281,7 +282,7 @@ private void startAMFromSLSTraces(Resource containerResource,
|
||||
Reader input =
|
||||
new InputStreamReader(new FileInputStream(inputTrace), "UTF-8");
|
||||
try {
|
||||
Iterator<Map> i = mapper.readValues(jsonF.createJsonParser(input),
|
||||
Iterator<Map> i = mapper.readValues(jsonF.createParser(input),
|
||||
Map.class);
|
||||
while (i.hasNext()) {
|
||||
Map jsonJob = i.next();
|
||||
|
@ -28,6 +28,8 @@
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import com.fasterxml.jackson.core.JsonFactory;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@ -37,8 +39,6 @@
|
||||
import org.apache.hadoop.tools.rumen.LoggedJob;
|
||||
import org.apache.hadoop.tools.rumen.LoggedTask;
|
||||
import org.apache.hadoop.tools.rumen.LoggedTaskAttempt;
|
||||
import org.codehaus.jackson.JsonFactory;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
@ -106,8 +106,7 @@ public static Set<String> parseNodesFromSLSTrace(String jobTrace)
|
||||
Reader input =
|
||||
new InputStreamReader(new FileInputStream(jobTrace), "UTF-8");
|
||||
try {
|
||||
Iterator<Map> i = mapper.readValues(
|
||||
jsonF.createJsonParser(input), Map.class);
|
||||
Iterator<Map> i = mapper.readValues(jsonF.createParser(input), Map.class);
|
||||
while (i.hasNext()) {
|
||||
Map jsonE = i.next();
|
||||
List tasks = (List) jsonE.get("job.tasks");
|
||||
@ -134,8 +133,7 @@ public static Set<String> parseNodesFromNodeFile(String nodeFile)
|
||||
Reader input =
|
||||
new InputStreamReader(new FileInputStream(nodeFile), "UTF-8");
|
||||
try {
|
||||
Iterator<Map> i = mapper.readValues(
|
||||
jsonF.createJsonParser(input), Map.class);
|
||||
Iterator<Map> i = mapper.readValues(jsonF.createParser(input), Map.class);
|
||||
while (i.hasNext()) {
|
||||
Map jsonE = i.next();
|
||||
String rack = "/" + jsonE.get("rack");
|
||||
|
Loading…
Reference in New Issue
Block a user