From 66f29a26f3a6f71f22515d339564cfa836e39dd2 Mon Sep 17 00:00:00 2001 From: David Mollitor Date: Wed, 21 Jul 2021 21:31:38 -0400 Subject: [PATCH] Remove use of String format in favor of SLF4J anchor --- src/main/java/com/uber/rss/StreamServer.java | 18 +++++++------- .../rss/clients/BlockingQueueReadClient.java | 5 ++-- .../rss/clients/BusyStatusSocketClient.java | 2 +- .../java/com/uber/rss/clients/ClientBase.java | 10 ++++---- .../clients/DataBlockSocketReadClient.java | 6 ++--- .../rss/clients/HeartbeatSocketClient.java | 2 +- .../clients/MultiServerAsyncWriteClient.java | 21 ++++++++-------- .../clients/MultiServerHeartbeatClient.java | 4 ++-- .../clients/MultiServerSocketReadClient.java | 4 ++-- .../clients/MultiServerSyncWriteClient.java | 6 ++--- .../com/uber/rss/clients/NotifyClient.java | 6 ++--- .../PooledShuffleDataSyncWriteClient.java | 4 ++-- .../rss/clients/PooledWriteClientFactory.java | 10 ++++---- .../rss/clients/RecordSocketReadClient.java | 8 +++---- .../com/uber/rss/clients/RegistryClient.java | 4 ++-- .../rss/clients/ReplicatedReadClient.java | 8 +++---- .../clients/RetriableSocketReadClient.java | 6 ++--- .../ServerBusyRetriableWriteClient.java | 4 ++-- .../ServerIdAwareSocketReadClient.java | 2 +- .../clients/ServerIdAwareSyncWriteClient.java | 4 ++-- .../clients/ShuffleDataSocketReadClient.java | 8 +++---- .../decoders/StreamServerVersionDecoder.java | 12 ++++------ .../rss/execution/LocalFileStateStore.java | 8 +++---- .../LocalFileStateStoreIterator.java | 10 ++++---- .../uber/rss/execution/ShuffleExecutor.java | 4 ++-- .../uber/rss/handlers/ChannelIdleCheck.java | 2 +- .../rss/metadata/ServiceRegistryUtils.java | 2 +- .../java/com/uber/rss/metrics/M3Stats.java | 2 +- .../rss/metrics/ScheduledMetricCollector.java | 2 +- .../rss/tools/StreamServerStressTool.java | 24 +++++++++---------- .../java/com/uber/rss/util/FileUtils.java | 2 +- 31 files changed, 102 insertions(+), 108 deletions(-) diff --git a/src/main/java/com/uber/rss/StreamServer.java b/src/main/java/com/uber/rss/StreamServer.java index 43f000d..3a354fa 100644 --- a/src/main/java/com/uber/rss/StreamServer.java +++ b/src/main/java/com/uber/rss/StreamServer.java @@ -159,10 +159,10 @@ private void createServiceRegistry(StreamServerConfig serverConfig) { } private Pair bindPort(ServerBootstrap bootstrap, int port) throws InterruptedException, BindException { - logger.info(String.format("Binding to specified port: %s", port)); + logger.info("Binding to specified port: {}", port); Channel channel = bootstrap.bind(port).sync().channel(); InetSocketAddress localAddress = (InetSocketAddress)channel.localAddress(); - logger.info(String.format("Bound to local address: %s", localAddress)); + logger.info("Bound to local address: {}", localAddress); return Pair.of(channel, localAddress.getPort()); } @@ -186,7 +186,7 @@ public void initChannel(final SocketChannel ch) { } public void run() throws InterruptedException, BindException { - logger.info(String.format("Number of opened files: %s", SystemUtils.getFileDescriptorCount())); + logger.info("Number of opened files: {}", SystemUtils.getFileDescriptorCount()); String serverId = getServerId(); @@ -213,12 +213,12 @@ public void run() throws InterruptedException, BindException { Pair channelAndPort = bindPort(streamServerBootstrap, serverConfig.getShufflePort()); channels.add(channelAndPort.getKey()); shufflePort = channelAndPort.getValue(); - logger.info(String.format("ShuffleServer: %s:%s", hostName, shufflePort)); + logger.info("ShuffleServer: {}:{}", hostName, shufflePort); if (this.serviceRegistry == null && serverConfig.getServiceRegistryType(). equalsIgnoreCase(ServiceRegistry.TYPE_STANDALONE)) { - logger.info(String.format("Creating registry client connecting to local stream server: %s:%s", - hostName, shufflePort)); + logger.info("Creating registry client connecting to local stream server: {}:{}", + hostName, shufflePort); this.serviceRegistry = new StandaloneServiceRegistryClient(this.hostName, shufflePort, serverConfig.getNetworkTimeout(), "streamServer"); } @@ -299,7 +299,7 @@ public void shutdown(boolean wait) { try { c.close(); } catch (Throwable e) { - logger.warn(String.format("Unable to shutdown channel %s:", c), e); + logger.warn("Unable to shutdown channel {}", c, e); exceptions.add(e); } } @@ -365,8 +365,8 @@ private static Thread addShutdownHook(StreamServer server) { public static void main(String[] args) throws Exception { StreamServerConfig serverConfig = StreamServerConfig.buildFromArgs(args); - logger.info(String.format("Starting server (version: %s, revision: %s) with config: %s", - RssBuildInfo.Version, RssBuildInfo.Revision, serverConfig)); + logger.info("Starting server (version: {}, revision: {}) with config: {}", + RssBuildInfo.Version, RssBuildInfo.Revision, serverConfig); StreamServer server = new StreamServer(serverConfig); server.run(); addShutdownHook(server); diff --git a/src/main/java/com/uber/rss/clients/BlockingQueueReadClient.java b/src/main/java/com/uber/rss/clients/BlockingQueueReadClient.java index 41c6613..a28ed5e 100644 --- a/src/main/java/com/uber/rss/clients/BlockingQueueReadClient.java +++ b/src/main/java/com/uber/rss/clients/BlockingQueueReadClient.java @@ -43,7 +43,7 @@ public BlockingQueueReadClient(BlockingSingleServerReadClient delegate, int queu this.delegate = delegate; this.recordQueue = new ArrayBlockingQueue<>(queueSize); - logger.info(String.format("Created blocking queue with size: %s", queueSize)); + logger.info("Created blocking queue with size: {}", queueSize); this.maxBlockingMillis = maxBlockingMillis; } @@ -65,8 +65,7 @@ record = delegate.readRecord(); recordQueue.put(new EofRecordKeyValuePair()); } catch (Throwable ex) { M3Stats.addException(ex, this.getClass().getSimpleName()); - String logStr = String.format("Failed to read record, %s", delegate.toString()); - logger.warn(logStr, ex); + logger.warn("Failed to read record, {}", delegate, ex); recordQueue.clear(); recordQueue.add(new FailedFetchRecordKeyValuePair(ex)); } diff --git a/src/main/java/com/uber/rss/clients/BusyStatusSocketClient.java b/src/main/java/com/uber/rss/clients/BusyStatusSocketClient.java index 0276cf1..e5d3a5a 100644 --- a/src/main/java/com/uber/rss/clients/BusyStatusSocketClient.java +++ b/src/main/java/com/uber/rss/clients/BusyStatusSocketClient.java @@ -36,7 +36,7 @@ public BusyStatusSocketClient(String host, int port, int timeoutMillis, String u public GetBusyStatusResponse getBusyStatus() { if (socket == null) { - logger.debug(String.format("Connecting to server to get busy status: %s", connectionInfo)); + logger.debug("Connecting to server to get busy status: {}", connectionInfo); connectSocket(); write(MessageConstants.UPLOAD_UPLINK_MAGIC_BYTE); diff --git a/src/main/java/com/uber/rss/clients/ClientBase.java b/src/main/java/com/uber/rss/clients/ClientBase.java index 8b7045e..6ff480a 100644 --- a/src/main/java/com/uber/rss/clients/ClientBase.java +++ b/src/main/java/com/uber/rss/clients/ClientBase.java @@ -82,7 +82,7 @@ public ClientBase(String host, int port, int timeoutMillis) { this.port = port; this.timeoutMillis = timeoutMillis; this.connectionInfo = String.format("%s %s [%s -> %s:%s]", this.getClass().getSimpleName(), internalClientId, NetworkUtils.getLocalHostName(), host, port); - logger.debug(String.format("Created instance (timeout: %s millis): %s", timeoutMillis, this)); + logger.debug("Created instance (timeout: {} millis): {}", timeoutMillis, this); } @Override @@ -148,7 +148,7 @@ protected void connectSocket() { while (System.currentTimeMillis() - startTime <= timeoutMillis) { ClientConnectMetrics metrics = metricGroupContainer.getMetricGroup(getClientConnectMetricsKey()); if (triedTimes >= 1) { - logger.info(String.format("Retrying connect to %s:%s, total retrying times: %s, elapsed milliseconds: %s", host, port, triedTimes, System.currentTimeMillis() - startTime)); + logger.info("Retrying connect to {}:{} total retrying times: {}, elapsed milliseconds: {}", host, port, triedTimes, System.currentTimeMillis() - startTime); metrics.getSocketConnectRetries().update(triedTimes); } triedTimes++; @@ -167,7 +167,7 @@ protected void connectSocket() { M3Stats.addException(socketException, this.getClass().getSimpleName()); socket = null; lastException = socketException; - logger.info(String.format("Failed to connect to %s:%s, %s", host, port, ExceptionUtils.getSimpleMessage(socketException))); + logger.info("Failed to connect to {}:{}", host, port, socketException); long elapsedTime = System.currentTimeMillis() - startTime; if (elapsedTime < timeoutMillis) { @@ -236,7 +236,7 @@ protected void writeMessageLengthAndContent(BaseMessage msg) { } protected void writeControlMessageNotWaitResponseStatus(BaseMessage msg) { - logger.debug(String.format("Writing control message: %s, connection: %s", msg, connectionInfo)); + logger.debug("Writing control message: {}, connection: {}", msg, connectionInfo); try { outputStream.write(ByteBufUtils.convertIntToBytes(msg.getMessageType())); } catch (IOException e) { @@ -252,7 +252,7 @@ protected void writeControlMessageAndWaitResponseStatus(BaseMessage msg) { writeControlMessageNotWaitResponseStatus(msg); readResponseStatus(); - logger.debug(String.format("Got OK response for control message: %s, connection: %s", msg, connectionInfo)); + logger.debug("Got OK response for control message: {}, connection: {}", msg, connectionInfo); } private int readStatus() { diff --git a/src/main/java/com/uber/rss/clients/DataBlockSocketReadClient.java b/src/main/java/com/uber/rss/clients/DataBlockSocketReadClient.java index f40a39a..de481c0 100644 --- a/src/main/java/com/uber/rss/clients/DataBlockSocketReadClient.java +++ b/src/main/java/com/uber/rss/clients/DataBlockSocketReadClient.java @@ -130,7 +130,7 @@ private ConnectDownloadResponse connectImpl() { throw new RssShuffleCorruptedException("Shuffle data corrupted for: " + appShufflePartitionId, ex); } catch (RssMissingShuffleWriteConfigException | RssShuffleStageNotStartedException ex) { exceptionWrapper.setException(ex); - logger.warn(String.format("Did not find data in server side, server may not run fast enough to get data from client or server hits some issue, %s", appShufflePartitionId), ex); + logger.warn("Did not find data in server side, server may not run fast enough to get data from client or server hits some issue, {}", appShufflePartitionId, ex); return null; } }); @@ -203,7 +203,7 @@ public GetDataAvailabilityResponse waitDataAvailable() { if (getDataAvailabilityRetryResult != null && getDataAvailabilityRetryResult.getMapTaskCommitStatus() != null) { MapTaskCommitStatus mapTaskCommitStatus = getDataAvailabilityRetryResult.getMapTaskCommitStatus(); if (mapTaskCommitStatus.getTaskAttemptIds().isEmpty()) { - taskAttemptIdInfo = String.format("no task attempt committed"); + taskAttemptIdInfo = "no task attempt committed"; } else { List taskAttemptIds = mapTaskCommitStatus.getTaskAttemptIds().values().stream().collect(Collectors.toList()); Collections.sort(taskAttemptIds); @@ -266,7 +266,7 @@ public void close() { super.close(); closeMetrics(); } catch (Throwable ex) { - logger.warn(String.format("Failed to close read client %s", this), ex); + logger.warn("Failed to close read client {}", this, ex); } } diff --git a/src/main/java/com/uber/rss/clients/HeartbeatSocketClient.java b/src/main/java/com/uber/rss/clients/HeartbeatSocketClient.java index 92c1911..4df9bfc 100644 --- a/src/main/java/com/uber/rss/clients/HeartbeatSocketClient.java +++ b/src/main/java/com/uber/rss/clients/HeartbeatSocketClient.java @@ -41,7 +41,7 @@ public HeartbeatSocketClient(String host, int port, int timeoutMillis, String us public void sendHeartbeat() { if (socket == null) { - logger.debug(String.format("Connecting to server for heartbeat: %s", connectionInfo)); + logger.debug("Connecting to server for heartbeat: {}", connectionInfo); connectSocket(); write(MessageConstants.UPLOAD_UPLINK_MAGIC_BYTE); diff --git a/src/main/java/com/uber/rss/clients/MultiServerAsyncWriteClient.java b/src/main/java/com/uber/rss/clients/MultiServerAsyncWriteClient.java index f3521c6..5eb65b3 100644 --- a/src/main/java/com/uber/rss/clients/MultiServerAsyncWriteClient.java +++ b/src/main/java/com/uber/rss/clients/MultiServerAsyncWriteClient.java @@ -25,7 +25,6 @@ import com.uber.rss.metrics.M3Stats; import com.uber.rss.metrics.WriteClientMetrics; import com.uber.rss.metrics.WriteClientMetricsKey; -import com.uber.rss.util.ExceptionUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -119,7 +118,7 @@ public MultiServerAsyncWriteClient(Collection servers, i this.servers.size())); } - logger.info(String.format("Created %s, threads: %s, queue size: %s", this.getClass().getSimpleName(), numThreads, writeQueueSize)); + logger.info("Created {}, threads: {}, queue size: {}", this.getClass().getSimpleName(), numThreads, writeQueueSize); } @Override @@ -140,7 +139,7 @@ public void connect() { for (int i = 0; i < threads.length; i++) { final int threadIndex = i; Thread thread = new Thread(() -> { - logger.info(String.format("Record Thread %s started", threadIndex)); + logger.info("Record Thread {} started", threadIndex); BlockingQueue recordQueue = recordQueues[threadIndex]; try { // TODO optimize the max wait time for poll @@ -163,7 +162,7 @@ public void connect() { } } } catch (Throwable e) { - logger.warn(String.format("Record Thread %s got exception, %s", threadIndex, ExceptionUtils.getSimpleMessage(e)), e); + logger.warn("Record Thread {} got exception", threadIndex, e); M3Stats.addException(e, this.getClass().getSimpleName()); exceptions.add(e); } @@ -172,7 +171,7 @@ public void connect() { exceptions.add(new RssQueueNotReadyException(String.format("Record queue %s has %s remaining records not sent out", threadIndex, remainingRecords))); } recordQueue.clear(); - logger.info(String.format("Record Thread %s finished, remaining records: %s", threadIndex, remainingRecords)); + logger.info("Record Thread {} finished, remaining records: {}", threadIndex, remainingRecords); }); thread.setName("Record Thread " + i); threads[threadIndex] = thread; @@ -219,7 +218,7 @@ public void writeDataBlock(int partition, ByteBuffer value) { long currentTime = System.currentTimeMillis(); if (currentTime - lastLogTime > logInterval) { for (int i = 0; i < recordQueues.length; i++) { - logger.info(String.format("Record queue %s size: %s", i, recordQueues[i].size())); + logger.info("Record queue {} size: {}", i, recordQueues[i].size()); } lastLogTime = currentTime; } @@ -246,13 +245,13 @@ public void finishUpload() { long underlyingClientFinishUploadTime = System.nanoTime() - underlyingClientFinishUploadStartTime; - logger.info(String.format("WriteClientTime (%s), queue insert seconds: %s, queue poll seconds: %s, socket seconds: %s, stop thread seconds: %s, finish upload seconds: %s", + logger.info("WriteClientTime ({}), queue insert seconds: {}, queue poll seconds: {}, socket seconds: {}, stop thread seconds: {}, finish upload seconds: {}", currentAppTaskAttemptId, TimeUnit.NANOSECONDS.toSeconds(queueInsertTime.get()), TimeUnit.NANOSECONDS.toSeconds(queuePollTime.get()), TimeUnit.NANOSECONDS.toSeconds(socketTime.get()), TimeUnit.NANOSECONDS.toSeconds(stopThreadTime), - TimeUnit.NANOSECONDS.toSeconds(underlyingClientFinishUploadTime))); + TimeUnit.NANOSECONDS.toSeconds(underlyingClientFinishUploadTime)); } finally { stopwatch.stop(); } @@ -308,7 +307,7 @@ private void connectSingleClient(ServerConnectionInfo server) { private void closeClient(ReplicatedWriteClient client) { try { if (client != null) { - logger.debug(String.format("Closing client: %s", client)); + logger.debug("Closing client: {}", client); client.close(); } } catch (Throwable ex) { @@ -327,7 +326,7 @@ private void stopThreads() { if (!inserted) { throw new RssQueueNotReadyException(String.format("stopThreads: Record queue has no space available after waiting %s millis", networkTimeoutMillis)); } - logger.debug(String.format("Inserted stop marker to record queue %s", i)); + logger.debug("Inserted stop marker to record queue {}", i); } catch (InterruptedException e) { throw new RssException("Interrupted when inserting stop marker to record queue", e); } @@ -370,7 +369,7 @@ private void closeMetrics() { metrics.close(); } catch (Throwable e) { M3Stats.addException(e, this.getClass().getSimpleName()); - logger.warn(String.format("Failed to close metrics: %s", this), e); + logger.warn("Failed to close metrics: {}", this, e); } } diff --git a/src/main/java/com/uber/rss/clients/MultiServerHeartbeatClient.java b/src/main/java/com/uber/rss/clients/MultiServerHeartbeatClient.java index af98f31..ff498f1 100644 --- a/src/main/java/com/uber/rss/clients/MultiServerHeartbeatClient.java +++ b/src/main/java/com/uber/rss/clients/MultiServerHeartbeatClient.java @@ -125,7 +125,7 @@ public void sendHeartbeats() { try { sendHeartbeat(serverDetail); } catch (Throwable ex) { - logger.warn(String.format("Failed to send RSS heartbeat to %s", serverDetail), ex); + logger.warn("Failed to send RSS heartbeat to {}", serverDetail, ex); } } } @@ -138,7 +138,7 @@ private void sendHeartbeat(ServerDetail serverDetail) { client.sendHeartbeat(); logger.info("Sent RSS heartbeat to {}, duration millis: {}", serverDetail, System.currentTimeMillis() - startTime); } catch (Throwable ex) { - logger.warn(String.format("Failed to send RSS heartbeat to %s", serverDetail), ex); + logger.warn("Failed to send RSS heartbeat to {}", serverDetail, ex); if (serverConnectionRefresher != null) { ServerDetail refreshedServerDetail = serverConnectionRefresher.refreshConnection(serverDetail); if (refreshedServerDetail != null) { diff --git a/src/main/java/com/uber/rss/clients/MultiServerSocketReadClient.java b/src/main/java/com/uber/rss/clients/MultiServerSocketReadClient.java index 1f01571..b25f0ba 100644 --- a/src/main/java/com/uber/rss/clients/MultiServerSocketReadClient.java +++ b/src/main/java/com/uber/rss/clients/MultiServerSocketReadClient.java @@ -138,7 +138,7 @@ private void connectAndInitializeClient() { } ServerReplicationGroup serverReplicationGroup = servers.get(nextClientIndex); - logger.info(String.format("Fetching data from server: %s (%s out of %s), partition: %s", serverReplicationGroup, nextClientIndex + 1, servers.size(), appShufflePartitionId)); + logger.info("Fetching data from server: {} ({} out of {}), partition: {}", serverReplicationGroup, nextClientIndex + 1, servers.size(), appShufflePartitionId); ExceptionWrapper exceptionWrapper = new ExceptionWrapper<>(); String failMsg = String.format("Failed to connect to server: %s, partition: %s", serverReplicationGroup, appShufflePartitionId); @@ -184,7 +184,7 @@ private void closeClient(ReplicatedReadClient client) { client.close(); } } catch (Throwable ex) { - logger.warn(String.format("Failed to close client %s", client)); + logger.warn("Failed to close client {}", client, ex); } } } diff --git a/src/main/java/com/uber/rss/clients/MultiServerSyncWriteClient.java b/src/main/java/com/uber/rss/clients/MultiServerSyncWriteClient.java index 71ffcb8..28d6196 100644 --- a/src/main/java/com/uber/rss/clients/MultiServerSyncWriteClient.java +++ b/src/main/java/com/uber/rss/clients/MultiServerSyncWriteClient.java @@ -89,7 +89,7 @@ public MultiServerSyncWriteClient(Collection servers, in this.servers.size())); } - logger.info(String.format("Created %s", this.getClass().getSimpleName())); + logger.info("Created {}", this.getClass().getSimpleName()); } @Override @@ -174,7 +174,7 @@ private void connectSingleClient(ServerConnectionInfo server) { private void closeClient(ReplicatedWriteClient client) { try { if (client != null) { - logger.debug(String.format("Closing client: %s", client)); + logger.debug("Closing client: {}", client); client.close(); } } catch (Throwable ex) { @@ -187,7 +187,7 @@ private void closeMetrics() { metrics.close(); } catch (Throwable e) { M3Stats.addException(e, this.getClass().getSimpleName()); - logger.warn(String.format("Failed to close metrics: %s", this), e); + logger.warn("Failed to close metrics: {}", this, e); } } diff --git a/src/main/java/com/uber/rss/clients/NotifyClient.java b/src/main/java/com/uber/rss/clients/NotifyClient.java index e14dfba..25ddd7f 100644 --- a/src/main/java/com/uber/rss/clients/NotifyClient.java +++ b/src/main/java/com/uber/rss/clients/NotifyClient.java @@ -50,7 +50,7 @@ public ConnectNotifyResponse connect() { throw new RssInvalidStateException(String.format("Already connected to server, cannot connect again: %s", connectionInfo)); } - logger.debug(String.format("Connecting to server: %s", connectionInfo)); + logger.debug("Connecting to server: {}", connectionInfo); connectSocket(); @@ -63,7 +63,7 @@ public ConnectNotifyResponse connect() { ConnectNotifyResponse connectResponse = readResponseMessage(MessageConstants.MESSAGE_ConnectNotifyResponse, ConnectNotifyResponse::deserialize); - logger.info(String.format("Connected to server: %s, response: %s", connectionInfo, connectResponse)); + logger.info("Connected to server: {}, response: {}", connectionInfo, connectResponse); return connectResponse; } @@ -94,7 +94,7 @@ private void closeMetrics() { } } catch (Throwable e) { M3Stats.addException(e, this.getClass().getSimpleName()); - logger.warn(String.format("Failed to close metrics: %s", connectionInfo), e); + logger.warn("Failed to close metrics: {}", connectionInfo, e); } } diff --git a/src/main/java/com/uber/rss/clients/PooledShuffleDataSyncWriteClient.java b/src/main/java/com/uber/rss/clients/PooledShuffleDataSyncWriteClient.java index a8f5c50..0ec4f45 100644 --- a/src/main/java/com/uber/rss/clients/PooledShuffleDataSyncWriteClient.java +++ b/src/main/java/com/uber/rss/clients/PooledShuffleDataSyncWriteClient.java @@ -135,12 +135,12 @@ public boolean isReusable() { } public void closeWithoutReuse() { - logger.info(String.format("Closing connection %s without reuse", this)); + logger.info("Closing connection {} without reuse", this); reusable = false; try { delegate.close(); } catch (Exception e) { - logger.warn(String.format("Failed to close underlying client %s", delegate), e); + logger.warn("Failed to close underlying client {}", delegate, e); } } diff --git a/src/main/java/com/uber/rss/clients/PooledWriteClientFactory.java b/src/main/java/com/uber/rss/clients/PooledWriteClientFactory.java index 4fcb4e5..d19b71c 100644 --- a/src/main/java/com/uber/rss/clients/PooledWriteClientFactory.java +++ b/src/main/java/com/uber/rss/clients/PooledWriteClientFactory.java @@ -76,7 +76,7 @@ public ShuffleDataSyncWriteClient getOrCreateClient(String host, int port, int t */ public void returnClientToPool(PooledShuffleDataSyncWriteClient client) { if (!client.isReusable()) { - logger.info(String.format("Client %s is not reusable, will close it instead of reuse it", client)); + logger.info("Client {} is not reusable, will close it instead of reuse it", client); client.closeWithoutReuse(); return; } @@ -84,7 +84,7 @@ public void returnClientToPool(PooledShuffleDataSyncWriteClient client) { ClientKey clientKey = new ClientKey(client); ClientPool pool = getPool(clientKey); pool.returnClientToPool(client); - logger.debug(String.format("Reuse client %s (%s)", client, clientKey)); + logger.debug("Reuse client {} ({})", client, clientKey); } public int getNumIdleClients() { @@ -107,7 +107,7 @@ public void shutdown() { try { clientAndState.closeClient(); } catch (Throwable ex) { - logger.warn(String.format("Failed to close pooled client %s", clientAndState.client), ex); + logger.warn("Failed to close pooled client {}", clientAndState.client, ex); } })); pools.clear(); @@ -247,7 +247,7 @@ public void closeLongIdleClients() { private PooledShuffleDataSyncWriteClient createClient(String host, int port, int timeoutMillis, boolean finishUploadAck, String user, String appId, String appAttempt, ShuffleWriteConfig shuffleWriteConfig) { ShuffleDataSyncWriteClient client; client = new PlainShuffleDataSyncWriteClient(host, port, timeoutMillis, finishUploadAck, user, appId, appAttempt, shuffleWriteConfig); - logger.info(String.format("Created new client: %s", client)); + logger.info("Created new client: {}", client); return new PooledShuffleDataSyncWriteClient(client, PooledWriteClientFactory.this); } } @@ -285,7 +285,7 @@ public void closeClient() { try { client.closeWithoutReuse(); } catch (Throwable ex) { - logger.warn(String.format("Failed to close client: %s", client)); + logger.warn("Failed to close client: {}", client); } } } diff --git a/src/main/java/com/uber/rss/clients/RecordSocketReadClient.java b/src/main/java/com/uber/rss/clients/RecordSocketReadClient.java index 15d8da6..20918e8 100644 --- a/src/main/java/com/uber/rss/clients/RecordSocketReadClient.java +++ b/src/main/java/com/uber/rss/clients/RecordSocketReadClient.java @@ -55,7 +55,7 @@ public DownloadServerVerboseInfo connect() { downloadServerVerboseInfo.setMapTaskCommitStatus(connectDownloadResponse.getMapTaskCommitStatus()); return downloadServerVerboseInfo; } catch (RuntimeException ex) { - logger.warn(String.format("Failed to connect %s", this), ex); + logger.warn("Failed to connect {}", this, ex); close(); throw ex; } @@ -66,7 +66,7 @@ public void close() { try { dataBlockSocketReadClient.close(); } catch (Throwable ex) { - logger.warn(String.format("Failed to close %s", this), ex); + logger.warn("Failed to close {}", this, ex); } closeMetrics(); @@ -82,7 +82,7 @@ public TaskDataBlock readDataBlock() { shuffleReadBytes += DataBlockHeader.NUM_BYTES + dataBlock.getPayload().length; return new TaskDataBlock(dataBlock.getPayload(), dataBlock.getHeader().getTaskAttemptId()); } catch (RuntimeException ex) { - logger.warn(String.format("Failed to read shuffle data %s", this), ex); + logger.warn("Failed to read shuffle data {}", this, ex); close(); throw ex; } @@ -108,7 +108,7 @@ private void closeMetrics() { } } catch (Throwable e) { M3Stats.addException(e, this.getClass().getSimpleName()); - logger.warn(String.format("Failed to close metrics: %s", this), e); + logger.warn("Failed to close metrics: {}", this, e); } } } diff --git a/src/main/java/com/uber/rss/clients/RegistryClient.java b/src/main/java/com/uber/rss/clients/RegistryClient.java index 73e662e..71e768b 100644 --- a/src/main/java/com/uber/rss/clients/RegistryClient.java +++ b/src/main/java/com/uber/rss/clients/RegistryClient.java @@ -47,7 +47,7 @@ public ConnectRegistryResponse connect() { throw new RssInvalidStateException(String.format("Already connected to server, cannot connect again: %s", connectionInfo)); } - logger.debug(String.format("Connecting to server: %s", connectionInfo)); + logger.debug("Connecting to server: {}", connectionInfo); connectSocket(); @@ -60,7 +60,7 @@ public ConnectRegistryResponse connect() { ConnectRegistryResponse connectResponse = readResponseMessage(MessageConstants.MESSAGE_ConnectRegistryResponse, ConnectRegistryResponse::deserialize); - logger.info(String.format("Connected to server: %s, response: %s", connectionInfo, connectResponse)); + logger.info("Connected to server: {}, response: {}", connectionInfo, connectResponse); return connectResponse; } diff --git a/src/main/java/com/uber/rss/clients/ReplicatedReadClient.java b/src/main/java/com/uber/rss/clients/ReplicatedReadClient.java index 1e3e6c1..98cf647 100644 --- a/src/main/java/com/uber/rss/clients/ReplicatedReadClient.java +++ b/src/main/java/com/uber/rss/clients/ReplicatedReadClient.java @@ -147,10 +147,8 @@ public synchronized void connect() { if (currentTime + sleepMillis - startTime > maxRetryTimeoutMillis) { throw ex; } else { - logger.warn(String.format( - "Failed to initialize, will wait %s millis and retry to connect to server replication group: %s", - sleepMillis, serverReplicationGroup), - ex); + logger.warn("Failed to initialize, will wait {} millis and retry to connect to server replication group: {}", + sleepMillis, serverReplicationGroup, ex); resetClientInstances(); try { Thread.sleep(sleepMillis); @@ -293,7 +291,7 @@ private void connectAndInitializeClient() { boolean succeeded = false; for (; currentClientIndex < clients.length; currentClientIndex++) { try { - logger.info(String.format("Trying to connect to server: %s", clients[currentClientIndex])); + logger.info("Trying to connect to server: {}", clients[currentClientIndex]); clients[currentClientIndex].connect(); clientsInitialized[currentClientIndex] = true; resetReadRecords(); diff --git a/src/main/java/com/uber/rss/clients/RetriableSocketReadClient.java b/src/main/java/com/uber/rss/clients/RetriableSocketReadClient.java index 00353f8..f9ad530 100644 --- a/src/main/java/com/uber/rss/clients/RetriableSocketReadClient.java +++ b/src/main/java/com/uber/rss/clients/RetriableSocketReadClient.java @@ -69,7 +69,7 @@ public DownloadServerVerboseInfo connect() { return delegate.connect(); } catch (RssNetworkException ex) { lastException = ex; - logger.warn(String.format("Failed to connect to server: %s", delegate), ex); + logger.warn("Failed to connect to server: {}", delegate, ex); closeDelegate(); long retryRemainingMillis = startTime + retryOptions.getRetryMaxMillis() - System.currentTimeMillis(); if (retryRemainingMillis <= 0) { @@ -77,8 +77,8 @@ public DownloadServerVerboseInfo connect() { } else { delegate = retryClientCreator.get(); long waitMillis = Math.min(retryOptions.getRetryIntervalMillis(), retryRemainingMillis); - logger.info(String.format("Waiting %s milliseconds (total retry milliseconds: %s, remaining milliseconds: %s) and retry to connect to server: %s", - waitMillis, retryOptions.getRetryMaxMillis(), retryRemainingMillis, delegate)); + logger.info("Waiting {} milliseconds (total retry milliseconds: {}, remaining milliseconds: {}) and retry to connect to server: {}", + waitMillis, retryOptions.getRetryMaxMillis(), retryRemainingMillis, delegate); ThreadUtils.sleep(waitMillis); } continue; diff --git a/src/main/java/com/uber/rss/clients/ServerBusyRetriableWriteClient.java b/src/main/java/com/uber/rss/clients/ServerBusyRetriableWriteClient.java index 1c3c928..2e663b9 100644 --- a/src/main/java/com/uber/rss/clients/ServerBusyRetriableWriteClient.java +++ b/src/main/java/com/uber/rss/clients/ServerBusyRetriableWriteClient.java @@ -76,7 +76,7 @@ public ConnectUploadResponse connect() { break; } catch (RssServerBusyException ex) { if (System.currentTimeMillis() - startTime < maxTryingMillis) { - logger.info(String.format("Server busy, will close current client and wait %s milliseconds to retry", waitMillis)); + logger.info("Server busy, will close current client and wait {} milliseconds to retry", waitMillis); closeDelegate(); delegate = null; try { @@ -139,7 +139,7 @@ private void closeDelegate() { try { delegate.close(); } catch (Throwable e) { - logger.warn(String.format("Failed to close delegate client %s", delegate), e); + logger.warn("Failed to close delegate client {}", delegate, e); } } } \ No newline at end of file diff --git a/src/main/java/com/uber/rss/clients/ServerIdAwareSocketReadClient.java b/src/main/java/com/uber/rss/clients/ServerIdAwareSocketReadClient.java index 403f421..d876e74 100644 --- a/src/main/java/com/uber/rss/clients/ServerIdAwareSocketReadClient.java +++ b/src/main/java/com/uber/rss/clients/ServerIdAwareSocketReadClient.java @@ -94,7 +94,7 @@ private void closeUnderlyingClient() { try { readClient.close(); } catch (Throwable ex) { - logger.warn(String.format("Failed to close underlying client %s", readClient), ex); + logger.warn("Failed to close underlying client {}", readClient, ex); } readClient = null; } diff --git a/src/main/java/com/uber/rss/clients/ServerIdAwareSyncWriteClient.java b/src/main/java/com/uber/rss/clients/ServerIdAwareSyncWriteClient.java index 55c2f92..b58bfcc 100644 --- a/src/main/java/com/uber/rss/clients/ServerIdAwareSyncWriteClient.java +++ b/src/main/java/com/uber/rss/clients/ServerIdAwareSyncWriteClient.java @@ -129,9 +129,9 @@ private ConnectUploadResponse connectImpl(ServerDetail serverDetail, ServerConne if (refresher == null) { throw ex; } else { - logger.warn(String.format("Failed to connect, retrying: %s", serverDetail), ex); + logger.warn("Failed to connect, retrying: {}", serverDetail, ex); ServerDetail newServerDetail = refresher.refreshConnection(serverDetail); - logger.info(String.format("Retry with %s for %s", newServerDetail, serverDetail)); + logger.info("Retry with {} for {}", newServerDetail, serverDetail); return connectImpl(newServerDetail, null, finishUploadAck); } } catch (Throwable ex) { diff --git a/src/main/java/com/uber/rss/clients/ShuffleDataSocketReadClient.java b/src/main/java/com/uber/rss/clients/ShuffleDataSocketReadClient.java index 4036601..73181aa 100644 --- a/src/main/java/com/uber/rss/clients/ShuffleDataSocketReadClient.java +++ b/src/main/java/com/uber/rss/clients/ShuffleDataSocketReadClient.java @@ -55,7 +55,7 @@ public DownloadServerVerboseInfo connect() { downloadServerVerboseInfo.setMapTaskCommitStatus(connectDownloadResponse.getMapTaskCommitStatus()); return downloadServerVerboseInfo; } catch (RuntimeException ex) { - logger.warn(String.format("Failed to connect %s", this), ex); + logger.warn("Failed to connect {}", this, ex); close(); throw ex; } @@ -66,7 +66,7 @@ public void close() { try { dataBlockSocketReadClient.close(); } catch (Throwable ex) { - logger.warn(String.format("Failed to close %s", this), ex); + logger.warn("Failed to close {}", this, ex); } closeMetrics(); @@ -82,7 +82,7 @@ public TaskDataBlock readDataBlock() { shuffleReadBytes += DataBlockHeader.NUM_BYTES + dataBlock.getPayload().length; return new TaskDataBlock(dataBlock.getPayload(), dataBlock.getHeader().getTaskAttemptId()); } catch (RuntimeException ex) { - logger.warn(String.format("Failed to read shuffle data %s", this), ex); + logger.warn("Failed to read shuffle data {}", this, ex); close(); throw ex; } @@ -108,7 +108,7 @@ private void closeMetrics() { } } catch (Throwable e) { M3Stats.addException(e, this.getClass().getSimpleName()); - logger.warn(String.format("Failed to close metrics: %s", this), e); + logger.warn("Failed to close metrics: {}", this, e); } } } diff --git a/src/main/java/com/uber/rss/decoders/StreamServerVersionDecoder.java b/src/main/java/com/uber/rss/decoders/StreamServerVersionDecoder.java index 61c9ca5..9fdccb4 100644 --- a/src/main/java/com/uber/rss/decoders/StreamServerVersionDecoder.java +++ b/src/main/java/com/uber/rss/decoders/StreamServerVersionDecoder.java @@ -100,11 +100,11 @@ private void addVersionDecoder(ChannelHandlerContext ctx, byte type, byte versio "Invalid upload version %d for link type %s from client %s", version, type, clientInfo)); ctx.close(); - logger.info(String.format("Closed connection to client %s", clientInfo)); + logger.info("Closed connection to client {}", clientInfo); return; } - logger.debug(String.format("Using version %d protocol for client %s", - version, NettyUtils.getServerConnectionInfo(ctx))); + logger.debug("Using version {} protocol for client {}", + version, NettyUtils.getServerConnectionInfo(ctx)); ctx.pipeline().replace(this, decoderName, newDecoder); ctx.pipeline().addAfter(decoderName, handlerName, newHandler); } @@ -130,11 +130,9 @@ protected void decode(ChannelHandlerContext ctx, break; default: String clientInfo = NettyUtils.getServerConnectionInfo(ctx); - logger.warn(String.format( - "Invalid magic byte %d from client %s", - magicByte, clientInfo)); + logger.warn("Invalid magic byte {} from client {}", magicByte, clientInfo); ctx.close(); - logger.info(String.format("Closed connection to client %s", clientInfo)); + logger.info("Closed connection to client {}", clientInfo); break; } } diff --git a/src/main/java/com/uber/rss/execution/LocalFileStateStore.java b/src/main/java/com/uber/rss/execution/LocalFileStateStore.java index 1fbb914..715faed 100644 --- a/src/main/java/com/uber/rss/execution/LocalFileStateStore.java +++ b/src/main/java/com/uber/rss/execution/LocalFileStateStore.java @@ -171,7 +171,7 @@ public String toString() { private void createNewFileIfNecessary() { synchronized (this) { if (closed) { - logger.info(String.format("State store already closed, do not create new file, %s", this)); + logger.info("State store already closed, do not create new file, {}", this); return; } @@ -188,7 +188,7 @@ private void createNewFileIfNecessary() { currentFileStream = new FileOutputStream(pathStr, true); currentFilePath = pathStr; currentFileCreateTime = System.currentTimeMillis(); - logger.info(String.format("Created state file: %s", pathStr)); + logger.info("Created state file: {}", pathStr); return; } catch (FileNotFoundException e) { throw new RssFileCorruptedException(String.format("Failed to create state file: %s", path)); @@ -205,7 +205,7 @@ private void deleteOldFiles() { try { FileUtils.cleanupOldFiles(stateDir, System.currentTimeMillis() - fileRetentionMillis); } catch (Throwable ex) { - logger.warn(String.format("Failed to clean up old state files in %s", stateDir), ex); + logger.warn("Failed to clean up old state files in {]", stateDir, ex); } } @@ -233,7 +233,7 @@ private void writeState(BaseMessage item) { private void closeFileNoLock() { if (currentFileStream != null) { - logger.info(String.format("Closing state file: %s", currentFilePath)); + logger.info("Closing state file: {}", currentFilePath); try { currentFileStream.close(); currentFileStream = null; diff --git a/src/main/java/com/uber/rss/execution/LocalFileStateStoreIterator.java b/src/main/java/com/uber/rss/execution/LocalFileStateStoreIterator.java index b552e33..26d303f 100644 --- a/src/main/java/com/uber/rss/execution/LocalFileStateStoreIterator.java +++ b/src/main/java/com/uber/rss/execution/LocalFileStateStoreIterator.java @@ -109,11 +109,11 @@ private void openFileIfNecessary() { currentFile = files.get(nextFileIndex++); try { - logger.info(String.format("Opening state file: %s", currentFile)); + logger.info("Opening state file: {}", currentFile); fileStream = new FileInputStream(currentFile); fileSize = fileStream.getChannel().size(); } catch (IOException e) { - logger.warn(String.format("Failed to open state file %s", currentFile), e); + logger.warn("Failed to open state file {}", currentFile, e); fileStream = null; fileSize = 0; continue; @@ -190,7 +190,7 @@ private byte[] readBytes(int numBytes) { return bytes; } } catch (Throwable e) { - logger.warn(String.format("Failed to read state file %s", currentFile), e); + logger.warn("Failed to read state file {}", currentFile, e); return null; } } @@ -201,10 +201,10 @@ private void closeCurrentFileStream() { } try { - logger.info(String.format("Closing state file: %s", currentFile)); + logger.info("Closing state file: {}", currentFile); fileStream.close(); } catch (IOException e) { - logger.warn(String.format("Failed to close state file: %s", currentFile), e); + logger.warn("Failed to close state file: {}", currentFile, e); } fileStream = null; fileSize = 0; diff --git a/src/main/java/com/uber/rss/execution/ShuffleExecutor.java b/src/main/java/com/uber/rss/execution/ShuffleExecutor.java index 225ef62..70b9106 100644 --- a/src/main/java/com/uber/rss/execution/ShuffleExecutor.java +++ b/src/main/java/com/uber/rss/execution/ShuffleExecutor.java @@ -583,7 +583,7 @@ private void removeExpiredApplications() { logger.info("Deleting expired application directory: {}", appDir); storage.deleteDirectory(appDir); } catch (Throwable ex) { - logger.warn(String.format("Failed to delete expired application directory: %s", appDir), ex); + logger.warn("Failed to delete expired application directory: {}", appDir, ex); } } } @@ -736,7 +736,7 @@ private void loadStateImpl(BaseMessage stateItem, Set appIds, Set getReachableServers(ServiceRegistry serviceRegi long requestLatency = System.currentTimeMillis() - startTime; return new ServerCandidate(t, requestLatency,getBusyStatusResponse.getMetrics().get(CONCURRENT_CONNS)); } catch (Throwable ex) { - logger.warn(String.format("Detected unreachable host %s", host), ex); + logger.warn("Detected unreachable host {}", host, ex); unreachableHosts.add(host); return null; } diff --git a/src/main/java/com/uber/rss/metrics/M3Stats.java b/src/main/java/com/uber/rss/metrics/M3Stats.java index c1650bb..3960cfb 100644 --- a/src/main/java/com/uber/rss/metrics/M3Stats.java +++ b/src/main/java/com/uber/rss/metrics/M3Stats.java @@ -132,7 +132,7 @@ private static Scope createScopeHelper() { if (scopeBuilderClassName == null || scopeBuilderClassName.isEmpty()) { scopeBuilderClassName = M3DummyScopeBuilder.class.getName(); } - logger.info(String.format("Using scope builder: %s", scopeBuilderClassName)); + logger.info("Using scope builder: {}", scopeBuilderClassName); ScopeBuilder scopeBuilder; try { diff --git a/src/main/java/com/uber/rss/metrics/ScheduledMetricCollector.java b/src/main/java/com/uber/rss/metrics/ScheduledMetricCollector.java index a50ffbc..dfde44c 100644 --- a/src/main/java/com/uber/rss/metrics/ScheduledMetricCollector.java +++ b/src/main/java/com/uber/rss/metrics/ScheduledMetricCollector.java @@ -113,7 +113,7 @@ public int compare(ServerDetail o1, ServerDetail o2) { } unreachableHosts.update(unreachableHostList.size()); if (!unreachableHostList.isEmpty()) { - logger.warn(String.format("Detected unreachable hosts: %s", StringUtils.join(unreachableHostList, ","))); + logger.warn("Detected unreachable hosts: {}", String.join(",", unreachableHostList)); } unreachableHostsCheckLatency.update(System.currentTimeMillis() - startTime); } diff --git a/src/main/java/com/uber/rss/tools/StreamServerStressTool.java b/src/main/java/com/uber/rss/tools/StreamServerStressTool.java index d3cd280..98769c9 100644 --- a/src/main/java/com/uber/rss/tools/StreamServerStressTool.java +++ b/src/main/java/com/uber/rss/tools/StreamServerStressTool.java @@ -489,12 +489,12 @@ public void uncaughtException(Thread t, Throwable e) { if (deleteFiles) { try { - logger.info(String.format("Deleting files: %s", StringUtils.join(serverRootDirs, ", "))); + logger.info("Deleting files: {}", String.join(", ", serverRootDirs)); deleteDirectories(serverRootDirs); - logger.info(String.format("Deleted files: %s", StringUtils.join(serverRootDirs, ", "))); + logger.info("Deleted files: {}", String.join(", ", serverRootDirs)); } catch (Throwable ex) { M3Stats.addException(ex, M3Stats.TAG_VALUE_STRESS_TOOL); - logger.info("Got some error when deleting files: %s, ignored them"); + logger.info("Got some error when deleting files: {}, ignored them", String.join(", ", serverRootDirs)); } } } @@ -558,7 +558,7 @@ private void simulateMapperTask(List testValues, ConcurrentHashMap numPartitionRecords) { if (mapDelay > 0) { int delayMillis = random.nextInt(mapDelay); - logger.info(String.format("Delaying map %s: %s", appMapId, delayMillis)); + logger.info("Delaying map {}: {}", appMapId, delayMillis); try { Thread.sleep(delayMillis); } catch (InterruptedException e) { @@ -586,7 +586,7 @@ private void simulateMapperTask(List testValues, writeClient.startUpload(new AppTaskAttemptId(appMapId, taskAttemptId), numMaps, numPartitions); } - logger.info(String.format("Map %s attempt %s started, write client: %s", appMapId, taskAttemptId, writeClient)); + logger.info("Map {} attempt {} started, write client: {}", appMapId, taskAttemptId, writeClient); if (!simulateEmptyData) { int partitionId = random.nextInt(numPartitions); @@ -654,7 +654,7 @@ private void simulateMapperTask(List testValues, synchronized (serverIdsToShutdownDuringShuffleWrite) { for (String serverId : serverIdsToShutdownDuringShuffleWrite) { StreamServer server = servers.stream().filter(t -> t != null).filter(t -> t.getServerId().equals(serverId)).findFirst().get(); - logger.info(String.format("Simulate bad server during shuffle write by shutting down server: %s", server)); + logger.info("Simulate bad server during shuffle write by shutting down server: {}", server); shutdownServer(server); int index = servers.indexOf(server); @@ -676,11 +676,11 @@ private void simulateMapperTask(List testValues, Double rate = rateCounter.addValueAndGetRate(bytes); if (rate != null) { long mapUploadedBytes = rateCounter.getOverallValue(); - logger.info(String.format("Map %s uploaded bytes: %s, rate: %s mb/s", appMapId, mapUploadedBytes, rate*(1000.0/(1024.0*1024.0)))); + logger.info("Map {} uploaded bytes: {}, rate: {} mb/s", appMapId, mapUploadedBytes, rate*(1000.0/(1024.0*1024.0))); } try { - logger.info(String.format("Closing write client: %s", writeClient)); + logger.info("Closing write client: {}", writeClient); writeClient.close(); } catch (Exception e) { M3Stats.addException(e, M3Stats.TAG_VALUE_STRESS_TOOL); @@ -692,11 +692,11 @@ private void simulateMapperTask(List testValues, if (isLastTaskAttempt) { throw ex; } else { - logger.debug(String.format("Got ignorable error from stale map task: %s", ExceptionUtils.getSimpleMessage(ex))); + logger.debug("Got ignorable error from stale map task", ex); } } - logger.info(String.format("Map %s attempt %s finished", appMapId, taskAttemptId)); + logger.info("Map {} attempt {} finished", appMapId, taskAttemptId); double overallBytesMb = rateCounter.getOverallValue()/(1024.0*1024.0); double overallRate = rateCounter.getOverallRate()*(1000.0/(1024.0*1024.0)); @@ -739,7 +739,7 @@ private void startNewServer() { } private void shutdownServer(StreamServer server) { - logger.info(String.format("Shutting down server: %s", server)); + logger.info("Shutting down server: {}", server); server.shutdown(true); } @@ -825,6 +825,6 @@ public static void main(String[] args) { M3Stats.closeDefaultScope(); - logger.info(String.format("%s finished", StreamServerStressToolLongRun.class.getSimpleName())); + logger.info("{} finished", StreamServerStressToolLongRun.class.getSimpleName()); } } diff --git a/src/main/java/com/uber/rss/util/FileUtils.java b/src/main/java/com/uber/rss/util/FileUtils.java index 5ba5538..2deea5b 100644 --- a/src/main/java/com/uber/rss/util/FileUtils.java +++ b/src/main/java/com/uber/rss/util/FileUtils.java @@ -114,7 +114,7 @@ public static long getFileStoreUsableSpace() { maxUsableSpace = storeUsableSpace; } } catch (Throwable e) { - logger.warn(String.format("Failed to check file store size for %s", root), e); + logger.warn("Failed to check file store size for {}", root, e); } } return maxUsableSpace;