diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java index c5699111f8..3026ab9ce5 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/HadoopFSKeyMaterialStore.java @@ -24,12 +24,13 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.parquet.crypto.ParquetCryptoRuntimeException; -import org.codehaus.jackson.JsonNode; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.OutputStream; import java.util.HashMap; import java.util.Map; import java.util.Set; @@ -75,7 +76,7 @@ public String getKeyMaterial(String keyIDInFile) throws ParquetCryptoRuntimeExc private void loadKeyMaterialMap() { try (FSDataInputStream keyMaterialStream = hadoopFileSystem.open(keyMaterialFile)) { JsonNode keyMaterialJson = objectMapper.readTree(keyMaterialStream); - keyMaterialMap = objectMapper.readValue(keyMaterialJson, + keyMaterialMap = objectMapper.readValue(keyMaterialJson.traverse(), new TypeReference>() { }); } catch (FileNotFoundException e) { throw new ParquetCryptoRuntimeException("External key material not found at " + keyMaterialFile, e); @@ -87,7 +88,7 @@ private void loadKeyMaterialMap() { @Override public void saveMaterial() throws ParquetCryptoRuntimeException { try (FSDataOutputStream keyMaterialStream = hadoopFileSystem.create(keyMaterialFile)) { - objectMapper.writeValue(keyMaterialStream, keyMaterialMap); + objectMapper.writeValue((OutputStream) keyMaterialStream, keyMaterialMap); } catch (IOException e) { throw new ParquetCryptoRuntimeException("Failed to save key material in " + keyMaterialFile, e); } diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyMaterial.java b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyMaterial.java index 3d49ff4ad5..8be7365003 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyMaterial.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyMaterial.java @@ -25,8 +25,8 @@ import java.util.Map; import org.apache.parquet.crypto.ParquetCryptoRuntimeException; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.core.type.TypeReference; /** * KeyMaterial class represents the "key material", keeping the information that allows readers to recover an encryption key (see diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyMetadata.java b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyMetadata.java index f75fe7e0da..6ecc0d7f60 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyMetadata.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/KeyMetadata.java @@ -26,8 +26,8 @@ import java.util.Map; import org.apache.parquet.crypto.ParquetCryptoRuntimeException; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.core.type.TypeReference; /** * Parquet encryption specification defines "key metadata" as an arbitrary byte array, generated by file writers for each encryption key, diff --git a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/LocalWrapKmsClient.java b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/LocalWrapKmsClient.java index 921a2dbeda..b696f81d69 100644 --- a/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/LocalWrapKmsClient.java +++ b/parquet-hadoop/src/main/java/org/apache/parquet/crypto/keytools/LocalWrapKmsClient.java @@ -22,8 +22,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.parquet.crypto.KeyAccessDeniedException; import org.apache.parquet.crypto.ParquetCryptoRuntimeException; -import org.codehaus.jackson.map.ObjectMapper; -import org.codehaus.jackson.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.core.type.TypeReference; import java.io.IOException; import java.io.StringReader; diff --git a/parquet-hadoop/src/test/java/org/apache/parquet/crypto/keytools/samples/VaultClient.java b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/keytools/samples/VaultClient.java index 9eb9c517f3..75876a6dde 100755 --- a/parquet-hadoop/src/test/java/org/apache/parquet/crypto/keytools/samples/VaultClient.java +++ b/parquet-hadoop/src/test/java/org/apache/parquet/crypto/keytools/samples/VaultClient.java @@ -30,7 +30,7 @@ import org.apache.parquet.crypto.ParquetCryptoRuntimeException; import org.apache.parquet.crypto.keytools.KeyToolkit; import org.apache.parquet.crypto.keytools.KmsClient; -import org.codehaus.jackson.map.ObjectMapper; +import com.fasterxml.jackson.databind.ObjectMapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -171,7 +171,7 @@ private String executeAndGetResponse(String endPoint, Request request) { private static String parseReturn(String response, String searchKey) { String matchingValue; try { - matchingValue = objectMapper.readTree(response).findValue(searchKey).getTextValue(); + matchingValue = objectMapper.readTree(response).findValue(searchKey).textValue(); } catch (IOException e) { throw new ParquetCryptoRuntimeException("Failed to parse vault response. " + searchKey + " not found." + response, e); }