HBASE-20884 Replace Base64 with j.u.Base64

Signed-off-by: Andrew Purtell <apurtell@apache.org>
Signed-off-by: tedyu <yuzhihong@gmail.com>
This commit is contained in:
Mike Drob 2018-07-13 09:40:51 -05:00
parent d3bb364595
commit 2a9162a503
No known key found for this signature in database
GPG Key ID: 3E48C0C6EF362B9E
14 changed files with 70 additions and 1825 deletions

View File

@ -28,6 +28,7 @@ import java.io.IOException;
import java.lang.reflect.InvocationTargetException; import java.lang.reflect.InvocationTargetException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays; import java.util.Arrays;
import java.util.Base64;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -41,7 +42,6 @@ import org.apache.hadoop.hbase.security.access.Permission;
import org.apache.hadoop.hbase.security.visibility.Authorizations; import org.apache.hadoop.hbase.security.visibility.Authorizations;
import org.apache.hadoop.hbase.testclassification.ClientTests; import org.apache.hadoop.hbase.testclassification.ClientTests;
import org.apache.hadoop.hbase.testclassification.SmallTests; import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.junit.Assert; import org.junit.Assert;
import org.junit.ClassRule; import org.junit.ClassRule;
@ -214,9 +214,9 @@ public class TestGet {
assertFalse("Should be deleted: " + jarFile.getPath(), jarFile.exists()); assertFalse("Should be deleted: " + jarFile.getPath(), jarFile.exists());
ClientProtos.Get getProto1 = ClientProtos.Get getProto1 =
ClientProtos.Get.parseFrom(Base64.decode(PB_GET)); ClientProtos.Get.parseFrom(Base64.getDecoder().decode(PB_GET));
ClientProtos.Get getProto2 = ClientProtos.Get getProto2 =
ClientProtos.Get.parseFrom(Base64.decode(PB_GET_WITH_FILTER_LIST)); ClientProtos.Get.parseFrom(Base64.getDecoder().decode(PB_GET_WITH_FILTER_LIST));
try { try {
ProtobufUtil.toGet(getProto1); ProtobufUtil.toGet(getProto1);
fail("Should not be able to load the filter class"); fail("Should not be able to load the filter class");
@ -233,7 +233,7 @@ public class TestGet {
instanceof DeserializationException); instanceof DeserializationException);
} }
FileOutputStream fos = new FileOutputStream(jarFile); FileOutputStream fos = new FileOutputStream(jarFile);
fos.write(Base64.decode(MOCK_FILTER_JAR)); fos.write(Base64.getDecoder().decode(MOCK_FILTER_JAR));
fos.close(); fos.close();
Get get1 = ProtobufUtil.toGet(getProto1); Get get1 = ProtobufUtil.toGet(getProto1);

View File

@ -1,76 +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.hadoop.hbase.util;
import java.io.UnsupportedEncodingException;
import java.util.Map;
import java.util.TreeMap;
import junit.framework.TestCase;
import org.apache.hadoop.hbase.HBaseClassTestRule;
import org.apache.hadoop.hbase.testclassification.MiscTests;
import org.apache.hadoop.hbase.testclassification.SmallTests;
import org.junit.ClassRule;
import org.junit.experimental.categories.Category;
/**
* Test order preservation characteristics of ordered Base64 dialect
*/
@Category({MiscTests.class, SmallTests.class})
public class TestBase64 extends TestCase {
@ClassRule
public static final HBaseClassTestRule CLASS_RULE =
HBaseClassTestRule.forClass(TestBase64.class);
// Note: uris is sorted. We need to prove that the ordered Base64
// preserves that ordering
private String[] uris = {
"dns://dns.powerset.com/www.powerset.com",
"dns:www.powerset.com",
"file:///usr/bin/java",
"filename",
"ftp://one.two.three/index.html",
"http://one.two.three/index.html",
"https://one.two.three:9443/index.html",
"r:dns://com.powerset.dns/www.powerset.com",
"r:ftp://three.two.one/index.html",
"r:http://three.two.one/index.html",
"r:https://three.two.one:9443/index.html"
};
/**
* the test
* @throws UnsupportedEncodingException
*/
public void testBase64() throws UnsupportedEncodingException {
TreeMap<String, String> sorted = new TreeMap<>();
for (int i = 0; i < uris.length; i++) {
byte[] bytes = uris[i].getBytes("UTF-8");
sorted.put(Base64.encodeBytes(bytes, Base64.ORDERED), uris[i]);
}
System.out.println();
int i = 0;
for (Map.Entry<String, String> e: sorted.entrySet()) {
assertTrue(uris[i++].compareTo(e.getValue()) == 0);
}
}
}

View File

@ -25,6 +25,7 @@ import java.nio.charset.Charset;
import java.nio.charset.CharsetDecoder; import java.nio.charset.CharsetDecoder;
import java.security.PrivilegedExceptionAction; import java.security.PrivilegedExceptionAction;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Base64;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -39,7 +40,7 @@ import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
import org.apache.hadoop.hbase.thrift.generated.Hbase; import org.apache.hadoop.hbase.thrift.generated.Hbase;
import org.apache.hadoop.hbase.thrift.generated.TCell; import org.apache.hadoop.hbase.thrift.generated.TCell;
import org.apache.hadoop.hbase.thrift.generated.TRowResult; import org.apache.hadoop.hbase.thrift.generated.TRowResult;
import org.apache.hadoop.hbase.util.Base64; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.thrift.protocol.TBinaryProtocol; import org.apache.thrift.protocol.TBinaryProtocol;
import org.apache.thrift.protocol.TProtocol; import org.apache.thrift.protocol.TProtocol;
import org.apache.thrift.transport.THttpClient; import org.apache.thrift.transport.THttpClient;
@ -218,7 +219,7 @@ public class HttpDoAsClient {
final byte[] outToken = context.initSecContext(new byte[0], 0, 0); final byte[] outToken = context.initSecContext(new byte[0], 0, 0);
StringBuffer outputBuffer = new StringBuffer(); StringBuffer outputBuffer = new StringBuffer();
outputBuffer.append("Negotiate "); outputBuffer.append("Negotiate ");
outputBuffer.append(Base64.encodeBytes(outToken).replace("\n", "")); outputBuffer.append(Bytes.toString(Base64.getEncoder().encode(outToken)));
System.out.print("Ticket is: " + outputBuffer); System.out.print("Ticket is: " + outputBuffer);
return outputBuffer.toString(); return outputBuffer.toString();
} }

View File

@ -23,6 +23,7 @@ import static java.lang.String.format;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Base64;
import java.util.HashSet; import java.util.HashSet;
import java.util.Set; import java.util.Set;
@ -47,7 +48,6 @@ import org.apache.hadoop.hbase.client.Put;
import org.apache.hadoop.hbase.client.RegionLocator; import org.apache.hadoop.hbase.client.RegionLocator;
import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.client.Table;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
@ -471,7 +471,7 @@ public class ImportTsv extends Configured implements Tool {
String actualSeparator = conf.get(SEPARATOR_CONF_KEY); String actualSeparator = conf.get(SEPARATOR_CONF_KEY);
if (actualSeparator != null) { if (actualSeparator != null) {
conf.set(SEPARATOR_CONF_KEY, conf.set(SEPARATOR_CONF_KEY,
Base64.encodeBytes(actualSeparator.getBytes())); Bytes.toString(Base64.getEncoder().encode(actualSeparator.getBytes())));
} }
// See if a non-default Mapper was set // See if a non-default Mapper was set

View File

@ -18,13 +18,14 @@
*/ */
package org.apache.hadoop.hbase.mapreduce; package org.apache.hadoop.hbase.mapreduce;
import java.util.Base64;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import org.apache.hadoop.conf.Configurable; import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.mapreduce.Partitioner; import org.apache.hadoop.mapreduce.Partitioner;
@ -63,11 +64,11 @@ implements Configurable {
private int lastReduces = -1; private int lastReduces = -1;
public static void setStartKey(Configuration conf, byte[] startKey) { public static void setStartKey(Configuration conf, byte[] startKey) {
conf.set(START_BASE64, Base64.encodeBytes(startKey)); conf.set(START_BASE64, Bytes.toString(Base64.getEncoder().encode(startKey)));
} }
public static void setEndKey(Configuration conf, byte[] endKey) { public static void setEndKey(Configuration conf, byte[] endKey) {
conf.set(END_BASE64, Base64.encodeBytes(endKey)); conf.set(END_BASE64, Bytes.toString(Base64.getEncoder().encode(endKey)));
} }
@SuppressWarnings("deprecation") @SuppressWarnings("deprecation")
@ -84,7 +85,7 @@ implements Configurable {
String base64Key, String deprecatedKey) { String base64Key, String deprecatedKey) {
String encoded = conf.get(base64Key); String encoded = conf.get(base64Key);
if (encoded != null) { if (encoded != null) {
return Base64.decode(encoded); return Base64.getDecoder().decode(encoded);
} }
String oldStyleVal = conf.get(deprecatedKey); String oldStyleVal = conf.get(deprecatedKey);
if (oldStyleVal == null) { if (oldStyleVal == null) {

View File

@ -23,6 +23,7 @@ import java.io.IOException;
import java.net.URL; import java.net.URL;
import java.net.URLDecoder; import java.net.URLDecoder;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Base64;
import java.util.Collection; import java.util.Collection;
import java.util.Enumeration; import java.util.Enumeration;
import java.util.HashMap; import java.util.HashMap;
@ -53,7 +54,6 @@ import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider; import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.security.token.TokenUtil; import org.apache.hadoop.hbase.security.token.TokenUtil;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.RegionSplitter; import org.apache.hadoop.hbase.util.RegionSplitter;
import org.apache.hadoop.hbase.zookeeper.ZKConfig; import org.apache.hadoop.hbase.zookeeper.ZKConfig;
@ -597,7 +597,7 @@ public class TableMapReduceUtil {
*/ */
public static String convertScanToString(Scan scan) throws IOException { public static String convertScanToString(Scan scan) throws IOException {
ClientProtos.Scan proto = ProtobufUtil.toScan(scan); ClientProtos.Scan proto = ProtobufUtil.toScan(scan);
return Base64.encodeBytes(proto.toByteArray()); return Bytes.toString(Base64.getEncoder().encode(proto.toByteArray()));
} }
/** /**
@ -608,7 +608,7 @@ public class TableMapReduceUtil {
* @throws IOException When reading the scan instance fails. * @throws IOException When reading the scan instance fails.
*/ */
public static Scan convertStringToScan(String base64) throws IOException { public static Scan convertStringToScan(String base64) throws IOException {
byte [] decoded = Base64.decode(base64); byte [] decoded = Base64.getDecoder().decode(base64);
return ProtobufUtil.toScan(ClientProtos.Scan.parseFrom(decoded)); return ProtobufUtil.toScan(ClientProtos.Scan.parseFrom(decoded));
} }

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.mapreduce;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Base64;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
@ -35,7 +36,6 @@ import org.apache.hadoop.hbase.TagType;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.security.visibility.InvalidLabelException; import org.apache.hadoop.hbase.security.visibility.InvalidLabelException;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.Counter;
@ -121,7 +121,7 @@ public class TextSortReducer extends
if (separator == null) { if (separator == null) {
separator = ImportTsv.DEFAULT_SEPARATOR; separator = ImportTsv.DEFAULT_SEPARATOR;
} else { } else {
separator = new String(Base64.decode(separator)); separator = Bytes.toString(Base64.getDecoder().decode(separator));
} }
// Should never get 0 as we are setting this to a valid value in job configuration. // Should never get 0 as we are setting this to a valid value in job configuration.

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.hbase.mapreduce;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Base64;
import java.util.List; import java.util.List;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -33,7 +34,6 @@ import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.BadTsvLineException; import org.apache.hadoop.hbase.mapreduce.ImportTsv.TsvParser.BadTsvLineException;
import org.apache.hadoop.hbase.security.visibility.CellVisibility; import org.apache.hadoop.hbase.security.visibility.CellVisibility;
import org.apache.hadoop.hbase.security.visibility.InvalidLabelException; import org.apache.hadoop.hbase.security.visibility.InvalidLabelException;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
@ -127,7 +127,7 @@ extends Mapper<LongWritable, Text, ImmutableBytesWritable, Put>
if (separator == null) { if (separator == null) {
separator = ImportTsv.DEFAULT_SEPARATOR; separator = ImportTsv.DEFAULT_SEPARATOR;
} else { } else {
separator = new String(Base64.decode(separator)); separator = new String(Base64.getDecoder().decode(separator));
} }
// Should never get 0 as we are setting this to a valid value in job // Should never get 0 as we are setting this to a valid value in job
// configuration. // configuration.

View File

@ -17,18 +17,18 @@
*/ */
package org.apache.hadoop.hbase.mapreduce; package org.apache.hadoop.hbase.mapreduce;
import java.io.IOException;
import java.util.Base64;
import org.apache.hadoop.io.LongWritable; import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.hbase.io.ImmutableBytesWritable; import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.Mapper;
import org.apache.hadoop.mapreduce.Counter; import org.apache.hadoop.mapreduce.Counter;
import org.apache.yetus.audience.InterfaceAudience; import org.apache.yetus.audience.InterfaceAudience;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import java.io.IOException;
/** /**
* Write table content out to map output files. * Write table content out to map output files.
*/ */
@ -61,7 +61,7 @@ extends Mapper<LongWritable, Text, ImmutableBytesWritable, Text>
/** /**
* Handles initializing this class with objects specific to it (i.e., the parser). * Handles initializing this class with objects specific to it (i.e., the parser).
* Common initialization that might be leveraged by a subsclass is done in * Common initialization that might be leveraged by a subclass is done in
* <code>doSetup</code>. Hence a subclass may choose to override this method * <code>doSetup</code>. Hence a subclass may choose to override this method
* and call <code>doSetup</code> as well before handling it's own custom params. * and call <code>doSetup</code> as well before handling it's own custom params.
* *
@ -92,7 +92,7 @@ extends Mapper<LongWritable, Text, ImmutableBytesWritable, Text>
if (separator == null) { if (separator == null) {
separator = ImportTsv.DEFAULT_SEPARATOR; separator = ImportTsv.DEFAULT_SEPARATOR;
} else { } else {
separator = new String(Base64.decode(separator)); separator = new String(Base64.getDecoder().decode(separator));
} }
skipBadLines = context.getConfiguration().getBoolean(ImportTsv.SKIP_LINES_CONF_KEY, true); skipBadLines = context.getConfiguration().getBoolean(ImportTsv.SKIP_LINES_CONF_KEY, true);

View File

@ -20,6 +20,7 @@
package org.apache.hadoop.hbase.rest; package org.apache.hadoop.hbase.rest;
import java.io.IOException; import java.io.IOException;
import java.util.Base64;
import javax.ws.rs.DELETE; import javax.ws.rs.DELETE;
import javax.ws.rs.GET; import javax.ws.rs.GET;
@ -40,7 +41,6 @@ import org.slf4j.LoggerFactory;
import org.apache.hadoop.hbase.rest.model.CellModel; import org.apache.hadoop.hbase.rest.model.CellModel;
import org.apache.hadoop.hbase.rest.model.CellSetModel; import org.apache.hadoop.hbase.rest.model.CellSetModel;
import org.apache.hadoop.hbase.rest.model.RowModel; import org.apache.hadoop.hbase.rest.model.RowModel;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@InterfaceAudience.Private @InterfaceAudience.Private
@ -171,10 +171,10 @@ public class ScannerInstanceResource extends ResourceBase {
} }
ResponseBuilder response = Response.ok(CellUtil.cloneValue(value)); ResponseBuilder response = Response.ok(CellUtil.cloneValue(value));
response.cacheControl(cacheControl); response.cacheControl(cacheControl);
response.header("X-Row", Base64.encodeBytes(CellUtil.cloneRow(value))); response.header("X-Row", Bytes.toString(Base64.getEncoder().encode(
response.header("X-Column", CellUtil.cloneRow(value))));
Base64.encodeBytes( response.header("X-Column", Bytes.toString(Base64.getEncoder().encode(
CellUtil.makeColumn(CellUtil.cloneFamily(value), CellUtil.cloneQualifier(value)))); CellUtil.makeColumn(CellUtil.cloneFamily(value), CellUtil.cloneQualifier(value)))));
response.header("X-Timestamp", value.getTimestamp()); response.header("X-Timestamp", value.getTimestamp());
servlet.getMetrics().incrementSucessfulGetRequests(1); servlet.getMetrics().incrementSucessfulGetRequests(1);
return response.build(); return response.build();

View File

@ -22,6 +22,7 @@ package org.apache.hadoop.hbase.rest.model;
import java.io.IOException; import java.io.IOException;
import java.io.Serializable; import java.io.Serializable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Base64;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.NavigableSet; import java.util.NavigableSet;
@ -72,7 +73,6 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler; import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
import org.apache.hadoop.hbase.rest.protobuf.generated.ScannerMessage.Scanner; import org.apache.hadoop.hbase.rest.protobuf.generated.ScannerMessage.Scanner;
import org.apache.hadoop.hbase.security.visibility.Authorizations; import org.apache.hadoop.hbase.security.visibility.Authorizations;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.Bytes;
@ -154,10 +154,10 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
switch (type) { switch (type) {
case BinaryComparator: case BinaryComparator:
case BinaryPrefixComparator: case BinaryPrefixComparator:
this.value = Base64.encodeBytes(comparator.getValue()); this.value = Bytes.toString(Base64.getEncoder().encode(comparator.getValue()));
break; break;
case BitComparator: case BitComparator:
this.value = Base64.encodeBytes(comparator.getValue()); this.value = Bytes.toString(Base64.getEncoder().encode(comparator.getValue()));
this.op = ((BitComparator)comparator).getOperator().toString(); this.op = ((BitComparator)comparator).getOperator().toString();
break; break;
case NullComparator: case NullComparator:
@ -175,13 +175,13 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
ByteArrayComparable comparator; ByteArrayComparable comparator;
switch (ComparatorType.valueOf(type)) { switch (ComparatorType.valueOf(type)) {
case BinaryComparator: case BinaryComparator:
comparator = new BinaryComparator(Base64.decode(value)); comparator = new BinaryComparator(Base64.getDecoder().decode(value));
break; break;
case BinaryPrefixComparator: case BinaryPrefixComparator:
comparator = new BinaryPrefixComparator(Base64.decode(value)); comparator = new BinaryPrefixComparator(Base64.getDecoder().decode(value));
break; break;
case BitComparator: case BitComparator:
comparator = new BitComparator(Base64.decode(value), comparator = new BitComparator(Base64.getDecoder().decode(value),
BitComparator.BitwiseOp.valueOf(op)); BitComparator.BitwiseOp.valueOf(op));
break; break;
case NullComparator: case NullComparator:
@ -265,20 +265,22 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
this.offset = ((ColumnPaginationFilter)filter).getOffset(); this.offset = ((ColumnPaginationFilter)filter).getOffset();
break; break;
case ColumnPrefixFilter: case ColumnPrefixFilter:
this.value = Base64.encodeBytes(((ColumnPrefixFilter)filter).getPrefix()); byte[] src = ((ColumnPrefixFilter)filter).getPrefix();
this.value = Bytes.toString(Base64.getEncoder().encode(src));
break; break;
case ColumnRangeFilter: case ColumnRangeFilter:
this.minColumn = Base64.encodeBytes(((ColumnRangeFilter)filter).getMinColumn()); ColumnRangeFilter crf = (ColumnRangeFilter)filter;
this.minColumnInclusive = ((ColumnRangeFilter)filter).getMinColumnInclusive(); this.minColumn = Bytes.toString(Base64.getEncoder().encode(crf.getMinColumn()));
this.maxColumn = Base64.encodeBytes(((ColumnRangeFilter)filter).getMaxColumn()); this.minColumnInclusive = crf.getMinColumnInclusive();
this.maxColumnInclusive = ((ColumnRangeFilter)filter).getMaxColumnInclusive(); this.maxColumn = Bytes.toString(Base64.getEncoder().encode(crf.getMaxColumn()));
this.maxColumnInclusive = crf.getMaxColumnInclusive();
break; break;
case DependentColumnFilter: { case DependentColumnFilter: {
DependentColumnFilter dcf = (DependentColumnFilter)filter; DependentColumnFilter dcf = (DependentColumnFilter)filter;
this.family = Base64.encodeBytes(dcf.getFamily()); this.family = Bytes.toString(Base64.getEncoder().encode(dcf.getFamily()));
byte[] qualifier = dcf.getQualifier(); byte[] qualifier = dcf.getQualifier();
if (qualifier != null) { if (qualifier != null) {
this.qualifier = Base64.encodeBytes(qualifier); this.qualifier = Bytes.toString(Base64.getEncoder().encode(qualifier));
} }
this.op = dcf.getOperator().toString(); this.op = dcf.getOperator().toString();
this.comparator = new ByteArrayComparableModel(dcf.getComparator()); this.comparator = new ByteArrayComparableModel(dcf.getComparator());
@ -295,13 +297,13 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
case KeyOnlyFilter: case KeyOnlyFilter:
break; break;
case InclusiveStopFilter: case InclusiveStopFilter:
this.value = this.value = Bytes.toString(Base64.getEncoder().encode(
Base64.encodeBytes(((InclusiveStopFilter)filter).getStopRowKey()); ((InclusiveStopFilter)filter).getStopRowKey()));
break; break;
case MultipleColumnPrefixFilter: case MultipleColumnPrefixFilter:
this.prefixes = new ArrayList<>(); this.prefixes = new ArrayList<>();
for (byte[] prefix: ((MultipleColumnPrefixFilter)filter).getPrefix()) { for (byte[] prefix: ((MultipleColumnPrefixFilter)filter).getPrefix()) {
this.prefixes.add(Base64.encodeBytes(prefix)); this.prefixes.add(Bytes.toString(Base64.getEncoder().encode(prefix)));
} }
break; break;
case MultiRowRangeFilter: case MultiRowRangeFilter:
@ -315,7 +317,8 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
this.value = Long.toString(((PageFilter)filter).getPageSize()); this.value = Long.toString(((PageFilter)filter).getPageSize());
break; break;
case PrefixFilter: case PrefixFilter:
this.value = Base64.encodeBytes(((PrefixFilter)filter).getPrefix()); this.value = Bytes.toString(Base64.getEncoder().encode(
((PrefixFilter)filter).getPrefix()));
break; break;
case FamilyFilter: case FamilyFilter:
case QualifierFilter: case QualifierFilter:
@ -332,10 +335,10 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
case SingleColumnValueExcludeFilter: case SingleColumnValueExcludeFilter:
case SingleColumnValueFilter: { case SingleColumnValueFilter: {
SingleColumnValueFilter scvf = (SingleColumnValueFilter) filter; SingleColumnValueFilter scvf = (SingleColumnValueFilter) filter;
this.family = Base64.encodeBytes(scvf.getFamily()); this.family = Bytes.toString(Base64.getEncoder().encode(scvf.getFamily()));
byte[] qualifier = scvf.getQualifier(); byte[] qualifier = scvf.getQualifier();
if (qualifier != null) { if (qualifier != null) {
this.qualifier = Base64.encodeBytes(qualifier); this.qualifier = Bytes.toString(Base64.getEncoder().encode(qualifier));
} }
this.op = scvf.getOperator().toString(); this.op = scvf.getOperator().toString();
this.comparator = this.comparator =
@ -374,16 +377,16 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
filter = new ColumnPaginationFilter(limit, offset); filter = new ColumnPaginationFilter(limit, offset);
break; break;
case ColumnPrefixFilter: case ColumnPrefixFilter:
filter = new ColumnPrefixFilter(Base64.decode(value)); filter = new ColumnPrefixFilter(Base64.getDecoder().decode(value));
break; break;
case ColumnRangeFilter: case ColumnRangeFilter:
filter = new ColumnRangeFilter(Base64.decode(minColumn), filter = new ColumnRangeFilter(Base64.getDecoder().decode(minColumn),
minColumnInclusive, Base64.decode(maxColumn), minColumnInclusive, Base64.getDecoder().decode(maxColumn),
maxColumnInclusive); maxColumnInclusive);
break; break;
case DependentColumnFilter: case DependentColumnFilter:
filter = new DependentColumnFilter(Base64.decode(family), filter = new DependentColumnFilter(Base64.getDecoder().decode(family),
qualifier != null ? Base64.decode(qualifier) : null, qualifier != null ? Base64.getDecoder().decode(qualifier) : null,
dropDependentColumn, CompareOperator.valueOf(op), comparator.build()); dropDependentColumn, CompareOperator.valueOf(op), comparator.build());
break; break;
case FamilyFilter: case FamilyFilter:
@ -400,7 +403,7 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
filter = new FirstKeyOnlyFilter(); filter = new FirstKeyOnlyFilter();
break; break;
case InclusiveStopFilter: case InclusiveStopFilter:
filter = new InclusiveStopFilter(Base64.decode(value)); filter = new InclusiveStopFilter(Base64.getDecoder().decode(value));
break; break;
case KeyOnlyFilter: case KeyOnlyFilter:
filter = new KeyOnlyFilter(); filter = new KeyOnlyFilter();
@ -408,7 +411,7 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
case MultipleColumnPrefixFilter: { case MultipleColumnPrefixFilter: {
byte[][] values = new byte[prefixes.size()][]; byte[][] values = new byte[prefixes.size()][];
for (int i = 0; i < prefixes.size(); i++) { for (int i = 0; i < prefixes.size(); i++) {
values[i] = Base64.decode(prefixes.get(i)); values[i] = Base64.getDecoder().decode(prefixes.get(i));
} }
filter = new MultipleColumnPrefixFilter(values); filter = new MultipleColumnPrefixFilter(values);
} break; } break;
@ -419,7 +422,7 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
filter = new PageFilter(Long.parseLong(value)); filter = new PageFilter(Long.parseLong(value));
break; break;
case PrefixFilter: case PrefixFilter:
filter = new PrefixFilter(Base64.decode(value)); filter = new PrefixFilter(Base64.getDecoder().decode(value));
break; break;
case QualifierFilter: case QualifierFilter:
filter = new QualifierFilter(CompareOperator.valueOf(op), comparator.build()); filter = new QualifierFilter(CompareOperator.valueOf(op), comparator.build());
@ -431,8 +434,8 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
filter = new RowFilter(CompareOperator.valueOf(op), comparator.build()); filter = new RowFilter(CompareOperator.valueOf(op), comparator.build());
break; break;
case SingleColumnValueFilter: case SingleColumnValueFilter:
filter = new SingleColumnValueFilter(Base64.decode(family), filter = new SingleColumnValueFilter(Base64.getDecoder().decode(family),
qualifier != null ? Base64.decode(qualifier) : null, qualifier != null ? Base64.getDecoder().decode(qualifier) : null,
CompareOperator.valueOf(op), comparator.build()); CompareOperator.valueOf(op), comparator.build());
if (ifMissing != null) { if (ifMissing != null) {
((SingleColumnValueFilter)filter).setFilterIfMissing(ifMissing); ((SingleColumnValueFilter)filter).setFilterIfMissing(ifMissing);
@ -442,8 +445,8 @@ public class ScannerModel implements ProtobufMessageHandler, Serializable {
} }
break; break;
case SingleColumnValueExcludeFilter: case SingleColumnValueExcludeFilter:
filter = new SingleColumnValueExcludeFilter(Base64.decode(family), filter = new SingleColumnValueExcludeFilter(Base64.getDecoder().decode(family),
qualifier != null ? Base64.decode(qualifier) : null, qualifier != null ? Base64.getDecoder().decode(qualifier) : null,
CompareOperator.valueOf(op), comparator.build()); CompareOperator.valueOf(op), comparator.build());
if (ifMissing != null) { if (ifMissing != null) {
((SingleColumnValueExcludeFilter)filter).setFilterIfMissing(ifMissing); ((SingleColumnValueExcludeFilter)filter).setFilterIfMissing(ifMissing);

View File

@ -25,12 +25,12 @@ import com.fasterxml.jackson.jaxrs.json.JacksonJaxbJsonProvider;
import java.io.IOException; import java.io.IOException;
import java.io.StringReader; import java.io.StringReader;
import java.io.StringWriter; import java.io.StringWriter;
import java.util.Base64;
import javax.ws.rs.core.MediaType; import javax.ws.rs.core.MediaType;
import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBContext;
import javax.xml.bind.JAXBException; import javax.xml.bind.JAXBException;
import org.apache.hadoop.hbase.rest.ProtobufMessageHandler; import org.apache.hadoop.hbase.rest.ProtobufMessageHandler;
import org.apache.hadoop.hbase.rest.provider.JAXBContextResolver; import org.apache.hadoop.hbase.rest.provider.JAXBContextResolver;
import org.apache.hadoop.hbase.util.Base64;
import org.junit.Test; import org.junit.Test;
public abstract class TestModelBase<T> { public abstract class TestModelBase<T> {
@ -91,7 +91,7 @@ public abstract class TestModelBase<T> {
Exception { Exception {
return (T)clazz.getMethod("getObjectFromMessage", byte[].class).invoke( return (T)clazz.getMethod("getObjectFromMessage", byte[].class).invoke(
clazz.getDeclaredConstructor().newInstance(), clazz.getDeclaredConstructor().newInstance(),
Base64.decode(AS_PB)); Base64.getDecoder().decode(AS_PB));
} }
protected abstract void checkModel(T model); protected abstract void checkModel(T model);

View File

@ -23,6 +23,7 @@ import static org.apache.hadoop.hbase.thrift.ThriftServerRunner.THRIFT_SPNEGO_PR
import java.io.IOException; import java.io.IOException;
import java.security.PrivilegedExceptionAction; import java.security.PrivilegedExceptionAction;
import java.util.Base64;
import javax.servlet.ServletException; import javax.servlet.ServletException;
import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletRequest;
@ -30,7 +31,6 @@ import javax.servlet.http.HttpServletResponse;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.security.SecurityUtil; import org.apache.hadoop.hbase.security.SecurityUtil;
import org.apache.hadoop.hbase.util.Base64;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.AuthorizationException; import org.apache.hadoop.security.authorize.AuthorizationException;
import org.apache.hadoop.security.authorize.ProxyUsers; import org.apache.hadoop.security.authorize.ProxyUsers;
@ -214,10 +214,10 @@ public class ThriftHttpServlet extends TServlet {
gssContext = manager.createContext(serverCreds); gssContext = manager.createContext(serverCreds);
// Get service ticket from the authorization header // Get service ticket from the authorization header
String serviceTicketBase64 = getAuthHeader(request); String serviceTicketBase64 = getAuthHeader(request);
byte[] inToken = Base64.decode(serviceTicketBase64); byte[] inToken = Base64.getDecoder().decode(serviceTicketBase64);
byte[] res = gssContext.acceptSecContext(inToken, 0, inToken.length); byte[] res = gssContext.acceptSecContext(inToken, 0, inToken.length);
if(res != null) { if(res != null) {
outToken = Base64.encodeBytes(res).replace("\n", ""); outToken = Base64.getEncoder().encodeToString(res).replace("\n", "");
} }
// Authenticate or deny based on its context completion // Authenticate or deny based on its context completion
if (!gssContext.isEstablished()) { if (!gssContext.isEstablished()) {