MINOR: Map `mkString` format updated to default java format

This is a minor change but it helps to improve the log readability.

Author: Kamal C <kamal.chandraprakash@gmail.com>

Reviewers: Ismael Juma <ismael@juma.me.uk>

Closes #2709 from Kamal15/util
This commit is contained in:
Kamal C 2017-03-30 13:17:09 +01:00 committed by Ismael Juma
parent c808e8955f
commit 43fb2df7a4
14 changed files with 30 additions and 59 deletions

View File

@ -73,7 +73,7 @@ public class CommonClientConfigs {
public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the <code>MetricReporter</code> interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics."; public static final String METRIC_REPORTER_CLASSES_DOC = "A list of classes to use as metrics reporters. Implementing the <code>MetricReporter</code> interface allows plugging in classes that will be notified of new metric creation. The JmxReporter is always included to register JMX statistics.";
public static final String SECURITY_PROTOCOL_CONFIG = "security.protocol"; public static final String SECURITY_PROTOCOL_CONFIG = "security.protocol";
public static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Valid values are: " + Utils.mkString(nonTestingSecurityProtocolNames(), ", ") + "."; public static final String SECURITY_PROTOCOL_DOC = "Protocol used to communicate with brokers. Valid values are: " + Utils.join(nonTestingSecurityProtocolNames(), ", ") + ".";
public static final String DEFAULT_SECURITY_PROTOCOL = "PLAINTEXT"; public static final String DEFAULT_SECURITY_PROTOCOL = "PLAINTEXT";
public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = "connections.max.idle.ms"; public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = "connections.max.idle.ms";

View File

@ -20,7 +20,6 @@ import org.apache.kafka.common.errors.UnsupportedVersionException;
import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.utils.Utils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
@ -88,8 +87,8 @@ public class CreateTopicsRequest extends AbstractRequest {
StringBuilder bld = new StringBuilder(); StringBuilder bld = new StringBuilder();
bld.append("(numPartitions=").append(numPartitions). bld.append("(numPartitions=").append(numPartitions).
append(", replicationFactor=").append(replicationFactor). append(", replicationFactor=").append(replicationFactor).
append(", replicasAssignments=").append(Utils.mkString(replicasAssignments)). append(", replicasAssignments=").append(replicasAssignments).
append(", configs=").append(Utils.mkString(configs)). append(", configs=").append(configs).
append(")"); append(")");
return bld.toString(); return bld.toString();
} }
@ -123,7 +122,7 @@ public class CreateTopicsRequest extends AbstractRequest {
public String toString() { public String toString() {
StringBuilder bld = new StringBuilder(); StringBuilder bld = new StringBuilder();
bld.append("(type=CreateTopicsRequest"). bld.append("(type=CreateTopicsRequest").
append(", topics=").append(Utils.mkString(topics)). append(", topics=").append(topics).
append(", timeout=").append(timeout). append(", timeout=").append(timeout).
append(", validateOnly=").append(validateOnly). append(", validateOnly=").append(validateOnly).
append(")"); append(")");

View File

@ -22,7 +22,6 @@ import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.utils.CollectionUtils; import org.apache.kafka.common.utils.CollectionUtils;
import org.apache.kafka.common.utils.Utils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
@ -68,7 +67,7 @@ public class DeleteRecordsRequest extends AbstractRequest {
StringBuilder builder = new StringBuilder(); StringBuilder builder = new StringBuilder();
builder.append("(type=DeleteRecordsRequest") builder.append("(type=DeleteRecordsRequest")
.append(", timeout=").append(timeout) .append(", timeout=").append(timeout)
.append(", partitionOffsets=(").append(Utils.mkString(partitionOffsets)) .append(", partitionOffsets=(").append(partitionOffsets)
.append("))"); .append("))");
return builder.toString(); return builder.toString();
} }

View File

@ -21,7 +21,6 @@ import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.record.MemoryRecords; import org.apache.kafka.common.record.MemoryRecords;
import org.apache.kafka.common.utils.Utils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
@ -151,7 +150,7 @@ public class FetchRequest extends AbstractRequest {
append(", maxWait=").append(maxWait). append(", maxWait=").append(maxWait).
append(", minBytes=").append(minBytes). append(", minBytes=").append(minBytes).
append(", maxBytes=").append(maxBytes). append(", maxBytes=").append(maxBytes).
append(", fetchData=").append(Utils.mkString(fetchData)). append(", fetchData=").append(fetchData).
append(")"); append(")");
return bld.toString(); return bld.toString();
} }

View File

@ -77,7 +77,7 @@ public class LeaderAndIsrRequest extends AbstractRequest {
bld.append("(type=LeaderAndIsRequest") bld.append("(type=LeaderAndIsRequest")
.append(", controllerId=").append(controllerId) .append(", controllerId=").append(controllerId)
.append(", controllerEpoch=").append(controllerEpoch) .append(", controllerEpoch=").append(controllerEpoch)
.append(", partitionStates=").append(Utils.mkString(partitionStates)) .append(", partitionStates=").append(partitionStates)
.append(", liveLeaders=(").append(Utils.join(liveLeaders, ", ")).append(")") .append(", liveLeaders=(").append(Utils.join(liveLeaders, ", ")).append(")")
.append(")"); .append(")");
return bld.toString(); return bld.toString();

View File

@ -22,7 +22,6 @@ import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.utils.CollectionUtils; import org.apache.kafka.common.utils.CollectionUtils;
import org.apache.kafka.common.utils.Utils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
@ -128,10 +127,10 @@ public class ListOffsetRequest extends AbstractRequest {
bld.append("(type=ListOffsetRequest") bld.append("(type=ListOffsetRequest")
.append(", replicaId=").append(replicaId); .append(", replicaId=").append(replicaId);
if (offsetData != null) { if (offsetData != null) {
bld.append(", offsetData=").append(Utils.mkString(offsetData)); bld.append(", offsetData=").append(offsetData);
} }
if (partitionTimestamps != null) { if (partitionTimestamps != null) {
bld.append(", partitionTimestamps=").append(Utils.mkString(partitionTimestamps)); bld.append(", partitionTimestamps=").append(partitionTimestamps);
} }
bld.append(", minVersion=").append(minVersion); bld.append(", minVersion=").append(minVersion);
bld.append(")"); bld.append(")");

View File

@ -23,7 +23,6 @@ import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Schema;
import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.protocol.types.Struct;
import org.apache.kafka.common.utils.CollectionUtils; import org.apache.kafka.common.utils.CollectionUtils;
import org.apache.kafka.common.utils.Utils;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
@ -149,7 +148,7 @@ public class OffsetCommitRequest extends AbstractRequest {
append(", memberId=").append(memberId). append(", memberId=").append(memberId).
append(", generationId=").append(generationId). append(", generationId=").append(generationId).
append(", retentionTime=").append(retentionTime). append(", retentionTime=").append(retentionTime).
append(", offsetData=").append(Utils.mkString(offsetData)). append(", offsetData=").append(offsetData).
append(")"); append(")");
return bld.toString(); return bld.toString();
} }

View File

@ -82,7 +82,7 @@ public class ProduceRequest extends AbstractRequest {
.append(", magic=").append(magic) .append(", magic=").append(magic)
.append(", acks=").append(acks) .append(", acks=").append(acks)
.append(", timeout=").append(timeout) .append(", timeout=").append(timeout)
.append(", partitionRecords=(").append(Utils.mkString(partitionRecords)) .append(", partitionRecords=(").append(partitionRecords)
.append("))"); .append("))");
return bld.toString(); return bld.toString();
} }

View File

@ -67,8 +67,8 @@ public class UpdateMetadataRequest extends AbstractRequest {
bld.append("(type: UpdateMetadataRequest="). bld.append("(type: UpdateMetadataRequest=").
append(", controllerId=").append(controllerId). append(", controllerId=").append(controllerId).
append(", controllerEpoch=").append(controllerEpoch). append(", controllerEpoch=").append(controllerEpoch).
append(", partitionStates=").append(Utils.mkString(partitionStates)). append(", partitionStates=").append(partitionStates).
append(", liveBrokers=").append(Utils.join(liveBrokers, " ,")). append(", liveBrokers=").append(Utils.join(liveBrokers, ", ")).
append(")"); append(")");
return bld.toString(); return bld.toString();
} }

View File

@ -352,43 +352,39 @@ public class Utils {
/** /**
* Create a string representation of an array joined by the given separator * Create a string representation of an array joined by the given separator
* @param strs The array of items * @param strs The array of items
* @param seperator The separator * @param separator The separator
* @return The string representation. * @return The string representation.
*/ */
public static <T> String join(T[] strs, String seperator) { public static <T> String join(T[] strs, String separator) {
return join(Arrays.asList(strs), seperator); return join(Arrays.asList(strs), separator);
} }
/** /**
* Create a string representation of a list joined by the given separator * Create a string representation of a list joined by the given separator
* @param list The list of items * @param list The list of items
* @param seperator The separator * @param separator The separator
* @return The string representation. * @return The string representation.
*/ */
public static <T> String join(Collection<T> list, String seperator) { public static <T> String join(Collection<T> list, String separator) {
StringBuilder sb = new StringBuilder(); StringBuilder sb = new StringBuilder();
Iterator<T> iter = list.iterator(); Iterator<T> iter = list.iterator();
while (iter.hasNext()) { while (iter.hasNext()) {
sb.append(iter.next()); sb.append(iter.next());
if (iter.hasNext()) if (iter.hasNext())
sb.append(seperator); sb.append(separator);
} }
return sb.toString(); return sb.toString();
} }
public static <K, V> String mkString(Map<K, V> map) {
return mkString(map, "{", "}", "=", " ,");
}
public static <K, V> String mkString(Map<K, V> map, String begin, String end, public static <K, V> String mkString(Map<K, V> map, String begin, String end,
String keyValueSeparator, String elementSeperator) { String keyValueSeparator, String elementSeparator) {
StringBuilder bld = new StringBuilder(); StringBuilder bld = new StringBuilder();
bld.append(begin); bld.append(begin);
String prefix = ""; String prefix = "";
for (Map.Entry<K, V> entry : map.entrySet()) { for (Map.Entry<K, V> entry : map.entrySet()) {
bld.append(prefix).append(entry.getKey()). bld.append(prefix).append(entry.getKey()).
append(keyValueSeparator).append(entry.getValue()); append(keyValueSeparator).append(entry.getValue());
prefix = elementSeperator; prefix = elementSeparator;
} }
bld.append(end); bld.append(end);
return bld.toString(); return bld.toString();
@ -439,7 +435,7 @@ public class Utils {
thread.setDaemon(daemon); thread.setDaemon(daemon);
thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
public void uncaughtException(Thread t, Throwable e) { public void uncaughtException(Thread t, Throwable e) {
log.error("Uncaught exception in thread '" + t.getName() + "':", e); log.error("Uncaught exception in thread '{}':", t.getName(), e);
} }
}); });
return thread; return thread;
@ -544,25 +540,6 @@ public class Utils {
return Arrays.asList(elems); return Arrays.asList(elems);
} }
/*
* Create a string from a collection
* @param coll the collection
* @param separator the separator
*/
public static <T> CharSequence mkString(Collection<T> coll, String separator) {
StringBuilder sb = new StringBuilder();
Iterator<T> iter = coll.iterator();
if (iter.hasNext()) {
sb.append(iter.next().toString());
while (iter.hasNext()) {
sb.append(separator);
sb.append(iter.next().toString());
}
}
return sb;
}
/** /**
* Recursively delete the given file/directory and any subfiles (if any exist) * Recursively delete the given file/directory and any subfiles (if any exist)
* *
@ -624,8 +601,8 @@ public class Utils {
} catch (IOException outer) { } catch (IOException outer) {
try { try {
Files.move(source, target, StandardCopyOption.REPLACE_EXISTING); Files.move(source, target, StandardCopyOption.REPLACE_EXISTING);
log.debug("Non-atomic move of " + source + " to " + target + " succeeded after atomic move failed due to " log.debug("Non-atomic move of {} to {} succeeded after atomic move failed due to {}", source, target,
+ outer.getMessage()); outer.getMessage());
} catch (IOException inner) { } catch (IOException inner) {
inner.addSuppressed(outer); inner.addSuppressed(outer);
throw inner; throw inner;
@ -663,7 +640,7 @@ public class Utils {
try { try {
closeable.close(); closeable.close();
} catch (Throwable t) { } catch (Throwable t) {
log.warn("Failed to close " + name, t); log.warn("Failed to close {}", name, t);
} }
} }
} }

View File

@ -736,7 +736,7 @@ public class StreamPartitionAssignor implements PartitionAssignor, Configurable
} else if (numPartitions != partitions) { } else if (numPartitions != partitions) {
final String[] topics = copartitionGroup.toArray(new String[copartitionGroup.size()]); final String[] topics = copartitionGroup.toArray(new String[copartitionGroup.size()]);
Arrays.sort(topics); Arrays.sort(topics);
throw new TopologyBuilderException(String.format("stream-thread [%s] Topics not co-partitioned: [%s]", threadName, Utils.mkString(Arrays.asList(topics), ","))); throw new TopologyBuilderException(String.format("stream-thread [%s] Topics not co-partitioned: [%s]", threadName, Utils.join(Arrays.asList(topics), ",")));
} }
} else if (allRepartitionTopicsNumPartitions.get(topic).numPartitions == NOT_AVAILABLE) { } else if (allRepartitionTopicsNumPartitions.get(topic).numPartitions == NOT_AVAILABLE) {
numPartitions = NOT_AVAILABLE; numPartitions = NOT_AVAILABLE;

View File

@ -83,7 +83,7 @@ public class StreamsConfigTest {
@Test @Test
public void defaultSerdeShouldBeConfigured() { public void defaultSerdeShouldBeConfigured() {
Map<String, Object> serializerConfigs = new HashMap<String, Object>(); Map<String, Object> serializerConfigs = new HashMap<>();
serializerConfigs.put("key.serializer.encoding", "UTF8"); serializerConfigs.put("key.serializer.encoding", "UTF8");
serializerConfigs.put("value.serializer.encoding", "UTF-16"); serializerConfigs.put("value.serializer.encoding", "UTF-16");
Serializer<String> serializer = Serdes.String().serializer(); Serializer<String> serializer = Serdes.String().serializer();
@ -103,7 +103,7 @@ public class StreamsConfigTest {
@Test @Test
public void shouldSupportMultipleBootstrapServers() { public void shouldSupportMultipleBootstrapServers() {
List<String> expectedBootstrapServers = Arrays.asList("broker1:9092", "broker2:9092"); List<String> expectedBootstrapServers = Arrays.asList("broker1:9092", "broker2:9092");
String bootstrapServersString = Utils.mkString(expectedBootstrapServers, ",").toString(); String bootstrapServersString = Utils.join(expectedBootstrapServers, ",");
Properties props = new Properties(); Properties props = new Properties();
props.put(StreamsConfig.APPLICATION_ID_CONFIG, "irrelevant"); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "irrelevant");
props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServersString); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, bootstrapServersString);

View File

@ -39,7 +39,6 @@ import org.apache.kafka.common.serialization.ByteArraySerializer;
import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.utils.Exit; import org.apache.kafka.common.utils.Exit;
import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.slf4j.Logger; import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
@ -208,7 +207,7 @@ public class ClientCompatibilityTest {
@Override @Override
public String toString() { public String toString() {
return Utils.mkString(result); return result.toString();
} }
} }