Skip to content

Commit

Permalink
HBASE-27234 Clean up error-prone warnings in hbase-examples
Browse files Browse the repository at this point in the history
  • Loading branch information
apurtell committed Jul 25, 2022
1 parent 0ae42dd commit 2198d96
Show file tree
Hide file tree
Showing 15 changed files with 98 additions and 77 deletions.
19 changes: 17 additions & 2 deletions hbase-common/src/main/java/org/apache/hadoop/hbase/util/Bytes.java
Original file line number Diff line number Diff line change
Expand Up @@ -485,8 +485,7 @@ private static byte[] readBytes(ByteBuffer buf) {
}

/**
* @param b Presumed UTF-8 encoded byte array.
* @return String made from <code>b</code>
* Returns String made from <code>b</code>
*/
public static String toString(final byte[] b) {
if (b == null) {
Expand All @@ -495,6 +494,21 @@ public static String toString(final byte[] b) {
return toString(b, 0, b.length);
}

/**
* Returns String made from <code>b</code>
*/
public static String toString(ByteBuffer buf) {
if (buf == null) {
return null;
}
if (!buf.hasArray()) {
buf = ByteBuffer.wrap(buf.array(), buf.arrayOffset(), buf.remaining());
return toString(buf.array());
} else {
return toString(buf.array(), buf.arrayOffset(), buf.remaining());
}
}

/**
* Joins two byte arrays together using a separator.
* @param b1 The first byte array.
Expand Down Expand Up @@ -2432,4 +2446,5 @@ public static int findCommonPrefix(byte[] left, byte[] right, int leftLength, in
}
return result;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@

import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Iterator;
import java.util.List;
import java.util.concurrent.ExecutionException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hbase.HBaseConfiguration;
Expand All @@ -34,6 +36,7 @@
import org.apache.yetus.audience.InterfaceAudience;

import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
import org.apache.hbase.thirdparty.com.google.common.base.Splitter;
import org.apache.hbase.thirdparty.com.google.common.base.Throwables;
import org.apache.hbase.thirdparty.io.netty.bootstrap.ServerBootstrap;
import org.apache.hbase.thirdparty.io.netty.buffer.ByteBuf;
Expand Down Expand Up @@ -152,12 +155,20 @@ private void write(ChannelHandlerContext ctx, HttpResponseStatus status, String
}

private Params parse(FullHttpRequest req) {
String[] components = new QueryStringDecoder(req.uri()).path().split("/");
Preconditions.checkArgument(components.length == 4, "Unrecognized uri: %s", req.uri());
List<String> components =
Splitter.on('/').splitToList(new QueryStringDecoder(req.uri()).path());
Preconditions.checkArgument(components.size() == 4, "Unrecognized uri: %s", req.uri());
Iterator<String> i = components.iterator();
// path is start with '/' so split will give an empty component
String[] cfAndCq = components[3].split(":");
Preconditions.checkArgument(cfAndCq.length == 2, "Unrecognized uri: %s", req.uri());
return new Params(components[1], components[2], cfAndCq[0], cfAndCq[1]);
i.next();
String table = i.next();
String row = i.next();
List<String> cfAndCq = Splitter.on(':').splitToList(i.next());
Preconditions.checkArgument(cfAndCq.size() == 2, "Unrecognized uri: %s", req.uri());
i = cfAndCq.iterator();
String family = i.next();
String qualifier = i.next();
return new Params(table, row, family, qualifier);
}

private void get(ChannelHandlerContext ctx, FullHttpRequest req) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@
import java.util.concurrent.TimeUnit;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.Cell.Type;
import org.apache.hadoop.hbase.CellBuilderFactory;
import org.apache.hadoop.hbase.CellBuilderType;
import org.apache.hadoop.hbase.TableName;
Expand Down Expand Up @@ -240,7 +239,7 @@ public Boolean call() throws Exception {
byte[] rk = Bytes.toBytes(ThreadLocalRandom.current().nextLong());
Put p = new Put(rk);
p.add(CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY).setRow(rk).setFamily(FAMILY)
.setQualifier(QUAL).setTimestamp(p.getTimestamp()).setType(Type.Put).setValue(value)
.setQualifier(QUAL).setTimestamp(p.getTimestamp()).setType(Cell.Type.Put).setValue(value)
.build());
t.put(p);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
import org.apache.hadoop.hbase.client.Delete;
import org.apache.hadoop.hbase.client.Mutation;
import org.apache.hadoop.hbase.client.Scan;
Expand All @@ -52,7 +51,6 @@
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest;
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType;
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse;
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.Builder;
import org.apache.hadoop.hbase.shaded.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService;
import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;

Expand Down Expand Up @@ -157,7 +155,7 @@ public void delete(RpcController controller, BulkDeleteRequest request,
}
OperationStatus[] opStatus = region.batchMutate(deleteArr);
for (i = 0; i < opStatus.length; i++) {
if (opStatus[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
if (opStatus[i].getOperationStatusCode() != HConstants.OperationStatusCode.SUCCESS) {
break;
}
totalRowsDeleted++;
Expand All @@ -183,7 +181,7 @@ public void delete(RpcController controller, BulkDeleteRequest request,
}
}
}
Builder responseBuilder = BulkDeleteResponse.newBuilder();
BulkDeleteResponse.Builder responseBuilder = BulkDeleteResponse.newBuilder();
responseBuilder.setRowsDeleted(totalRowsDeleted);
if (deleteType == DeleteType.VERSION) {
responseBuilder.setVersionsDeleted(totalVersionsDeleted);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,8 @@ private void performCostlyOperation() {
// simulate the operation by sleeping.
Thread.sleep(ThreadLocalRandom.current().nextLong(100));
} catch (InterruptedException ignore) {
// Restore the interrupt status
Thread.currentThread().interrupt();
}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import org.apache.commons.io.IOUtils;
import org.apache.commons.lang3.mutable.MutableLong;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.CoprocessorEnvironment;
Expand Down Expand Up @@ -97,10 +99,7 @@ public void getRowCount(RpcController controller, CountRequest request,
CoprocessorRpcUtils.setControllerException(controller, ioe);
} finally {
if (scanner != null) {
try {
scanner.close();
} catch (IOException ignored) {
}
IOUtils.closeQuietly(scanner);
}
}
done.run(response);
Expand All @@ -118,24 +117,18 @@ public void getKeyValueCount(RpcController controller, CountRequest request,
scanner = env.getRegion().getScanner(new Scan());
List<Cell> results = new ArrayList<>();
boolean hasMore = false;
long count = 0;
MutableLong count = new MutableLong();
do {
hasMore = scanner.next(results);
for (Cell kv : results) {
count++;
}
results.forEach((r) -> count.increment());
results.clear();
} while (hasMore);

response = CountResponse.newBuilder().setCount(count).build();
response = CountResponse.newBuilder().setCount(count.longValue()).build();
} catch (IOException ioe) {
CoprocessorRpcUtils.setControllerException(controller, ioe);
} finally {
if (scanner != null) {
try {
scanner.close();
} catch (IOException ignored) {
}
IOUtils.closeQuietly(scanner);
}
}
done.run(response);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,7 @@ public static Job configureJob(Configuration conf, String[] args) throws IOExcep
return job;
}

@Override
public int run(String[] args) throws Exception {
Configuration conf = HBaseConfiguration.create(getConf());
if (args.length < 3) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,8 @@
package org.apache.hadoop.hbase.mapreduce;

import java.io.IOException;
import java.util.Iterator;
import java.util.List;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.Path;
Expand All @@ -37,6 +39,8 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import org.apache.hbase.thirdparty.com.google.common.base.Splitter;

/**
* Sample Uploader MapReduce
* <p>
Expand Down Expand Up @@ -80,16 +84,16 @@ public void map(LongWritable key, Text line, Context context) throws IOException
// Each line is comma-delimited; row,family,qualifier,value

// Split CSV line
String[] values = line.toString().split(",");
if (values.length != 4) {
List<String> values = Splitter.on(',').splitToList(line.toString());
if (values.size() != 4) {
return;
}

Iterator<String> i = values.iterator();
// Extract each value
byte[] row = Bytes.toBytes(values[0]);
byte[] family = Bytes.toBytes(values[1]);
byte[] qualifier = Bytes.toBytes(values[2]);
byte[] value = Bytes.toBytes(values[3]);
byte[] row = Bytes.toBytes(i.next());
byte[] family = Bytes.toBytes(i.next());
byte[] qualifier = Bytes.toBytes(i.next());
byte[] value = Bytes.toBytes(i.next());

// Create Put
Put put = new Put(row);
Expand Down Expand Up @@ -136,6 +140,7 @@ public static Job configureJob(Configuration conf, String[] args) throws IOExcep
* @param otherArgs The command line parameters after ToolRunner handles standard.
* @throws Exception When running the job fails.
*/
@Override
public int run(String[] otherArgs) throws Exception {
if (otherArgs.length != 2) {
System.err.println("Wrong number of arguments: " + otherArgs.length);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.nio.charset.StandardCharsets;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
Expand Down Expand Up @@ -86,8 +87,8 @@ Map<String, char[]> readPasswordDB(Configuration conf) throws IOException {
}

Map<String, char[]> passwordDb = new HashMap<>();
try (FSDataInputStream fdis = fs.open(passwordFile);
BufferedReader reader = new BufferedReader(new InputStreamReader(fdis))) {
try (FSDataInputStream fdis = fs.open(passwordFile); BufferedReader reader =
new BufferedReader(new InputStreamReader(fdis, StandardCharsets.UTF_8))) {
String line = null;
int offset = 0;
while ((line = reader.readLine()) != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,15 +41,12 @@
import org.apache.thrift.transport.TSocket;
import org.apache.thrift.transport.TTransport;
import org.apache.yetus.audience.InterfaceAudience;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* See the instructions under hbase-examples/README.txt
*/
@InterfaceAudience.Private
public class DemoClient {
private static final Logger LOG = LoggerFactory.getLogger(DemoClient.class);

static protected int port;
static protected String host;
Expand Down Expand Up @@ -128,15 +125,15 @@ private void run() throws Exception {
System.out.println("scanning tables...");

for (ByteBuffer name : client.getTableNames()) {
System.out.println(" found: " + ClientUtils.utf8(name.array()));
System.out.println(" found: " + ClientUtils.utf8(name));

if (name.equals(demoTable) || name.equals(disabledTable)) {
if (client.isTableEnabled(name)) {
System.out.println(" disabling table: " + ClientUtils.utf8(name.array()));
System.out.println(" disabling table: " + ClientUtils.utf8(name));
client.disableTable(name);
}

System.out.println(" deleting table: " + ClientUtils.utf8(name.array()));
System.out.println(" deleting table: " + ClientUtils.utf8(name));
client.deleteTable(name);
}
}
Expand Down Expand Up @@ -324,7 +321,7 @@ private void run() throws Exception {
columnNames.clear();

for (ColumnDescriptor col2 : client.getColumnDescriptors(demoTable).values()) {
System.out.println("column with name: " + new String(col2.name.array()));
System.out.println("column with name: " + ClientUtils.utf8(col2.name));
System.out.println(col2.toString());

columnNames.add(col2.name);
Expand Down Expand Up @@ -356,7 +353,7 @@ private void printVersions(ByteBuffer row, List<TCell> versions) {
rowStr.append("; ");
}

System.out.println("row: " + ClientUtils.utf8(row.array()) + ", values: " + rowStr);
System.out.println("row: " + ClientUtils.utf8(row) + ", values: " + rowStr);
}

private void printRow(TRowResult rowResult) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import java.util.Base64;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import javax.security.auth.Subject;
Expand All @@ -36,8 +35,6 @@
import org.apache.hadoop.hbase.thrift.generated.AlreadyExists;
import org.apache.hadoop.hbase.thrift.generated.ColumnDescriptor;
import org.apache.hadoop.hbase.thrift.generated.Hbase;
import org.apache.hadoop.hbase.thrift.generated.TCell;
import org.apache.hadoop.hbase.thrift.generated.TRowResult;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.ClientUtils;
import org.apache.thrift.protocol.TBinaryProtocol;
Expand Down Expand Up @@ -129,13 +126,13 @@ private void run() throws Exception {
//
System.out.println("scanning tables...");
for (ByteBuffer name : refresh(client, httpClient).getTableNames()) {
System.out.println(" found: " + ClientUtils.utf8(name.array()));
if (ClientUtils.utf8(name.array()).equals(ClientUtils.utf8(t))) {
System.out.println(" found: " + ClientUtils.utf8(name));
if (ClientUtils.utf8(name).equals(ClientUtils.utf8(t))) {
if (refresh(client, httpClient).isTableEnabled(name)) {
System.out.println(" disabling table: " + ClientUtils.utf8(name.array()));
System.out.println(" disabling table: " + ClientUtils.utf8(name));
refresh(client, httpClient).disableTable(name);
}
System.out.println(" deleting table: " + ClientUtils.utf8(name.array()));
System.out.println(" deleting table: " + ClientUtils.utf8(name));
refresh(client, httpClient).deleteTable(name);
}
}
Expand Down Expand Up @@ -167,8 +164,8 @@ private void run() throws Exception {
Map<ByteBuffer, ColumnDescriptor> columnMap =
refresh(client, httpClient).getColumnDescriptors(ByteBuffer.wrap(t));
for (ColumnDescriptor col2 : columnMap.values()) {
System.out.println(
" column: " + ClientUtils.utf8(col2.name.array()) + ", maxVer: " + col2.maxVersions);
System.out
.println(" column: " + ClientUtils.utf8(col2.name) + ", maxVer: " + col2.maxVersions);
}

transport.close();
Expand Down Expand Up @@ -205,26 +202,13 @@ private String generateTicket() throws GSSException {
context.requestInteg(true);

final byte[] outToken = context.initSecContext(new byte[0], 0, 0);
StringBuffer outputBuffer = new StringBuffer();
StringBuilder outputBuffer = new StringBuilder();
outputBuffer.append("Negotiate ");
outputBuffer.append(Bytes.toString(Base64.getEncoder().encode(outToken)));
System.out.print("Ticket is: " + outputBuffer);
return outputBuffer.toString();
}

private void printVersions(ByteBuffer row, List<TCell> versions) {
StringBuilder rowStr = new StringBuilder();
for (TCell cell : versions) {
rowStr.append(ClientUtils.utf8(cell.value.array()));
rowStr.append("; ");
}
System.out.println("row: " + ClientUtils.utf8(row.array()) + ", values: " + rowStr);
}

private void printRow(TRowResult rowResult) {
ClientUtils.printRow(rowResult);
}

static Subject getSubject() throws Exception {
if (!secure) {
return new Subject();
Expand Down
Loading

0 comments on commit 2198d96

Please sign in to comment.