001 /**
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements. See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership. The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License. You may obtain a copy of the License at
009 *
010 * http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018 package org.apache.hadoop.hdfs;
019
020 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
021 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
022 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT;
023 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY;
024 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT;
025 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY;
026 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT;
027 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY;
028 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT;
029 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHED_CONN_RETRY_KEY;
030 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS;
031 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES;
032 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CACHE_READAHEAD;
033 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY;
034 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT;
035 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT;
036 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY;
037 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT;
038 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY;
039 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT;
040 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY;
041 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_RETRY_MAX_ATTEMPTS_DEFAULT;
042 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_RETRY_MAX_ATTEMPTS_KEY;
043 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT;
044 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
045 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_READ_PREFETCH_SIZE_KEY;
046 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_RETRY_WINDOW_BASE;
047 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT;
048 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY;
049 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT;
050 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY;
051 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
052 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME;
053 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT;
054 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL;
055 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT;
056 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
057 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
058 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
059 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_DEFAULT;
060 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_REPLICATION_KEY;
061 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT;
062 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT;
063
064 import java.io.BufferedOutputStream;
065 import java.io.DataInputStream;
066 import java.io.DataOutputStream;
067 import java.io.FileNotFoundException;
068 import java.io.IOException;
069 import java.io.InputStream;
070 import java.io.OutputStream;
071 import java.net.InetAddress;
072 import java.net.InetSocketAddress;
073 import java.net.NetworkInterface;
074 import java.net.Socket;
075 import java.net.SocketException;
076 import java.net.SocketAddress;
077 import java.net.URI;
078 import java.net.UnknownHostException;
079 import java.util.ArrayList;
080 import java.util.Collections;
081 import java.util.EnumSet;
082 import java.util.HashMap;
083 import java.util.LinkedHashMap;
084 import java.util.List;
085 import java.util.Map;
086 import java.util.Random;
087 import java.util.concurrent.SynchronousQueue;
088 import java.util.concurrent.ThreadPoolExecutor;
089 import java.util.concurrent.TimeUnit;
090 import java.util.concurrent.atomic.AtomicInteger;
091
092 import javax.net.SocketFactory;
093
094 import org.apache.commons.logging.Log;
095 import org.apache.commons.logging.LogFactory;
096 import org.apache.hadoop.classification.InterfaceAudience;
097 import org.apache.hadoop.conf.Configuration;
098 import org.apache.hadoop.fs.BlockLocation;
099 import org.apache.hadoop.fs.BlockStorageLocation;
100 import org.apache.hadoop.fs.CacheFlag;
101 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
102 import org.apache.hadoop.fs.ContentSummary;
103 import org.apache.hadoop.fs.CreateFlag;
104 import org.apache.hadoop.fs.FileAlreadyExistsException;
105 import org.apache.hadoop.fs.FileSystem;
106 import org.apache.hadoop.fs.FsServerDefaults;
107 import org.apache.hadoop.fs.FsStatus;
108 import org.apache.hadoop.fs.HdfsBlockLocation;
109 import org.apache.hadoop.fs.InvalidPathException;
110 import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
111 import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
112 import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
113 import org.apache.hadoop.fs.Options;
114 import org.apache.hadoop.fs.RemoteIterator;
115 import org.apache.hadoop.fs.Options.ChecksumOpt;
116 import org.apache.hadoop.fs.ParentNotDirectoryException;
117 import org.apache.hadoop.fs.Path;
118 import org.apache.hadoop.fs.UnresolvedLinkException;
119 import org.apache.hadoop.fs.VolumeId;
120 import org.apache.hadoop.fs.permission.AclEntry;
121 import org.apache.hadoop.fs.permission.AclStatus;
122 import org.apache.hadoop.fs.permission.FsPermission;
123 import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
124 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
125 import org.apache.hadoop.hdfs.protocol.AclException;
126 import org.apache.hadoop.hdfs.net.Peer;
127 import org.apache.hadoop.hdfs.net.TcpPeerServer;
128 import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
129 import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
130 import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
131 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
132 import org.apache.hadoop.hdfs.protocol.CachePoolIterator;
133 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
134 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
135 import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
136 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
137 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
138 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
139 import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
140 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
141 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
142 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
143 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
144 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
145 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
146 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
147 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
148 import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
149 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
150 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
151 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
152 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
153 import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
154 import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferEncryptor;
155 import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
156 import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
157 import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
158 import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
159 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
160 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
161 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
162 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
163 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
164 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
165 import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
166 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
167 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
168 import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
169 import org.apache.hadoop.hdfs.server.namenode.NameNode;
170 import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
171 import org.apache.hadoop.io.DataOutputBuffer;
172 import org.apache.hadoop.io.EnumSetWritable;
173 import org.apache.hadoop.io.IOUtils;
174 import org.apache.hadoop.io.MD5Hash;
175 import org.apache.hadoop.io.Text;
176 import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
177 import org.apache.hadoop.ipc.Client;
178 import org.apache.hadoop.ipc.RPC;
179 import org.apache.hadoop.ipc.RemoteException;
180 import org.apache.hadoop.net.DNS;
181 import org.apache.hadoop.net.NetUtils;
182 import org.apache.hadoop.security.AccessControlException;
183 import org.apache.hadoop.security.UserGroupInformation;
184 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
185 import org.apache.hadoop.security.token.Token;
186 import org.apache.hadoop.security.token.TokenRenewer;
187 import org.apache.hadoop.util.Daemon;
188 import org.apache.hadoop.util.DataChecksum;
189 import org.apache.hadoop.util.DataChecksum.Type;
190 import org.apache.hadoop.util.Progressable;
191 import org.apache.hadoop.util.Time;
192
193 import com.google.common.annotations.VisibleForTesting;
194 import com.google.common.base.Joiner;
195 import com.google.common.base.Preconditions;
196 import com.google.common.net.InetAddresses;
197
198 /********************************************************
199 * DFSClient can connect to a Hadoop Filesystem and
200 * perform basic file tasks. It uses the ClientProtocol
201 * to communicate with a NameNode daemon, and connects
202 * directly to DataNodes to read/write block data.
203 *
204 * Hadoop DFS users should obtain an instance of
205 * DistributedFileSystem, which uses DFSClient to handle
206 * filesystem tasks.
207 *
208 ********************************************************/
209 @InterfaceAudience.Private
210 public class DFSClient implements java.io.Closeable, RemotePeerFactory {
211 public static final Log LOG = LogFactory.getLog(DFSClient.class);
212 public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
213 static final int TCP_WINDOW_SIZE = 128 * 1024; // 128 KB
214
215 private final Configuration conf;
216 private final Conf dfsClientConf;
217 final ClientProtocol namenode;
218 /* The service used for delegation tokens */
219 private Text dtService;
220
221 final UserGroupInformation ugi;
222 volatile boolean clientRunning = true;
223 volatile long lastLeaseRenewal;
224 private volatile FsServerDefaults serverDefaults;
225 private volatile long serverDefaultsLastUpdate;
226 final String clientName;
227 final SocketFactory socketFactory;
228 final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
229 final FileSystem.Statistics stats;
230 private final String authority;
231 private final Random r = new Random();
232 private SocketAddress[] localInterfaceAddrs;
233 private DataEncryptionKey encryptionKey;
234 final TrustedChannelResolver trustedChannelResolver;
235 private final CachingStrategy defaultReadCachingStrategy;
236 private final CachingStrategy defaultWriteCachingStrategy;
237 private final ClientContext clientContext;
238 private volatile long hedgedReadThresholdMillis;
239 private static final DFSHedgedReadMetrics HEDGED_READ_METRIC =
240 new DFSHedgedReadMetrics();
241 private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
242
243 /**
244 * DFSClient configuration
245 */
246 public static class Conf {
247 final int hdfsTimeout; // timeout value for a DFS operation.
248
249 final int maxFailoverAttempts;
250 final int maxRetryAttempts;
251 final int failoverSleepBaseMillis;
252 final int failoverSleepMaxMillis;
253 final int maxBlockAcquireFailures;
254 final int confTime;
255 final int ioBufferSize;
256 final ChecksumOpt defaultChecksumOpt;
257 final int writePacketSize;
258 final int socketTimeout;
259 final int socketCacheCapacity;
260 final long socketCacheExpiry;
261 final long excludedNodesCacheExpiry;
262 /** Wait time window (in msec) if BlockMissingException is caught */
263 final int timeWindow;
264 final int nCachedConnRetry;
265 final int nBlockWriteRetry;
266 final int nBlockWriteLocateFollowingRetry;
267 final long defaultBlockSize;
268 final long prefetchSize;
269 final short defaultReplication;
270 final String taskId;
271 final FsPermission uMask;
272 final boolean connectToDnViaHostname;
273 final boolean getHdfsBlocksMetadataEnabled;
274 final int getFileBlockStorageLocationsNumThreads;
275 final int getFileBlockStorageLocationsTimeoutMs;
276 final int retryTimesForGetLastBlockLength;
277 final int retryIntervalForGetLastBlockLength;
278 final long datanodeRestartTimeout;
279
280 final boolean useLegacyBlockReader;
281 final boolean useLegacyBlockReaderLocal;
282 final String domainSocketPath;
283 final boolean skipShortCircuitChecksums;
284 final int shortCircuitBufferSize;
285 final boolean shortCircuitLocalReads;
286 final boolean domainSocketDataTraffic;
287 final int shortCircuitStreamsCacheSize;
288 final long shortCircuitStreamsCacheExpiryMs;
289 final int shortCircuitSharedMemoryWatcherInterruptCheckMs;
290
291 final boolean shortCircuitMmapEnabled;
292 final int shortCircuitMmapCacheSize;
293 final long shortCircuitMmapCacheExpiryMs;
294 final long shortCircuitMmapCacheRetryTimeout;
295 final long shortCircuitCacheStaleThresholdMs;
296
297 public Conf(Configuration conf) {
298 // The hdfsTimeout is currently the same as the ipc timeout
299 hdfsTimeout = Client.getTimeout(conf);
300 maxFailoverAttempts = conf.getInt(
301 DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY,
302 DFS_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT);
303 maxRetryAttempts = conf.getInt(
304 DFS_CLIENT_RETRY_MAX_ATTEMPTS_KEY,
305 DFS_CLIENT_RETRY_MAX_ATTEMPTS_DEFAULT);
306 failoverSleepBaseMillis = conf.getInt(
307 DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_KEY,
308 DFS_CLIENT_FAILOVER_SLEEPTIME_BASE_DEFAULT);
309 failoverSleepMaxMillis = conf.getInt(
310 DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_KEY,
311 DFS_CLIENT_FAILOVER_SLEEPTIME_MAX_DEFAULT);
312
313 maxBlockAcquireFailures = conf.getInt(
314 DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
315 DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT);
316 confTime = conf.getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
317 HdfsServerConstants.WRITE_TIMEOUT);
318 ioBufferSize = conf.getInt(
319 CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
320 CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
321 defaultChecksumOpt = getChecksumOptFromConf(conf);
322 socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
323 HdfsServerConstants.READ_TIMEOUT);
324 /** dfs.write.packet.size is an internal config variable */
325 writePacketSize = conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
326 DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
327 defaultBlockSize = conf.getLongBytes(DFS_BLOCK_SIZE_KEY,
328 DFS_BLOCK_SIZE_DEFAULT);
329 defaultReplication = (short) conf.getInt(
330 DFS_REPLICATION_KEY, DFS_REPLICATION_DEFAULT);
331 taskId = conf.get("mapreduce.task.attempt.id", "NONMAPREDUCE");
332 socketCacheCapacity = conf.getInt(DFS_CLIENT_SOCKET_CACHE_CAPACITY_KEY,
333 DFS_CLIENT_SOCKET_CACHE_CAPACITY_DEFAULT);
334 socketCacheExpiry = conf.getLong(DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_KEY,
335 DFS_CLIENT_SOCKET_CACHE_EXPIRY_MSEC_DEFAULT);
336 excludedNodesCacheExpiry = conf.getLong(
337 DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL,
338 DFS_CLIENT_WRITE_EXCLUDE_NODES_CACHE_EXPIRY_INTERVAL_DEFAULT);
339 prefetchSize = conf.getLong(DFS_CLIENT_READ_PREFETCH_SIZE_KEY,
340 10 * defaultBlockSize);
341 timeWindow = conf.getInt(DFS_CLIENT_RETRY_WINDOW_BASE, 3000);
342 nCachedConnRetry = conf.getInt(DFS_CLIENT_CACHED_CONN_RETRY_KEY,
343 DFS_CLIENT_CACHED_CONN_RETRY_DEFAULT);
344 nBlockWriteRetry = conf.getInt(DFS_CLIENT_BLOCK_WRITE_RETRIES_KEY,
345 DFS_CLIENT_BLOCK_WRITE_RETRIES_DEFAULT);
346 nBlockWriteLocateFollowingRetry = conf.getInt(
347 DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_KEY,
348 DFS_CLIENT_BLOCK_WRITE_LOCATEFOLLOWINGBLOCK_RETRIES_DEFAULT);
349 uMask = FsPermission.getUMask(conf);
350 connectToDnViaHostname = conf.getBoolean(DFS_CLIENT_USE_DN_HOSTNAME,
351 DFS_CLIENT_USE_DN_HOSTNAME_DEFAULT);
352 getHdfsBlocksMetadataEnabled = conf.getBoolean(
353 DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED,
354 DFSConfigKeys.DFS_HDFS_BLOCKS_METADATA_ENABLED_DEFAULT);
355 getFileBlockStorageLocationsNumThreads = conf.getInt(
356 DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS,
357 DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_NUM_THREADS_DEFAULT);
358 getFileBlockStorageLocationsTimeoutMs = conf.getInt(
359 DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS,
360 DFSConfigKeys.DFS_CLIENT_FILE_BLOCK_STORAGE_LOCATIONS_TIMEOUT_MS_DEFAULT);
361 retryTimesForGetLastBlockLength = conf.getInt(
362 DFSConfigKeys.DFS_CLIENT_RETRY_TIMES_GET_LAST_BLOCK_LENGTH,
363 DFSConfigKeys.DFS_CLIENT_RETRY_TIMES_GET_LAST_BLOCK_LENGTH_DEFAULT);
364 retryIntervalForGetLastBlockLength = conf.getInt(
365 DFSConfigKeys.DFS_CLIENT_RETRY_INTERVAL_GET_LAST_BLOCK_LENGTH,
366 DFSConfigKeys.DFS_CLIENT_RETRY_INTERVAL_GET_LAST_BLOCK_LENGTH_DEFAULT);
367
368 useLegacyBlockReader = conf.getBoolean(
369 DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER,
370 DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADER_DEFAULT);
371 useLegacyBlockReaderLocal = conf.getBoolean(
372 DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL,
373 DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL_DEFAULT);
374 shortCircuitLocalReads = conf.getBoolean(
375 DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY,
376 DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_DEFAULT);
377 domainSocketDataTraffic = conf.getBoolean(
378 DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC,
379 DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC_DEFAULT);
380 domainSocketPath = conf.getTrimmed(
381 DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY,
382 DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_DEFAULT);
383
384 if (BlockReaderLocal.LOG.isDebugEnabled()) {
385 BlockReaderLocal.LOG.debug(
386 DFSConfigKeys.DFS_CLIENT_USE_LEGACY_BLOCKREADERLOCAL
387 + " = " + useLegacyBlockReaderLocal);
388 BlockReaderLocal.LOG.debug(
389 DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_KEY
390 + " = " + shortCircuitLocalReads);
391 BlockReaderLocal.LOG.debug(
392 DFSConfigKeys.DFS_CLIENT_DOMAIN_SOCKET_DATA_TRAFFIC
393 + " = " + domainSocketDataTraffic);
394 BlockReaderLocal.LOG.debug(
395 DFSConfigKeys.DFS_DOMAIN_SOCKET_PATH_KEY
396 + " = " + domainSocketPath);
397 }
398
399 skipShortCircuitChecksums = conf.getBoolean(
400 DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_KEY,
401 DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_SKIP_CHECKSUM_DEFAULT);
402 shortCircuitBufferSize = conf.getInt(
403 DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_KEY,
404 DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_BUFFER_SIZE_DEFAULT);
405 shortCircuitStreamsCacheSize = conf.getInt(
406 DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_KEY,
407 DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_SIZE_DEFAULT);
408 shortCircuitStreamsCacheExpiryMs = conf.getLong(
409 DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_KEY,
410 DFSConfigKeys.DFS_CLIENT_READ_SHORTCIRCUIT_STREAMS_CACHE_EXPIRY_MS_DEFAULT);
411 shortCircuitMmapEnabled = conf.getBoolean(
412 DFSConfigKeys.DFS_CLIENT_MMAP_ENABLED,
413 DFSConfigKeys.DFS_CLIENT_MMAP_ENABLED_DEFAULT);
414 shortCircuitMmapCacheSize = conf.getInt(
415 DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE,
416 DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_SIZE_DEFAULT);
417 shortCircuitMmapCacheExpiryMs = conf.getLong(
418 DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS,
419 DFSConfigKeys.DFS_CLIENT_MMAP_CACHE_TIMEOUT_MS_DEFAULT);
420 shortCircuitMmapCacheRetryTimeout = conf.getLong(
421 DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS,
422 DFSConfigKeys.DFS_CLIENT_MMAP_RETRY_TIMEOUT_MS_DEFAULT);
423 shortCircuitCacheStaleThresholdMs = conf.getLong(
424 DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS,
425 DFSConfigKeys.DFS_CLIENT_SHORT_CIRCUIT_REPLICA_STALE_THRESHOLD_MS_DEFAULT);
426 shortCircuitSharedMemoryWatcherInterruptCheckMs = conf.getInt(
427 DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS,
428 DFSConfigKeys.DFS_SHORT_CIRCUIT_SHARED_MEMORY_WATCHER_INTERRUPT_CHECK_MS_DEFAULT);
429
430 datanodeRestartTimeout = conf.getLong(
431 DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY,
432 DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT) * 1000;
433 }
434
435 private DataChecksum.Type getChecksumType(Configuration conf) {
436 final String checksum = conf.get(
437 DFSConfigKeys.DFS_CHECKSUM_TYPE_KEY,
438 DFSConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT);
439 try {
440 return DataChecksum.Type.valueOf(checksum);
441 } catch(IllegalArgumentException iae) {
442 LOG.warn("Bad checksum type: " + checksum + ". Using default "
443 + DFSConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT);
444 return DataChecksum.Type.valueOf(
445 DFSConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT);
446 }
447 }
448
449 // Construct a checksum option from conf
450 private ChecksumOpt getChecksumOptFromConf(Configuration conf) {
451 DataChecksum.Type type = getChecksumType(conf);
452 int bytesPerChecksum = conf.getInt(DFS_BYTES_PER_CHECKSUM_KEY,
453 DFS_BYTES_PER_CHECKSUM_DEFAULT);
454 return new ChecksumOpt(type, bytesPerChecksum);
455 }
456
457 // create a DataChecksum with the default option.
458 private DataChecksum createChecksum() throws IOException {
459 return createChecksum(null);
460 }
461
462 private DataChecksum createChecksum(ChecksumOpt userOpt)
463 throws IOException {
464 // Fill in any missing field with the default.
465 ChecksumOpt myOpt = ChecksumOpt.processChecksumOpt(
466 defaultChecksumOpt, userOpt);
467 DataChecksum dataChecksum = DataChecksum.newDataChecksum(
468 myOpt.getChecksumType(),
469 myOpt.getBytesPerChecksum());
470 if (dataChecksum == null) {
471 throw new IOException("Invalid checksum type specified: "
472 + myOpt.getChecksumType().name());
473 }
474 return dataChecksum;
475 }
476 }
477
478 public Conf getConf() {
479 return dfsClientConf;
480 }
481
482 Configuration getConfiguration() {
483 return conf;
484 }
485
486 /**
487 * A map from file names to {@link DFSOutputStream} objects
488 * that are currently being written by this client.
489 * Note that a file can only be written by a single client.
490 */
491 private final Map<String, DFSOutputStream> filesBeingWritten
492 = new HashMap<String, DFSOutputStream>();
493
494 /**
495 * Same as this(NameNode.getAddress(conf), conf);
496 * @see #DFSClient(InetSocketAddress, Configuration)
497 * @deprecated Deprecated at 0.21
498 */
499 @Deprecated
500 public DFSClient(Configuration conf) throws IOException {
501 this(NameNode.getAddress(conf), conf);
502 }
503
504 public DFSClient(InetSocketAddress address, Configuration conf) throws IOException {
505 this(NameNode.getUri(address), conf);
506 }
507
508 /**
509 * Same as this(nameNodeUri, conf, null);
510 * @see #DFSClient(URI, Configuration, FileSystem.Statistics)
511 */
512 public DFSClient(URI nameNodeUri, Configuration conf
513 ) throws IOException {
514 this(nameNodeUri, conf, null);
515 }
516
517 /**
518 * Same as this(nameNodeUri, null, conf, stats);
519 * @see #DFSClient(URI, ClientProtocol, Configuration, FileSystem.Statistics)
520 */
521 public DFSClient(URI nameNodeUri, Configuration conf,
522 FileSystem.Statistics stats)
523 throws IOException {
524 this(nameNodeUri, null, conf, stats);
525 }
526
527 /**
528 * Create a new DFSClient connected to the given nameNodeUri or rpcNamenode.
529 * If HA is enabled and a positive value is set for
530 * {@link DFSConfigKeys#DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY} in the
531 * configuration, the DFSClient will use {@link LossyRetryInvocationHandler}
532 * as its RetryInvocationHandler. Otherwise one of nameNodeUri or rpcNamenode
533 * must be null.
534 */
535 @VisibleForTesting
536 public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
537 Configuration conf, FileSystem.Statistics stats)
538 throws IOException {
539 // Copy only the required DFSClient configuration
540 this.dfsClientConf = new Conf(conf);
541 if (this.dfsClientConf.useLegacyBlockReaderLocal) {
542 LOG.debug("Using legacy short-circuit local reads.");
543 }
544 this.conf = conf;
545 this.stats = stats;
546 this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
547 this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
548
549 this.ugi = UserGroupInformation.getCurrentUser();
550
551 this.authority = nameNodeUri == null? "null": nameNodeUri.getAuthority();
552 this.clientName = "DFSClient_" + dfsClientConf.taskId + "_" +
553 DFSUtil.getRandom().nextInt() + "_" + Thread.currentThread().getId();
554
555 int numResponseToDrop = conf.getInt(
556 DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
557 DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
558 NameNodeProxies.ProxyAndInfo<ClientProtocol> proxyInfo = null;
559 if (numResponseToDrop > 0) {
560 // This case is used for testing.
561 LOG.warn(DFSConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
562 + " is set to " + numResponseToDrop
563 + ", this hacked client will proactively drop responses");
564 proxyInfo = NameNodeProxies.createProxyWithLossyRetryHandler(conf,
565 nameNodeUri, ClientProtocol.class, numResponseToDrop);
566 }
567
568 if (proxyInfo != null) {
569 this.dtService = proxyInfo.getDelegationTokenService();
570 this.namenode = proxyInfo.getProxy();
571 } else if (rpcNamenode != null) {
572 // This case is used for testing.
573 Preconditions.checkArgument(nameNodeUri == null);
574 this.namenode = rpcNamenode;
575 dtService = null;
576 } else {
577 Preconditions.checkArgument(nameNodeUri != null,
578 "null URI");
579 proxyInfo = NameNodeProxies.createProxy(conf, nameNodeUri,
580 ClientProtocol.class);
581 this.dtService = proxyInfo.getDelegationTokenService();
582 this.namenode = proxyInfo.getProxy();
583 }
584
585 String localInterfaces[] =
586 conf.getTrimmedStrings(DFSConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
587 localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
588 if (LOG.isDebugEnabled() && 0 != localInterfaces.length) {
589 LOG.debug("Using local interfaces [" +
590 Joiner.on(',').join(localInterfaces)+ "] with addresses [" +
591 Joiner.on(',').join(localInterfaceAddrs) + "]");
592 }
593
594 Boolean readDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_READS) == null) ?
595 null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_READS, false);
596 Long readahead = (conf.get(DFS_CLIENT_CACHE_READAHEAD) == null) ?
597 null : conf.getLong(DFS_CLIENT_CACHE_READAHEAD, 0);
598 Boolean writeDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES) == null) ?
599 null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES, false);
600 this.defaultReadCachingStrategy =
601 new CachingStrategy(readDropBehind, readahead);
602 this.defaultWriteCachingStrategy =
603 new CachingStrategy(writeDropBehind, readahead);
604 this.clientContext = ClientContext.get(
605 conf.get(DFS_CLIENT_CONTEXT, DFS_CLIENT_CONTEXT_DEFAULT),
606 dfsClientConf);
607 this.hedgedReadThresholdMillis = conf.getLong(
608 DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS,
609 DFSConfigKeys.DEFAULT_DFSCLIENT_HEDGED_READ_THRESHOLD_MILLIS);
610 int numThreads = conf.getInt(
611 DFSConfigKeys.DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE,
612 DFSConfigKeys.DEFAULT_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE);
613 if (numThreads > 0) {
614 this.initThreadsNumForHedgedReads(numThreads);
615 }
616 this.trustedChannelResolver = TrustedChannelResolver.getInstance(getConfiguration());
617 }
618
619 /**
620 * Return the socket addresses to use with each configured
621 * local interface. Local interfaces may be specified by IP
622 * address, IP address range using CIDR notation, interface
623 * name (e.g. eth0) or sub-interface name (e.g. eth0:0).
624 * The socket addresses consist of the IPs for the interfaces
625 * and the ephemeral port (port 0). If an IP, IP range, or
626 * interface name matches an interface with sub-interfaces
627 * only the IP of the interface is used. Sub-interfaces can
628 * be used by specifying them explicitly (by IP or name).
629 *
630 * @return SocketAddresses for the configured local interfaces,
631 * or an empty array if none are configured
632 * @throws UnknownHostException if a given interface name is invalid
633 */
634 private static SocketAddress[] getLocalInterfaceAddrs(
635 String interfaceNames[]) throws UnknownHostException {
636 List<SocketAddress> localAddrs = new ArrayList<SocketAddress>();
637 for (String interfaceName : interfaceNames) {
638 if (InetAddresses.isInetAddress(interfaceName)) {
639 localAddrs.add(new InetSocketAddress(interfaceName, 0));
640 } else if (NetUtils.isValidSubnet(interfaceName)) {
641 for (InetAddress addr : NetUtils.getIPs(interfaceName, false)) {
642 localAddrs.add(new InetSocketAddress(addr, 0));
643 }
644 } else {
645 for (String ip : DNS.getIPs(interfaceName, false)) {
646 localAddrs.add(new InetSocketAddress(ip, 0));
647 }
648 }
649 }
650 return localAddrs.toArray(new SocketAddress[localAddrs.size()]);
651 }
652
653 /**
654 * Select one of the configured local interfaces at random. We use a random
655 * interface because other policies like round-robin are less effective
656 * given that we cache connections to datanodes.
657 *
658 * @return one of the local interface addresses at random, or null if no
659 * local interfaces are configured
660 */
661 SocketAddress getRandomLocalInterfaceAddr() {
662 if (localInterfaceAddrs.length == 0) {
663 return null;
664 }
665 final int idx = r.nextInt(localInterfaceAddrs.length);
666 final SocketAddress addr = localInterfaceAddrs[idx];
667 if (LOG.isDebugEnabled()) {
668 LOG.debug("Using local interface " + addr);
669 }
670 return addr;
671 }
672
673 /**
674 * Return the number of times the client should go back to the namenode
675 * to retrieve block locations when reading.
676 */
677 int getMaxBlockAcquireFailures() {
678 return dfsClientConf.maxBlockAcquireFailures;
679 }
680
681 /**
682 * Return the timeout that clients should use when writing to datanodes.
683 * @param numNodes the number of nodes in the pipeline.
684 */
685 int getDatanodeWriteTimeout(int numNodes) {
686 return (dfsClientConf.confTime > 0) ?
687 (dfsClientConf.confTime + HdfsServerConstants.WRITE_TIMEOUT_EXTENSION * numNodes) : 0;
688 }
689
690 int getDatanodeReadTimeout(int numNodes) {
691 return dfsClientConf.socketTimeout > 0 ?
692 (HdfsServerConstants.READ_TIMEOUT_EXTENSION * numNodes +
693 dfsClientConf.socketTimeout) : 0;
694 }
695
696 int getHdfsTimeout() {
697 return dfsClientConf.hdfsTimeout;
698 }
699
700 @VisibleForTesting
701 public String getClientName() {
702 return clientName;
703 }
704
705 void checkOpen() throws IOException {
706 if (!clientRunning) {
707 IOException result = new IOException("Filesystem closed");
708 throw result;
709 }
710 }
711
712 /** Return the lease renewer instance. The renewer thread won't start
713 * until the first output stream is created. The same instance will
714 * be returned until all output streams are closed.
715 */
716 public LeaseRenewer getLeaseRenewer() throws IOException {
717 return LeaseRenewer.getInstance(authority, ugi, this);
718 }
719
720 /** Get a lease and start automatic renewal */
721 private void beginFileLease(final String src, final DFSOutputStream out)
722 throws IOException {
723 getLeaseRenewer().put(src, out, this);
724 }
725
726 /** Stop renewal of lease for the file. */
727 void endFileLease(final String src) throws IOException {
728 getLeaseRenewer().closeFile(src, this);
729 }
730
731
732 /** Put a file. Only called from LeaseRenewer, where proper locking is
733 * enforced to consistently update its local dfsclients array and
734 * client's filesBeingWritten map.
735 */
736 void putFileBeingWritten(final String src, final DFSOutputStream out) {
737 synchronized(filesBeingWritten) {
738 filesBeingWritten.put(src, out);
739 // update the last lease renewal time only when there was no
740 // writes. once there is one write stream open, the lease renewer
741 // thread keeps it updated well with in anyone's expiration time.
742 if (lastLeaseRenewal == 0) {
743 updateLastLeaseRenewal();
744 }
745 }
746 }
747
748 /** Remove a file. Only called from LeaseRenewer. */
749 void removeFileBeingWritten(final String src) {
750 synchronized(filesBeingWritten) {
751 filesBeingWritten.remove(src);
752 if (filesBeingWritten.isEmpty()) {
753 lastLeaseRenewal = 0;
754 }
755 }
756 }
757
758 /** Is file-being-written map empty? */
759 boolean isFilesBeingWrittenEmpty() {
760 synchronized(filesBeingWritten) {
761 return filesBeingWritten.isEmpty();
762 }
763 }
764
765 /** @return true if the client is running */
766 boolean isClientRunning() {
767 return clientRunning;
768 }
769
770 long getLastLeaseRenewal() {
771 return lastLeaseRenewal;
772 }
773
774 void updateLastLeaseRenewal() {
775 synchronized(filesBeingWritten) {
776 if (filesBeingWritten.isEmpty()) {
777 return;
778 }
779 lastLeaseRenewal = Time.now();
780 }
781 }
782
783 /**
784 * Renew leases.
785 * @return true if lease was renewed. May return false if this
786 * client has been closed or has no files open.
787 **/
788 boolean renewLease() throws IOException {
789 if (clientRunning && !isFilesBeingWrittenEmpty()) {
790 try {
791 namenode.renewLease(clientName);
792 updateLastLeaseRenewal();
793 return true;
794 } catch (IOException e) {
795 // Abort if the lease has already expired.
796 final long elapsed = Time.now() - getLastLeaseRenewal();
797 if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
798 LOG.warn("Failed to renew lease for " + clientName + " for "
799 + (elapsed/1000) + " seconds (>= hard-limit ="
800 + (HdfsConstants.LEASE_HARDLIMIT_PERIOD/1000) + " seconds.) "
801 + "Closing all files being written ...", e);
802 closeAllFilesBeingWritten(true);
803 } else {
804 // Let the lease renewer handle it and retry.
805 throw e;
806 }
807 }
808 }
809 return false;
810 }
811
812 /**
813 * Close connections the Namenode.
814 */
815 void closeConnectionToNamenode() {
816 RPC.stopProxy(namenode);
817 }
818
819 /** Abort and release resources held. Ignore all errors. */
820 void abort() {
821 clientRunning = false;
822 closeAllFilesBeingWritten(true);
823 try {
824 // remove reference to this client and stop the renewer,
825 // if there is no more clients under the renewer.
826 getLeaseRenewer().closeClient(this);
827 } catch (IOException ioe) {
828 LOG.info("Exception occurred while aborting the client " + ioe);
829 }
830 closeConnectionToNamenode();
831 }
832
833 /** Close/abort all files being written. */
834 private void closeAllFilesBeingWritten(final boolean abort) {
835 for(;;) {
836 final String src;
837 final DFSOutputStream out;
838 synchronized(filesBeingWritten) {
839 if (filesBeingWritten.isEmpty()) {
840 return;
841 }
842 src = filesBeingWritten.keySet().iterator().next();
843 out = filesBeingWritten.remove(src);
844 }
845 if (out != null) {
846 try {
847 if (abort) {
848 out.abort();
849 } else {
850 out.close();
851 }
852 } catch(IOException ie) {
853 LOG.error("Failed to " + (abort? "abort": "close") + " file " + src,
854 ie);
855 }
856 }
857 }
858 }
859
860 /**
861 * Close the file system, abandoning all of the leases and files being
862 * created and close connections to the namenode.
863 */
864 @Override
865 public synchronized void close() throws IOException {
866 if(clientRunning) {
867 closeAllFilesBeingWritten(false);
868 clientRunning = false;
869 getLeaseRenewer().closeClient(this);
870 // close connections to the namenode
871 closeConnectionToNamenode();
872 }
873 }
874
875 /**
876 * Get the default block size for this cluster
877 * @return the default block size in bytes
878 */
879 public long getDefaultBlockSize() {
880 return dfsClientConf.defaultBlockSize;
881 }
882
883 /**
884 * @see ClientProtocol#getPreferredBlockSize(String)
885 */
886 public long getBlockSize(String f) throws IOException {
887 try {
888 return namenode.getPreferredBlockSize(f);
889 } catch (IOException ie) {
890 LOG.warn("Problem getting block size", ie);
891 throw ie;
892 }
893 }
894
895 /**
896 * Get server default values for a number of configuration params.
897 * @see ClientProtocol#getServerDefaults()
898 */
899 public FsServerDefaults getServerDefaults() throws IOException {
900 long now = Time.now();
901 if (now - serverDefaultsLastUpdate > SERVER_DEFAULTS_VALIDITY_PERIOD) {
902 serverDefaults = namenode.getServerDefaults();
903 serverDefaultsLastUpdate = now;
904 }
905 return serverDefaults;
906 }
907
908 /**
909 * Get a canonical token service name for this client's tokens. Null should
910 * be returned if the client is not using tokens.
911 * @return the token service for the client
912 */
913 @InterfaceAudience.LimitedPrivate( { "HDFS" })
914 public String getCanonicalServiceName() {
915 return (dtService != null) ? dtService.toString() : null;
916 }
917
918 /**
919 * @see ClientProtocol#getDelegationToken(Text)
920 */
921 public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
922 throws IOException {
923 assert dtService != null;
924 Token<DelegationTokenIdentifier> token =
925 namenode.getDelegationToken(renewer);
926
927 if (token != null) {
928 token.setService(this.dtService);
929 LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(token));
930 } else {
931 LOG.info("Cannot get delegation token from " + renewer);
932 }
933 return token;
934
935 }
936
937 /**
938 * Renew a delegation token
939 * @param token the token to renew
940 * @return the new expiration time
941 * @throws InvalidToken
942 * @throws IOException
943 * @deprecated Use Token.renew instead.
944 */
945 @Deprecated
946 public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
947 throws InvalidToken, IOException {
948 LOG.info("Renewing " + DelegationTokenIdentifier.stringifyToken(token));
949 try {
950 return token.renew(conf);
951 } catch (InterruptedException ie) {
952 throw new RuntimeException("caught interrupted", ie);
953 } catch (RemoteException re) {
954 throw re.unwrapRemoteException(InvalidToken.class,
955 AccessControlException.class);
956 }
957 }
958
959 private static final Map<String, Boolean> localAddrMap = Collections
960 .synchronizedMap(new HashMap<String, Boolean>());
961
962 static boolean isLocalAddress(InetSocketAddress targetAddr) {
963 InetAddress addr = targetAddr.getAddress();
964 Boolean cached = localAddrMap.get(addr.getHostAddress());
965 if (cached != null) {
966 if (LOG.isTraceEnabled()) {
967 LOG.trace("Address " + targetAddr +
968 (cached ? " is local" : " is not local"));
969 }
970 return cached;
971 }
972
973 boolean local = NetUtils.isLocalAddress(addr);
974
975 if (LOG.isTraceEnabled()) {
976 LOG.trace("Address " + targetAddr +
977 (local ? " is local" : " is not local"));
978 }
979 localAddrMap.put(addr.getHostAddress(), local);
980 return local;
981 }
982
983 /**
984 * Should the block access token be refetched on an exception
985 *
986 * @param ex Exception received
987 * @param targetAddr Target datanode address from where exception was received
988 * @return true if block access token has expired or invalid and it should be
989 * refetched
990 */
991 private static boolean tokenRefetchNeeded(IOException ex,
992 InetSocketAddress targetAddr) {
993 /*
994 * Get a new access token and retry. Retry is needed in 2 cases. 1) When
995 * both NN and DN re-started while DFSClient holding a cached access token.
996 * 2) In the case that NN fails to update its access key at pre-set interval
997 * (by a wide margin) and subsequently restarts. In this case, DN
998 * re-registers itself with NN and receives a new access key, but DN will
999 * delete the old access key from its memory since it's considered expired
1000 * based on the estimated expiration date.
1001 */
1002 if (ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken) {
1003 LOG.info("Access token was invalid when connecting to " + targetAddr
1004 + " : " + ex);
1005 return true;
1006 }
1007 return false;
1008 }
1009
1010 /**
1011 * Cancel a delegation token
1012 * @param token the token to cancel
1013 * @throws InvalidToken
1014 * @throws IOException
1015 * @deprecated Use Token.cancel instead.
1016 */
1017 @Deprecated
1018 public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
1019 throws InvalidToken, IOException {
1020 LOG.info("Cancelling " + DelegationTokenIdentifier.stringifyToken(token));
1021 try {
1022 token.cancel(conf);
1023 } catch (InterruptedException ie) {
1024 throw new RuntimeException("caught interrupted", ie);
1025 } catch (RemoteException re) {
1026 throw re.unwrapRemoteException(InvalidToken.class,
1027 AccessControlException.class);
1028 }
1029 }
1030
1031 @InterfaceAudience.Private
1032 public static class Renewer extends TokenRenewer {
1033
1034 static {
1035 //Ensure that HDFS Configuration files are loaded before trying to use
1036 // the renewer.
1037 HdfsConfiguration.init();
1038 }
1039
1040 @Override
1041 public boolean handleKind(Text kind) {
1042 return DelegationTokenIdentifier.HDFS_DELEGATION_KIND.equals(kind);
1043 }
1044
1045 @SuppressWarnings("unchecked")
1046 @Override
1047 public long renew(Token<?> token, Configuration conf) throws IOException {
1048 Token<DelegationTokenIdentifier> delToken =
1049 (Token<DelegationTokenIdentifier>) token;
1050 ClientProtocol nn = getNNProxy(delToken, conf);
1051 try {
1052 return nn.renewDelegationToken(delToken);
1053 } catch (RemoteException re) {
1054 throw re.unwrapRemoteException(InvalidToken.class,
1055 AccessControlException.class);
1056 }
1057 }
1058
1059 @SuppressWarnings("unchecked")
1060 @Override
1061 public void cancel(Token<?> token, Configuration conf) throws IOException {
1062 Token<DelegationTokenIdentifier> delToken =
1063 (Token<DelegationTokenIdentifier>) token;
1064 LOG.info("Cancelling " +
1065 DelegationTokenIdentifier.stringifyToken(delToken));
1066 ClientProtocol nn = getNNProxy(delToken, conf);
1067 try {
1068 nn.cancelDelegationToken(delToken);
1069 } catch (RemoteException re) {
1070 throw re.unwrapRemoteException(InvalidToken.class,
1071 AccessControlException.class);
1072 }
1073 }
1074
1075 private static ClientProtocol getNNProxy(
1076 Token<DelegationTokenIdentifier> token, Configuration conf)
1077 throws IOException {
1078 URI uri = HAUtil.getServiceUriFromToken(HdfsConstants.HDFS_URI_SCHEME,
1079 token);
1080 if (HAUtil.isTokenForLogicalUri(token) &&
1081 !HAUtil.isLogicalUri(conf, uri)) {
1082 // If the token is for a logical nameservice, but the configuration
1083 // we have disagrees about that, we can't actually renew it.
1084 // This can be the case in MR, for example, if the RM doesn't
1085 // have all of the HA clusters configured in its configuration.
1086 throw new IOException("Unable to map logical nameservice URI '" +
1087 uri + "' to a NameNode. Local configuration does not have " +
1088 "a failover proxy provider configured.");
1089 }
1090
1091 NameNodeProxies.ProxyAndInfo<ClientProtocol> info =
1092 NameNodeProxies.createProxy(conf, uri, ClientProtocol.class);
1093 assert info.getDelegationTokenService().equals(token.getService()) :
1094 "Returned service '" + info.getDelegationTokenService().toString() +
1095 "' doesn't match expected service '" +
1096 token.getService().toString() + "'";
1097
1098 return info.getProxy();
1099 }
1100
1101 @Override
1102 public boolean isManaged(Token<?> token) throws IOException {
1103 return true;
1104 }
1105
1106 }
1107
1108 /**
1109 * Report corrupt blocks that were discovered by the client.
1110 * @see ClientProtocol#reportBadBlocks(LocatedBlock[])
1111 */
1112 public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
1113 namenode.reportBadBlocks(blocks);
1114 }
1115
1116 public short getDefaultReplication() {
1117 return dfsClientConf.defaultReplication;
1118 }
1119
1120 public LocatedBlocks getLocatedBlocks(String src, long start)
1121 throws IOException {
1122 return getLocatedBlocks(src, start, dfsClientConf.prefetchSize);
1123 }
1124
1125 /*
1126 * This is just a wrapper around callGetBlockLocations, but non-static so that
1127 * we can stub it out for tests.
1128 */
1129 @VisibleForTesting
1130 public LocatedBlocks getLocatedBlocks(String src, long start, long length)
1131 throws IOException {
1132 return callGetBlockLocations(namenode, src, start, length);
1133 }
1134
1135 /**
1136 * @see ClientProtocol#getBlockLocations(String, long, long)
1137 */
1138 static LocatedBlocks callGetBlockLocations(ClientProtocol namenode,
1139 String src, long start, long length)
1140 throws IOException {
1141 try {
1142 return namenode.getBlockLocations(src, start, length);
1143 } catch(RemoteException re) {
1144 throw re.unwrapRemoteException(AccessControlException.class,
1145 FileNotFoundException.class,
1146 UnresolvedPathException.class);
1147 }
1148 }
1149
1150 /**
1151 * Recover a file's lease
1152 * @param src a file's path
1153 * @return true if the file is already closed
1154 * @throws IOException
1155 */
1156 boolean recoverLease(String src) throws IOException {
1157 checkOpen();
1158
1159 try {
1160 return namenode.recoverLease(src, clientName);
1161 } catch (RemoteException re) {
1162 throw re.unwrapRemoteException(FileNotFoundException.class,
1163 AccessControlException.class,
1164 UnresolvedPathException.class);
1165 }
1166 }
1167
1168 /**
1169 * Get block location info about file
1170 *
1171 * getBlockLocations() returns a list of hostnames that store
1172 * data for a specific file region. It returns a set of hostnames
1173 * for every block within the indicated region.
1174 *
1175 * This function is very useful when writing code that considers
1176 * data-placement when performing operations. For example, the
1177 * MapReduce system tries to schedule tasks on the same machines
1178 * as the data-block the task processes.
1179 */
1180 public BlockLocation[] getBlockLocations(String src, long start,
1181 long length) throws IOException, UnresolvedLinkException {
1182 LocatedBlocks blocks = getLocatedBlocks(src, start, length);
1183 BlockLocation[] locations = DFSUtil.locatedBlocks2Locations(blocks);
1184 HdfsBlockLocation[] hdfsLocations = new HdfsBlockLocation[locations.length];
1185 for (int i = 0; i < locations.length; i++) {
1186 hdfsLocations[i] = new HdfsBlockLocation(locations[i], blocks.get(i));
1187 }
1188 return hdfsLocations;
1189 }
1190
1191 /**
1192 * Get block location information about a list of {@link HdfsBlockLocation}.
1193 * Used by {@link DistributedFileSystem#getFileBlockStorageLocations(List)} to
1194 * get {@link BlockStorageLocation}s for blocks returned by
1195 * {@link DistributedFileSystem#getFileBlockLocations(org.apache.hadoop.fs.FileStatus, long, long)}
1196 * .
1197 *
1198 * This is done by making a round of RPCs to the associated datanodes, asking
1199 * the volume of each block replica. The returned array of
1200 * {@link BlockStorageLocation} expose this information as a
1201 * {@link VolumeId}.
1202 *
1203 * @param blockLocations
1204 * target blocks on which to query volume location information
1205 * @return volumeBlockLocations original block array augmented with additional
1206 * volume location information for each replica.
1207 */
1208 public BlockStorageLocation[] getBlockStorageLocations(
1209 List<BlockLocation> blockLocations) throws IOException,
1210 UnsupportedOperationException, InvalidBlockTokenException {
1211 if (!getConf().getHdfsBlocksMetadataEnabled) {
1212 throw new UnsupportedOperationException("Datanode-side support for " +
1213 "getVolumeBlockLocations() must also be enabled in the client " +
1214 "configuration.");
1215 }
1216 // Downcast blockLocations and fetch out required LocatedBlock(s)
1217 List<LocatedBlock> blocks = new ArrayList<LocatedBlock>();
1218 for (BlockLocation loc : blockLocations) {
1219 if (!(loc instanceof HdfsBlockLocation)) {
1220 throw new ClassCastException("DFSClient#getVolumeBlockLocations " +
1221 "expected to be passed HdfsBlockLocations");
1222 }
1223 HdfsBlockLocation hdfsLoc = (HdfsBlockLocation) loc;
1224 blocks.add(hdfsLoc.getLocatedBlock());
1225 }
1226
1227 // Re-group the LocatedBlocks to be grouped by datanodes, with the values
1228 // a list of the LocatedBlocks on the datanode.
1229 Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks =
1230 new LinkedHashMap<DatanodeInfo, List<LocatedBlock>>();
1231 for (LocatedBlock b : blocks) {
1232 for (DatanodeInfo info : b.getLocations()) {
1233 if (!datanodeBlocks.containsKey(info)) {
1234 datanodeBlocks.put(info, new ArrayList<LocatedBlock>());
1235 }
1236 List<LocatedBlock> l = datanodeBlocks.get(info);
1237 l.add(b);
1238 }
1239 }
1240
1241 // Make RPCs to the datanodes to get volume locations for its replicas
1242 Map<DatanodeInfo, HdfsBlocksMetadata> metadatas = BlockStorageLocationUtil
1243 .queryDatanodesForHdfsBlocksMetadata(conf, datanodeBlocks,
1244 getConf().getFileBlockStorageLocationsNumThreads,
1245 getConf().getFileBlockStorageLocationsTimeoutMs,
1246 getConf().connectToDnViaHostname);
1247
1248 if (LOG.isTraceEnabled()) {
1249 LOG.trace("metadata returned: "
1250 + Joiner.on("\n").withKeyValueSeparator("=").join(metadatas));
1251 }
1252
1253 // Regroup the returned VolumeId metadata to again be grouped by
1254 // LocatedBlock rather than by datanode
1255 Map<LocatedBlock, List<VolumeId>> blockVolumeIds = BlockStorageLocationUtil
1256 .associateVolumeIdsWithBlocks(blocks, metadatas);
1257
1258 // Combine original BlockLocations with new VolumeId information
1259 BlockStorageLocation[] volumeBlockLocations = BlockStorageLocationUtil
1260 .convertToVolumeBlockLocations(blocks, blockVolumeIds);
1261
1262 return volumeBlockLocations;
1263 }
1264
1265 public DFSInputStream open(String src)
1266 throws IOException, UnresolvedLinkException {
1267 return open(src, dfsClientConf.ioBufferSize, true, null);
1268 }
1269
1270 /**
1271 * Create an input stream that obtains a nodelist from the
1272 * namenode, and then reads from all the right places. Creates
1273 * inner subclass of InputStream that does the right out-of-band
1274 * work.
1275 * @deprecated Use {@link #open(String, int, boolean)} instead.
1276 */
1277 @Deprecated
1278 public DFSInputStream open(String src, int buffersize, boolean verifyChecksum,
1279 FileSystem.Statistics stats)
1280 throws IOException, UnresolvedLinkException {
1281 return open(src, buffersize, verifyChecksum);
1282 }
1283
1284
1285 /**
1286 * Create an input stream that obtains a nodelist from the
1287 * namenode, and then reads from all the right places. Creates
1288 * inner subclass of InputStream that does the right out-of-band
1289 * work.
1290 */
1291 public DFSInputStream open(String src, int buffersize, boolean verifyChecksum)
1292 throws IOException, UnresolvedLinkException {
1293 checkOpen();
1294 // Get block info from namenode
1295 return new DFSInputStream(this, src, buffersize, verifyChecksum);
1296 }
1297
1298 /**
1299 * Get the namenode associated with this DFSClient object
1300 * @return the namenode associated with this DFSClient object
1301 */
1302 public ClientProtocol getNamenode() {
1303 return namenode;
1304 }
1305
1306 /**
1307 * Call {@link #create(String, boolean, short, long, Progressable)} with
1308 * default <code>replication</code> and <code>blockSize<code> and null <code>
1309 * progress</code>.
1310 */
1311 public OutputStream create(String src, boolean overwrite)
1312 throws IOException {
1313 return create(src, overwrite, dfsClientConf.defaultReplication,
1314 dfsClientConf.defaultBlockSize, null);
1315 }
1316
1317 /**
1318 * Call {@link #create(String, boolean, short, long, Progressable)} with
1319 * default <code>replication</code> and <code>blockSize<code>.
1320 */
1321 public OutputStream create(String src,
1322 boolean overwrite,
1323 Progressable progress) throws IOException {
1324 return create(src, overwrite, dfsClientConf.defaultReplication,
1325 dfsClientConf.defaultBlockSize, progress);
1326 }
1327
1328 /**
1329 * Call {@link #create(String, boolean, short, long, Progressable)} with
1330 * null <code>progress</code>.
1331 */
1332 public OutputStream create(String src,
1333 boolean overwrite,
1334 short replication,
1335 long blockSize) throws IOException {
1336 return create(src, overwrite, replication, blockSize, null);
1337 }
1338
1339 /**
1340 * Call {@link #create(String, boolean, short, long, Progressable, int)}
1341 * with default bufferSize.
1342 */
1343 public OutputStream create(String src, boolean overwrite, short replication,
1344 long blockSize, Progressable progress) throws IOException {
1345 return create(src, overwrite, replication, blockSize, progress,
1346 dfsClientConf.ioBufferSize);
1347 }
1348
1349 /**
1350 * Call {@link #create(String, FsPermission, EnumSet, short, long,
1351 * Progressable, int, ChecksumOpt)} with default <code>permission</code>
1352 * {@link FsPermission#getFileDefault()}.
1353 *
1354 * @param src File name
1355 * @param overwrite overwrite an existing file if true
1356 * @param replication replication factor for the file
1357 * @param blockSize maximum block size
1358 * @param progress interface for reporting client progress
1359 * @param buffersize underlying buffersize
1360 *
1361 * @return output stream
1362 */
1363 public OutputStream create(String src,
1364 boolean overwrite,
1365 short replication,
1366 long blockSize,
1367 Progressable progress,
1368 int buffersize)
1369 throws IOException {
1370 return create(src, FsPermission.getFileDefault(),
1371 overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
1372 : EnumSet.of(CreateFlag.CREATE), replication, blockSize, progress,
1373 buffersize, null);
1374 }
1375
1376 /**
1377 * Call {@link #create(String, FsPermission, EnumSet, boolean, short,
1378 * long, Progressable, int, ChecksumOpt)} with <code>createParent</code>
1379 * set to true.
1380 */
1381 public DFSOutputStream create(String src,
1382 FsPermission permission,
1383 EnumSet<CreateFlag> flag,
1384 short replication,
1385 long blockSize,
1386 Progressable progress,
1387 int buffersize,
1388 ChecksumOpt checksumOpt)
1389 throws IOException {
1390 return create(src, permission, flag, true,
1391 replication, blockSize, progress, buffersize, checksumOpt, null);
1392 }
1393
1394 /**
1395 * Create a new dfs file with the specified block replication
1396 * with write-progress reporting and return an output stream for writing
1397 * into the file.
1398 *
1399 * @param src File name
1400 * @param permission The permission of the directory being created.
1401 * If null, use default permission {@link FsPermission#getFileDefault()}
1402 * @param flag indicates create a new file or create/overwrite an
1403 * existing file or append to an existing file
1404 * @param createParent create missing parent directory if true
1405 * @param replication block replication
1406 * @param blockSize maximum block size
1407 * @param progress interface for reporting client progress
1408 * @param buffersize underlying buffer size
1409 * @param checksumOpt checksum options
1410 *
1411 * @return output stream
1412 *
1413 * @see ClientProtocol#create(String, FsPermission, String, EnumSetWritable,
1414 * boolean, short, long) for detailed description of exceptions thrown
1415 */
1416 public DFSOutputStream create(String src,
1417 FsPermission permission,
1418 EnumSet<CreateFlag> flag,
1419 boolean createParent,
1420 short replication,
1421 long blockSize,
1422 Progressable progress,
1423 int buffersize,
1424 ChecksumOpt checksumOpt) throws IOException {
1425 return create(src, permission, flag, createParent, replication, blockSize,
1426 progress, buffersize, checksumOpt, null);
1427 }
1428
1429 /**
1430 * Same as {@link #create(String, FsPermission, EnumSet, boolean, short, long,
1431 * Progressable, int, ChecksumOpt)} with the addition of favoredNodes that is
1432 * a hint to where the namenode should place the file blocks.
1433 * The favored nodes hint is not persisted in HDFS. Hence it may be honored
1434 * at the creation time only. HDFS could move the blocks during balancing or
1435 * replication, to move the blocks from favored nodes. A value of null means
1436 * no favored nodes for this create
1437 */
1438 public DFSOutputStream create(String src,
1439 FsPermission permission,
1440 EnumSet<CreateFlag> flag,
1441 boolean createParent,
1442 short replication,
1443 long blockSize,
1444 Progressable progress,
1445 int buffersize,
1446 ChecksumOpt checksumOpt,
1447 InetSocketAddress[] favoredNodes) throws IOException {
1448 checkOpen();
1449 if (permission == null) {
1450 permission = FsPermission.getFileDefault();
1451 }
1452 FsPermission masked = permission.applyUMask(dfsClientConf.uMask);
1453 if(LOG.isDebugEnabled()) {
1454 LOG.debug(src + ": masked=" + masked);
1455 }
1456 String[] favoredNodeStrs = null;
1457 if (favoredNodes != null) {
1458 favoredNodeStrs = new String[favoredNodes.length];
1459 for (int i = 0; i < favoredNodes.length; i++) {
1460 favoredNodeStrs[i] =
1461 favoredNodes[i].getHostName() + ":"
1462 + favoredNodes[i].getPort();
1463 }
1464 }
1465 final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this,
1466 src, masked, flag, createParent, replication, blockSize, progress,
1467 buffersize, dfsClientConf.createChecksum(checksumOpt), favoredNodeStrs);
1468 beginFileLease(src, result);
1469 return result;
1470 }
1471
1472 /**
1473 * Append to an existing file if {@link CreateFlag#APPEND} is present
1474 */
1475 private DFSOutputStream primitiveAppend(String src, EnumSet<CreateFlag> flag,
1476 int buffersize, Progressable progress) throws IOException {
1477 if (flag.contains(CreateFlag.APPEND)) {
1478 HdfsFileStatus stat = getFileInfo(src);
1479 if (stat == null) { // No file to append to
1480 // New file needs to be created if create option is present
1481 if (!flag.contains(CreateFlag.CREATE)) {
1482 throw new FileNotFoundException("failed to append to non-existent file "
1483 + src + " on client " + clientName);
1484 }
1485 return null;
1486 }
1487 return callAppend(stat, src, buffersize, progress);
1488 }
1489 return null;
1490 }
1491
1492 /**
1493 * Same as {{@link #create(String, FsPermission, EnumSet, short, long,
1494 * Progressable, int, ChecksumOpt)} except that the permission
1495 * is absolute (ie has already been masked with umask.
1496 */
1497 public DFSOutputStream primitiveCreate(String src,
1498 FsPermission absPermission,
1499 EnumSet<CreateFlag> flag,
1500 boolean createParent,
1501 short replication,
1502 long blockSize,
1503 Progressable progress,
1504 int buffersize,
1505 ChecksumOpt checksumOpt)
1506 throws IOException, UnresolvedLinkException {
1507 checkOpen();
1508 CreateFlag.validate(flag);
1509 DFSOutputStream result = primitiveAppend(src, flag, buffersize, progress);
1510 if (result == null) {
1511 DataChecksum checksum = dfsClientConf.createChecksum(checksumOpt);
1512 result = DFSOutputStream.newStreamForCreate(this, src, absPermission,
1513 flag, createParent, replication, blockSize, progress, buffersize,
1514 checksum);
1515 }
1516 beginFileLease(src, result);
1517 return result;
1518 }
1519
1520 /**
1521 * Creates a symbolic link.
1522 *
1523 * @see ClientProtocol#createSymlink(String, String,FsPermission, boolean)
1524 */
1525 public void createSymlink(String target, String link, boolean createParent)
1526 throws IOException {
1527 try {
1528 FsPermission dirPerm =
1529 FsPermission.getDefault().applyUMask(dfsClientConf.uMask);
1530 namenode.createSymlink(target, link, dirPerm, createParent);
1531 } catch (RemoteException re) {
1532 throw re.unwrapRemoteException(AccessControlException.class,
1533 FileAlreadyExistsException.class,
1534 FileNotFoundException.class,
1535 ParentNotDirectoryException.class,
1536 NSQuotaExceededException.class,
1537 DSQuotaExceededException.class,
1538 UnresolvedPathException.class,
1539 SnapshotAccessControlException.class);
1540 }
1541 }
1542
1543 /**
1544 * Resolve the *first* symlink, if any, in the path.
1545 *
1546 * @see ClientProtocol#getLinkTarget(String)
1547 */
1548 public String getLinkTarget(String path) throws IOException {
1549 checkOpen();
1550 try {
1551 return namenode.getLinkTarget(path);
1552 } catch (RemoteException re) {
1553 throw re.unwrapRemoteException(AccessControlException.class,
1554 FileNotFoundException.class);
1555 }
1556 }
1557
1558 /** Method to get stream returned by append call */
1559 private DFSOutputStream callAppend(HdfsFileStatus stat, String src,
1560 int buffersize, Progressable progress) throws IOException {
1561 LocatedBlock lastBlock = null;
1562 try {
1563 lastBlock = namenode.append(src, clientName);
1564 } catch(RemoteException re) {
1565 throw re.unwrapRemoteException(AccessControlException.class,
1566 FileNotFoundException.class,
1567 SafeModeException.class,
1568 DSQuotaExceededException.class,
1569 UnsupportedOperationException.class,
1570 UnresolvedPathException.class,
1571 SnapshotAccessControlException.class);
1572 }
1573 return DFSOutputStream.newStreamForAppend(this, src, buffersize, progress,
1574 lastBlock, stat, dfsClientConf.createChecksum());
1575 }
1576
1577 /**
1578 * Append to an existing HDFS file.
1579 *
1580 * @param src file name
1581 * @param buffersize buffer size
1582 * @param progress for reporting write-progress; null is acceptable.
1583 * @param statistics file system statistics; null is acceptable.
1584 * @return an output stream for writing into the file
1585 *
1586 * @see ClientProtocol#append(String, String)
1587 */
1588 public HdfsDataOutputStream append(final String src, final int buffersize,
1589 final Progressable progress, final FileSystem.Statistics statistics
1590 ) throws IOException {
1591 final DFSOutputStream out = append(src, buffersize, progress);
1592 return new HdfsDataOutputStream(out, statistics, out.getInitialLen());
1593 }
1594
1595 private DFSOutputStream append(String src, int buffersize, Progressable progress)
1596 throws IOException {
1597 checkOpen();
1598 HdfsFileStatus stat = getFileInfo(src);
1599 if (stat == null) { // No file found
1600 throw new FileNotFoundException("failed to append to non-existent file "
1601 + src + " on client " + clientName);
1602 }
1603 final DFSOutputStream result = callAppend(stat, src, buffersize, progress);
1604 beginFileLease(src, result);
1605 return result;
1606 }
1607
1608 /**
1609 * Set replication for an existing file.
1610 * @param src file name
1611 * @param replication
1612 *
1613 * @see ClientProtocol#setReplication(String, short)
1614 */
1615 public boolean setReplication(String src, short replication)
1616 throws IOException {
1617 try {
1618 return namenode.setReplication(src, replication);
1619 } catch(RemoteException re) {
1620 throw re.unwrapRemoteException(AccessControlException.class,
1621 FileNotFoundException.class,
1622 SafeModeException.class,
1623 DSQuotaExceededException.class,
1624 UnresolvedPathException.class,
1625 SnapshotAccessControlException.class);
1626 }
1627 }
1628
1629 /**
1630 * Rename file or directory.
1631 * @see ClientProtocol#rename(String, String)
1632 * @deprecated Use {@link #rename(String, String, Options.Rename...)} instead.
1633 */
1634 @Deprecated
1635 public boolean rename(String src, String dst) throws IOException {
1636 checkOpen();
1637 try {
1638 return namenode.rename(src, dst);
1639 } catch(RemoteException re) {
1640 throw re.unwrapRemoteException(AccessControlException.class,
1641 NSQuotaExceededException.class,
1642 DSQuotaExceededException.class,
1643 UnresolvedPathException.class,
1644 SnapshotAccessControlException.class);
1645 }
1646 }
1647
1648 /**
1649 * Move blocks from src to trg and delete src
1650 * See {@link ClientProtocol#concat(String, String [])}.
1651 */
1652 public void concat(String trg, String [] srcs) throws IOException {
1653 checkOpen();
1654 try {
1655 namenode.concat(trg, srcs);
1656 } catch(RemoteException re) {
1657 throw re.unwrapRemoteException(AccessControlException.class,
1658 UnresolvedPathException.class,
1659 SnapshotAccessControlException.class);
1660 }
1661 }
1662 /**
1663 * Rename file or directory.
1664 * @see ClientProtocol#rename2(String, String, Options.Rename...)
1665 */
1666 public void rename(String src, String dst, Options.Rename... options)
1667 throws IOException {
1668 checkOpen();
1669 try {
1670 namenode.rename2(src, dst, options);
1671 } catch(RemoteException re) {
1672 throw re.unwrapRemoteException(AccessControlException.class,
1673 DSQuotaExceededException.class,
1674 FileAlreadyExistsException.class,
1675 FileNotFoundException.class,
1676 ParentNotDirectoryException.class,
1677 SafeModeException.class,
1678 NSQuotaExceededException.class,
1679 UnresolvedPathException.class,
1680 SnapshotAccessControlException.class);
1681 }
1682 }
1683 /**
1684 * Delete file or directory.
1685 * See {@link ClientProtocol#delete(String, boolean)}.
1686 */
1687 @Deprecated
1688 public boolean delete(String src) throws IOException {
1689 checkOpen();
1690 return namenode.delete(src, true);
1691 }
1692
1693 /**
1694 * delete file or directory.
1695 * delete contents of the directory if non empty and recursive
1696 * set to true
1697 *
1698 * @see ClientProtocol#delete(String, boolean)
1699 */
1700 public boolean delete(String src, boolean recursive) throws IOException {
1701 checkOpen();
1702 try {
1703 return namenode.delete(src, recursive);
1704 } catch(RemoteException re) {
1705 throw re.unwrapRemoteException(AccessControlException.class,
1706 FileNotFoundException.class,
1707 SafeModeException.class,
1708 UnresolvedPathException.class,
1709 SnapshotAccessControlException.class);
1710 }
1711 }
1712
1713 /** Implemented using getFileInfo(src)
1714 */
1715 public boolean exists(String src) throws IOException {
1716 checkOpen();
1717 return getFileInfo(src) != null;
1718 }
1719
1720 /**
1721 * Get a partial listing of the indicated directory
1722 * No block locations need to be fetched
1723 */
1724 public DirectoryListing listPaths(String src, byte[] startAfter)
1725 throws IOException {
1726 return listPaths(src, startAfter, false);
1727 }
1728
1729 /**
1730 * Get a partial listing of the indicated directory
1731 *
1732 * Recommend to use HdfsFileStatus.EMPTY_NAME as startAfter
1733 * if the application wants to fetch a listing starting from
1734 * the first entry in the directory
1735 *
1736 * @see ClientProtocol#getListing(String, byte[], boolean)
1737 */
1738 public DirectoryListing listPaths(String src, byte[] startAfter,
1739 boolean needLocation)
1740 throws IOException {
1741 checkOpen();
1742 try {
1743 return namenode.getListing(src, startAfter, needLocation);
1744 } catch(RemoteException re) {
1745 throw re.unwrapRemoteException(AccessControlException.class,
1746 FileNotFoundException.class,
1747 UnresolvedPathException.class);
1748 }
1749 }
1750
1751 /**
1752 * Get the file info for a specific file or directory.
1753 * @param src The string representation of the path to the file
1754 * @return object containing information regarding the file
1755 * or null if file not found
1756 *
1757 * @see ClientProtocol#getFileInfo(String) for description of exceptions
1758 */
1759 public HdfsFileStatus getFileInfo(String src) throws IOException {
1760 checkOpen();
1761 try {
1762 return namenode.getFileInfo(src);
1763 } catch(RemoteException re) {
1764 throw re.unwrapRemoteException(AccessControlException.class,
1765 FileNotFoundException.class,
1766 UnresolvedPathException.class);
1767 }
1768 }
1769
1770 /**
1771 * Close status of a file
1772 * @return true if file is already closed
1773 */
1774 public boolean isFileClosed(String src) throws IOException{
1775 checkOpen();
1776 try {
1777 return namenode.isFileClosed(src);
1778 } catch(RemoteException re) {
1779 throw re.unwrapRemoteException(AccessControlException.class,
1780 FileNotFoundException.class,
1781 UnresolvedPathException.class);
1782 }
1783 }
1784
1785 /**
1786 * Get the file info for a specific file or directory. If src
1787 * refers to a symlink then the FileStatus of the link is returned.
1788 * @param src path to a file or directory.
1789 *
1790 * For description of exceptions thrown
1791 * @see ClientProtocol#getFileLinkInfo(String)
1792 */
1793 public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
1794 checkOpen();
1795 try {
1796 return namenode.getFileLinkInfo(src);
1797 } catch(RemoteException re) {
1798 throw re.unwrapRemoteException(AccessControlException.class,
1799 UnresolvedPathException.class);
1800 }
1801 }
1802
1803 /**
1804 * Get the checksum of a file.
1805 * @param src The file path
1806 * @return The checksum
1807 * @see DistributedFileSystem#getFileChecksum(Path)
1808 */
1809 public MD5MD5CRC32FileChecksum getFileChecksum(String src) throws IOException {
1810 checkOpen();
1811 return getFileChecksum(src, clientName, namenode, socketFactory,
1812 dfsClientConf.socketTimeout, getDataEncryptionKey(),
1813 dfsClientConf.connectToDnViaHostname);
1814 }
1815
1816 @InterfaceAudience.Private
1817 public void clearDataEncryptionKey() {
1818 LOG.debug("Clearing encryption key");
1819 synchronized (this) {
1820 encryptionKey = null;
1821 }
1822 }
1823
1824 /**
1825 * @return true if data sent between this client and DNs should be encrypted,
1826 * false otherwise.
1827 * @throws IOException in the event of error communicating with the NN
1828 */
1829 boolean shouldEncryptData() throws IOException {
1830 FsServerDefaults d = getServerDefaults();
1831 return d == null ? false : d.getEncryptDataTransfer();
1832 }
1833
1834 @InterfaceAudience.Private
1835 public DataEncryptionKey getDataEncryptionKey()
1836 throws IOException {
1837 if (shouldEncryptData() &&
1838 !this.trustedChannelResolver.isTrusted()) {
1839 synchronized (this) {
1840 if (encryptionKey == null ||
1841 encryptionKey.expiryDate < Time.now()) {
1842 LOG.debug("Getting new encryption token from NN");
1843 encryptionKey = namenode.getDataEncryptionKey();
1844 }
1845 return encryptionKey;
1846 }
1847 } else {
1848 return null;
1849 }
1850 }
1851
1852 /**
1853 * Get the checksum of a file.
1854 * @param src The file path
1855 * @param clientName the name of the client requesting the checksum.
1856 * @param namenode the RPC proxy for the namenode
1857 * @param socketFactory to create sockets to connect to DNs
1858 * @param socketTimeout timeout to use when connecting and waiting for a response
1859 * @param encryptionKey the key needed to communicate with DNs in this cluster
1860 * @param connectToDnViaHostname whether the client should use hostnames instead of IPs
1861 * @return The checksum
1862 */
1863 private static MD5MD5CRC32FileChecksum getFileChecksum(String src,
1864 String clientName,
1865 ClientProtocol namenode, SocketFactory socketFactory, int socketTimeout,
1866 DataEncryptionKey encryptionKey, boolean connectToDnViaHostname)
1867 throws IOException {
1868 //get all block locations
1869 LocatedBlocks blockLocations = callGetBlockLocations(namenode, src, 0, Long.MAX_VALUE);
1870 if (null == blockLocations) {
1871 throw new FileNotFoundException("File does not exist: " + src);
1872 }
1873 List<LocatedBlock> locatedblocks = blockLocations.getLocatedBlocks();
1874 final DataOutputBuffer md5out = new DataOutputBuffer();
1875 int bytesPerCRC = -1;
1876 DataChecksum.Type crcType = DataChecksum.Type.DEFAULT;
1877 long crcPerBlock = 0;
1878 boolean refetchBlocks = false;
1879 int lastRetriedIndex = -1;
1880
1881 //get block checksum for each block
1882 for(int i = 0; i < locatedblocks.size(); i++) {
1883 if (refetchBlocks) { // refetch to get fresh tokens
1884 blockLocations = callGetBlockLocations(namenode, src, 0, Long.MAX_VALUE);
1885 if (null == blockLocations) {
1886 throw new FileNotFoundException("File does not exist: " + src);
1887 }
1888 locatedblocks = blockLocations.getLocatedBlocks();
1889 refetchBlocks = false;
1890 }
1891 LocatedBlock lb = locatedblocks.get(i);
1892 final ExtendedBlock block = lb.getBlock();
1893 final DatanodeInfo[] datanodes = lb.getLocations();
1894
1895 //try each datanode location of the block
1896 final int timeout = 3000 * datanodes.length + socketTimeout;
1897 boolean done = false;
1898 for(int j = 0; !done && j < datanodes.length; j++) {
1899 DataOutputStream out = null;
1900 DataInputStream in = null;
1901
1902 try {
1903 //connect to a datanode
1904 IOStreamPair pair = connectToDN(socketFactory, connectToDnViaHostname,
1905 encryptionKey, datanodes[j], timeout);
1906 out = new DataOutputStream(new BufferedOutputStream(pair.out,
1907 HdfsConstants.SMALL_BUFFER_SIZE));
1908 in = new DataInputStream(pair.in);
1909
1910 if (LOG.isDebugEnabled()) {
1911 LOG.debug("write to " + datanodes[j] + ": "
1912 + Op.BLOCK_CHECKSUM + ", block=" + block);
1913 }
1914 // get block MD5
1915 new Sender(out).blockChecksum(block, lb.getBlockToken());
1916
1917 final BlockOpResponseProto reply =
1918 BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in));
1919
1920 if (reply.getStatus() != Status.SUCCESS) {
1921 if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN) {
1922 throw new InvalidBlockTokenException();
1923 } else {
1924 throw new IOException("Bad response " + reply + " for block "
1925 + block + " from datanode " + datanodes[j]);
1926 }
1927 }
1928
1929 OpBlockChecksumResponseProto checksumData =
1930 reply.getChecksumResponse();
1931
1932 //read byte-per-checksum
1933 final int bpc = checksumData.getBytesPerCrc();
1934 if (i == 0) { //first block
1935 bytesPerCRC = bpc;
1936 }
1937 else if (bpc != bytesPerCRC) {
1938 throw new IOException("Byte-per-checksum not matched: bpc=" + bpc
1939 + " but bytesPerCRC=" + bytesPerCRC);
1940 }
1941
1942 //read crc-per-block
1943 final long cpb = checksumData.getCrcPerBlock();
1944 if (locatedblocks.size() > 1 && i == 0) {
1945 crcPerBlock = cpb;
1946 }
1947
1948 //read md5
1949 final MD5Hash md5 = new MD5Hash(
1950 checksumData.getMd5().toByteArray());
1951 md5.write(md5out);
1952
1953 // read crc-type
1954 final DataChecksum.Type ct;
1955 if (checksumData.hasCrcType()) {
1956 ct = PBHelper.convert(checksumData
1957 .getCrcType());
1958 } else {
1959 LOG.debug("Retrieving checksum from an earlier-version DataNode: " +
1960 "inferring checksum by reading first byte");
1961 ct = inferChecksumTypeByReading(
1962 clientName, socketFactory, socketTimeout, lb, datanodes[j],
1963 encryptionKey, connectToDnViaHostname);
1964 }
1965
1966 if (i == 0) { // first block
1967 crcType = ct;
1968 } else if (crcType != DataChecksum.Type.MIXED
1969 && crcType != ct) {
1970 // if crc types are mixed in a file
1971 crcType = DataChecksum.Type.MIXED;
1972 }
1973
1974 done = true;
1975
1976 if (LOG.isDebugEnabled()) {
1977 if (i == 0) {
1978 LOG.debug("set bytesPerCRC=" + bytesPerCRC
1979 + ", crcPerBlock=" + crcPerBlock);
1980 }
1981 LOG.debug("got reply from " + datanodes[j] + ": md5=" + md5);
1982 }
1983 } catch (InvalidBlockTokenException ibte) {
1984 if (i > lastRetriedIndex) {
1985 if (LOG.isDebugEnabled()) {
1986 LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
1987 + "for file " + src + " for block " + block
1988 + " from datanode " + datanodes[j]
1989 + ". Will retry the block once.");
1990 }
1991 lastRetriedIndex = i;
1992 done = true; // actually it's not done; but we'll retry
1993 i--; // repeat at i-th block
1994 refetchBlocks = true;
1995 break;
1996 }
1997 } catch (IOException ie) {
1998 LOG.warn("src=" + src + ", datanodes["+j+"]=" + datanodes[j], ie);
1999 } finally {
2000 IOUtils.closeStream(in);
2001 IOUtils.closeStream(out);
2002 }
2003 }
2004
2005 if (!done) {
2006 throw new IOException("Fail to get block MD5 for " + block);
2007 }
2008 }
2009
2010 //compute file MD5
2011 final MD5Hash fileMD5 = MD5Hash.digest(md5out.getData());
2012 switch (crcType) {
2013 case CRC32:
2014 return new MD5MD5CRC32GzipFileChecksum(bytesPerCRC,
2015 crcPerBlock, fileMD5);
2016 case CRC32C:
2017 return new MD5MD5CRC32CastagnoliFileChecksum(bytesPerCRC,
2018 crcPerBlock, fileMD5);
2019 default:
2020 // If there is no block allocated for the file,
2021 // return one with the magic entry that matches what previous
2022 // hdfs versions return.
2023 if (locatedblocks.size() == 0) {
2024 return new MD5MD5CRC32GzipFileChecksum(0, 0, fileMD5);
2025 }
2026
2027 // we should never get here since the validity was checked
2028 // when getCrcType() was called above.
2029 return null;
2030 }
2031 }
2032
2033 /**
2034 * Connect to the given datanode's datantrasfer port, and return
2035 * the resulting IOStreamPair. This includes encryption wrapping, etc.
2036 */
2037 private static IOStreamPair connectToDN(
2038 SocketFactory socketFactory, boolean connectToDnViaHostname,
2039 DataEncryptionKey encryptionKey, DatanodeInfo dn, int timeout)
2040 throws IOException
2041 {
2042 boolean success = false;
2043 Socket sock = null;
2044 try {
2045 sock = socketFactory.createSocket();
2046 String dnAddr = dn.getXferAddr(connectToDnViaHostname);
2047 if (LOG.isDebugEnabled()) {
2048 LOG.debug("Connecting to datanode " + dnAddr);
2049 }
2050 NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr), timeout);
2051 sock.setSoTimeout(timeout);
2052
2053 OutputStream unbufOut = NetUtils.getOutputStream(sock);
2054 InputStream unbufIn = NetUtils.getInputStream(sock);
2055 IOStreamPair ret;
2056 if (encryptionKey != null) {
2057 ret = DataTransferEncryptor.getEncryptedStreams(
2058 unbufOut, unbufIn, encryptionKey);
2059 } else {
2060 ret = new IOStreamPair(unbufIn, unbufOut);
2061 }
2062 success = true;
2063 return ret;
2064 } finally {
2065 if (!success) {
2066 IOUtils.closeSocket(sock);
2067 }
2068 }
2069 }
2070
2071 /**
2072 * Infer the checksum type for a replica by sending an OP_READ_BLOCK
2073 * for the first byte of that replica. This is used for compatibility
2074 * with older HDFS versions which did not include the checksum type in
2075 * OpBlockChecksumResponseProto.
2076 *
2077 * @param in input stream from datanode
2078 * @param out output stream to datanode
2079 * @param lb the located block
2080 * @param clientName the name of the DFSClient requesting the checksum
2081 * @param dn the connected datanode
2082 * @return the inferred checksum type
2083 * @throws IOException if an error occurs
2084 */
2085 private static Type inferChecksumTypeByReading(
2086 String clientName, SocketFactory socketFactory, int socketTimeout,
2087 LocatedBlock lb, DatanodeInfo dn,
2088 DataEncryptionKey encryptionKey, boolean connectToDnViaHostname)
2089 throws IOException {
2090 IOStreamPair pair = connectToDN(socketFactory, connectToDnViaHostname,
2091 encryptionKey, dn, socketTimeout);
2092
2093 try {
2094 DataOutputStream out = new DataOutputStream(new BufferedOutputStream(pair.out,
2095 HdfsConstants.SMALL_BUFFER_SIZE));
2096 DataInputStream in = new DataInputStream(pair.in);
2097
2098 new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName,
2099 0, 1, true, CachingStrategy.newDefaultStrategy());
2100 final BlockOpResponseProto reply =
2101 BlockOpResponseProto.parseFrom(PBHelper.vintPrefixed(in));
2102
2103 if (reply.getStatus() != Status.SUCCESS) {
2104 if (reply.getStatus() == Status.ERROR_ACCESS_TOKEN) {
2105 throw new InvalidBlockTokenException();
2106 } else {
2107 throw new IOException("Bad response " + reply + " trying to read "
2108 + lb.getBlock() + " from datanode " + dn);
2109 }
2110 }
2111
2112 return PBHelper.convert(reply.getReadOpChecksumInfo().getChecksum().getType());
2113 } finally {
2114 IOUtils.cleanup(null, pair.in, pair.out);
2115 }
2116 }
2117
2118 /**
2119 * Set permissions to a file or directory.
2120 * @param src path name.
2121 * @param permission
2122 *
2123 * @see ClientProtocol#setPermission(String, FsPermission)
2124 */
2125 public void setPermission(String src, FsPermission permission)
2126 throws IOException {
2127 checkOpen();
2128 try {
2129 namenode.setPermission(src, permission);
2130 } catch(RemoteException re) {
2131 throw re.unwrapRemoteException(AccessControlException.class,
2132 FileNotFoundException.class,
2133 SafeModeException.class,
2134 UnresolvedPathException.class,
2135 SnapshotAccessControlException.class);
2136 }
2137 }
2138
2139 /**
2140 * Set file or directory owner.
2141 * @param src path name.
2142 * @param username user id.
2143 * @param groupname user group.
2144 *
2145 * @see ClientProtocol#setOwner(String, String, String)
2146 */
2147 public void setOwner(String src, String username, String groupname)
2148 throws IOException {
2149 checkOpen();
2150 try {
2151 namenode.setOwner(src, username, groupname);
2152 } catch(RemoteException re) {
2153 throw re.unwrapRemoteException(AccessControlException.class,
2154 FileNotFoundException.class,
2155 SafeModeException.class,
2156 UnresolvedPathException.class,
2157 SnapshotAccessControlException.class);
2158 }
2159 }
2160
2161 /**
2162 * @see ClientProtocol#getStats()
2163 */
2164 public FsStatus getDiskStatus() throws IOException {
2165 long rawNums[] = namenode.getStats();
2166 return new FsStatus(rawNums[0], rawNums[1], rawNums[2]);
2167 }
2168
2169 /**
2170 * Returns count of blocks with no good replicas left. Normally should be
2171 * zero.
2172 * @throws IOException
2173 */
2174 public long getMissingBlocksCount() throws IOException {
2175 return namenode.getStats()[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX];
2176 }
2177
2178 /**
2179 * Returns count of blocks with one of more replica missing.
2180 * @throws IOException
2181 */
2182 public long getUnderReplicatedBlocksCount() throws IOException {
2183 return namenode.getStats()[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX];
2184 }
2185
2186 /**
2187 * Returns count of blocks with at least one replica marked corrupt.
2188 * @throws IOException
2189 */
2190 public long getCorruptBlocksCount() throws IOException {
2191 return namenode.getStats()[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX];
2192 }
2193
2194 /**
2195 * @return a list in which each entry describes a corrupt file/block
2196 * @throws IOException
2197 */
2198 public CorruptFileBlocks listCorruptFileBlocks(String path,
2199 String cookie)
2200 throws IOException {
2201 return namenode.listCorruptFileBlocks(path, cookie);
2202 }
2203
2204 public DatanodeInfo[] datanodeReport(DatanodeReportType type)
2205 throws IOException {
2206 return namenode.getDatanodeReport(type);
2207 }
2208
2209 /**
2210 * Enter, leave or get safe mode.
2211 *
2212 * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction,boolean)
2213 */
2214 public boolean setSafeMode(SafeModeAction action) throws IOException {
2215 return setSafeMode(action, false);
2216 }
2217
2218 /**
2219 * Enter, leave or get safe mode.
2220 *
2221 * @param action
2222 * One of SafeModeAction.GET, SafeModeAction.ENTER and
2223 * SafeModeActiob.LEAVE
2224 * @param isChecked
2225 * If true, then check only active namenode's safemode status, else
2226 * check first namenode's status.
2227 * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean)
2228 */
2229 public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
2230 return namenode.setSafeMode(action, isChecked);
2231 }
2232
2233 /**
2234 * Create one snapshot.
2235 *
2236 * @param snapshotRoot The directory where the snapshot is to be taken
2237 * @param snapshotName Name of the snapshot
2238 * @return the snapshot path.
2239 * @see ClientProtocol#createSnapshot(String, String)
2240 */
2241 public String createSnapshot(String snapshotRoot, String snapshotName)
2242 throws IOException {
2243 checkOpen();
2244 try {
2245 return namenode.createSnapshot(snapshotRoot, snapshotName);
2246 } catch(RemoteException re) {
2247 throw re.unwrapRemoteException();
2248 }
2249 }
2250
2251 /**
2252 * Delete a snapshot of a snapshottable directory.
2253 *
2254 * @param snapshotRoot The snapshottable directory that the
2255 * to-be-deleted snapshot belongs to
2256 * @param snapshotName The name of the to-be-deleted snapshot
2257 * @throws IOException
2258 * @see ClientProtocol#deleteSnapshot(String, String)
2259 */
2260 public void deleteSnapshot(String snapshotRoot, String snapshotName)
2261 throws IOException {
2262 try {
2263 namenode.deleteSnapshot(snapshotRoot, snapshotName);
2264 } catch(RemoteException re) {
2265 throw re.unwrapRemoteException();
2266 }
2267 }
2268
2269 /**
2270 * Rename a snapshot.
2271 * @param snapshotDir The directory path where the snapshot was taken
2272 * @param snapshotOldName Old name of the snapshot
2273 * @param snapshotNewName New name of the snapshot
2274 * @throws IOException
2275 * @see ClientProtocol#renameSnapshot(String, String, String)
2276 */
2277 public void renameSnapshot(String snapshotDir, String snapshotOldName,
2278 String snapshotNewName) throws IOException {
2279 checkOpen();
2280 try {
2281 namenode.renameSnapshot(snapshotDir, snapshotOldName, snapshotNewName);
2282 } catch(RemoteException re) {
2283 throw re.unwrapRemoteException();
2284 }
2285 }
2286
2287 /**
2288 * Get all the current snapshottable directories.
2289 * @return All the current snapshottable directories
2290 * @throws IOException
2291 * @see ClientProtocol#getSnapshottableDirListing()
2292 */
2293 public SnapshottableDirectoryStatus[] getSnapshottableDirListing()
2294 throws IOException {
2295 checkOpen();
2296 try {
2297 return namenode.getSnapshottableDirListing();
2298 } catch(RemoteException re) {
2299 throw re.unwrapRemoteException();
2300 }
2301 }
2302
2303 /**
2304 * Allow snapshot on a directory.
2305 *
2306 * @see ClientProtocol#allowSnapshot(String snapshotRoot)
2307 */
2308 public void allowSnapshot(String snapshotRoot) throws IOException {
2309 checkOpen();
2310 try {
2311 namenode.allowSnapshot(snapshotRoot);
2312 } catch (RemoteException re) {
2313 throw re.unwrapRemoteException();
2314 }
2315 }
2316
2317 /**
2318 * Disallow snapshot on a directory.
2319 *
2320 * @see ClientProtocol#disallowSnapshot(String snapshotRoot)
2321 */
2322 public void disallowSnapshot(String snapshotRoot) throws IOException {
2323 checkOpen();
2324 try {
2325 namenode.disallowSnapshot(snapshotRoot);
2326 } catch (RemoteException re) {
2327 throw re.unwrapRemoteException();
2328 }
2329 }
2330
2331 /**
2332 * Get the difference between two snapshots, or between a snapshot and the
2333 * current tree of a directory.
2334 * @see ClientProtocol#getSnapshotDiffReport(String, String, String)
2335 */
2336 public SnapshotDiffReport getSnapshotDiffReport(String snapshotDir,
2337 String fromSnapshot, String toSnapshot) throws IOException {
2338 checkOpen();
2339 try {
2340 return namenode.getSnapshotDiffReport(snapshotDir,
2341 fromSnapshot, toSnapshot);
2342 } catch(RemoteException re) {
2343 throw re.unwrapRemoteException();
2344 }
2345 }
2346
2347 public long addCacheDirective(
2348 CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
2349 checkOpen();
2350 try {
2351 return namenode.addCacheDirective(info, flags);
2352 } catch (RemoteException re) {
2353 throw re.unwrapRemoteException();
2354 }
2355 }
2356
2357 public void modifyCacheDirective(
2358 CacheDirectiveInfo info, EnumSet<CacheFlag> flags) throws IOException {
2359 checkOpen();
2360 try {
2361 namenode.modifyCacheDirective(info, flags);
2362 } catch (RemoteException re) {
2363 throw re.unwrapRemoteException();
2364 }
2365 }
2366
2367 public void removeCacheDirective(long id)
2368 throws IOException {
2369 checkOpen();
2370 try {
2371 namenode.removeCacheDirective(id);
2372 } catch (RemoteException re) {
2373 throw re.unwrapRemoteException();
2374 }
2375 }
2376
2377 public RemoteIterator<CacheDirectiveEntry> listCacheDirectives(
2378 CacheDirectiveInfo filter) throws IOException {
2379 return new CacheDirectiveIterator(namenode, filter);
2380 }
2381
2382 public void addCachePool(CachePoolInfo info) throws IOException {
2383 checkOpen();
2384 try {
2385 namenode.addCachePool(info);
2386 } catch (RemoteException re) {
2387 throw re.unwrapRemoteException();
2388 }
2389 }
2390
2391 public void modifyCachePool(CachePoolInfo info) throws IOException {
2392 checkOpen();
2393 try {
2394 namenode.modifyCachePool(info);
2395 } catch (RemoteException re) {
2396 throw re.unwrapRemoteException();
2397 }
2398 }
2399
2400 public void removeCachePool(String poolName) throws IOException {
2401 checkOpen();
2402 try {
2403 namenode.removeCachePool(poolName);
2404 } catch (RemoteException re) {
2405 throw re.unwrapRemoteException();
2406 }
2407 }
2408
2409 public RemoteIterator<CachePoolEntry> listCachePools() throws IOException {
2410 return new CachePoolIterator(namenode);
2411 }
2412
2413 /**
2414 * Save namespace image.
2415 *
2416 * @see ClientProtocol#saveNamespace()
2417 */
2418 void saveNamespace() throws AccessControlException, IOException {
2419 try {
2420 namenode.saveNamespace();
2421 } catch(RemoteException re) {
2422 throw re.unwrapRemoteException(AccessControlException.class);
2423 }
2424 }
2425
2426 /**
2427 * Rolls the edit log on the active NameNode.
2428 * @return the txid of the new log segment
2429 *
2430 * @see ClientProtocol#rollEdits()
2431 */
2432 long rollEdits() throws AccessControlException, IOException {
2433 try {
2434 return namenode.rollEdits();
2435 } catch(RemoteException re) {
2436 throw re.unwrapRemoteException(AccessControlException.class);
2437 }
2438 }
2439
2440 @VisibleForTesting
2441 ExtendedBlock getPreviousBlock(String file) {
2442 return filesBeingWritten.get(file).getBlock();
2443 }
2444
2445 /**
2446 * enable/disable restore failed storage.
2447 *
2448 * @see ClientProtocol#restoreFailedStorage(String arg)
2449 */
2450 boolean restoreFailedStorage(String arg)
2451 throws AccessControlException, IOException{
2452 return namenode.restoreFailedStorage(arg);
2453 }
2454
2455 /**
2456 * Refresh the hosts and exclude files. (Rereads them.)
2457 * See {@link ClientProtocol#refreshNodes()}
2458 * for more details.
2459 *
2460 * @see ClientProtocol#refreshNodes()
2461 */
2462 public void refreshNodes() throws IOException {
2463 namenode.refreshNodes();
2464 }
2465
2466 /**
2467 * Dumps DFS data structures into specified file.
2468 *
2469 * @see ClientProtocol#metaSave(String)
2470 */
2471 public void metaSave(String pathname) throws IOException {
2472 namenode.metaSave(pathname);
2473 }
2474
2475 /**
2476 * Requests the namenode to tell all datanodes to use a new, non-persistent
2477 * bandwidth value for dfs.balance.bandwidthPerSec.
2478 * See {@link ClientProtocol#setBalancerBandwidth(long)}
2479 * for more details.
2480 *
2481 * @see ClientProtocol#setBalancerBandwidth(long)
2482 */
2483 public void setBalancerBandwidth(long bandwidth) throws IOException {
2484 namenode.setBalancerBandwidth(bandwidth);
2485 }
2486
2487 /**
2488 * @see ClientProtocol#finalizeUpgrade()
2489 */
2490 public void finalizeUpgrade() throws IOException {
2491 namenode.finalizeUpgrade();
2492 }
2493
2494 RollingUpgradeInfo rollingUpgrade(RollingUpgradeAction action) throws IOException {
2495 return namenode.rollingUpgrade(action);
2496 }
2497
2498 /**
2499 */
2500 @Deprecated
2501 public boolean mkdirs(String src) throws IOException {
2502 return mkdirs(src, null, true);
2503 }
2504
2505 /**
2506 * Create a directory (or hierarchy of directories) with the given
2507 * name and permission.
2508 *
2509 * @param src The path of the directory being created
2510 * @param permission The permission of the directory being created.
2511 * If permission == null, use {@link FsPermission#getDefault()}.
2512 * @param createParent create missing parent directory if true
2513 *
2514 * @return True if the operation success.
2515 *
2516 * @see ClientProtocol#mkdirs(String, FsPermission, boolean)
2517 */
2518 public boolean mkdirs(String src, FsPermission permission,
2519 boolean createParent) throws IOException {
2520 if (permission == null) {
2521 permission = FsPermission.getDefault();
2522 }
2523 FsPermission masked = permission.applyUMask(dfsClientConf.uMask);
2524 return primitiveMkdir(src, masked, createParent);
2525 }
2526
2527 /**
2528 * Same {{@link #mkdirs(String, FsPermission, boolean)} except
2529 * that the permissions has already been masked against umask.
2530 */
2531 public boolean primitiveMkdir(String src, FsPermission absPermission)
2532 throws IOException {
2533 return primitiveMkdir(src, absPermission, true);
2534 }
2535
2536 /**
2537 * Same {{@link #mkdirs(String, FsPermission, boolean)} except
2538 * that the permissions has already been masked against umask.
2539 */
2540 public boolean primitiveMkdir(String src, FsPermission absPermission,
2541 boolean createParent)
2542 throws IOException {
2543 checkOpen();
2544 if (absPermission == null) {
2545 absPermission =
2546 FsPermission.getDefault().applyUMask(dfsClientConf.uMask);
2547 }
2548
2549 if(LOG.isDebugEnabled()) {
2550 LOG.debug(src + ": masked=" + absPermission);
2551 }
2552 try {
2553 return namenode.mkdirs(src, absPermission, createParent);
2554 } catch(RemoteException re) {
2555 throw re.unwrapRemoteException(AccessControlException.class,
2556 InvalidPathException.class,
2557 FileAlreadyExistsException.class,
2558 FileNotFoundException.class,
2559 ParentNotDirectoryException.class,
2560 SafeModeException.class,
2561 NSQuotaExceededException.class,
2562 DSQuotaExceededException.class,
2563 UnresolvedPathException.class,
2564 SnapshotAccessControlException.class);
2565 }
2566 }
2567
2568 /**
2569 * Get {@link ContentSummary} rooted at the specified directory.
2570 * @param path The string representation of the path
2571 *
2572 * @see ClientProtocol#getContentSummary(String)
2573 */
2574 ContentSummary getContentSummary(String src) throws IOException {
2575 try {
2576 return namenode.getContentSummary(src);
2577 } catch(RemoteException re) {
2578 throw re.unwrapRemoteException(AccessControlException.class,
2579 FileNotFoundException.class,
2580 UnresolvedPathException.class);
2581 }
2582 }
2583
2584 /**
2585 * Sets or resets quotas for a directory.
2586 * @see ClientProtocol#setQuota(String, long, long)
2587 */
2588 void setQuota(String src, long namespaceQuota, long diskspaceQuota)
2589 throws IOException {
2590 // sanity check
2591 if ((namespaceQuota <= 0 && namespaceQuota != HdfsConstants.QUOTA_DONT_SET &&
2592 namespaceQuota != HdfsConstants.QUOTA_RESET) ||
2593 (diskspaceQuota <= 0 && diskspaceQuota != HdfsConstants.QUOTA_DONT_SET &&
2594 diskspaceQuota != HdfsConstants.QUOTA_RESET)) {
2595 throw new IllegalArgumentException("Invalid values for quota : " +
2596 namespaceQuota + " and " +
2597 diskspaceQuota);
2598
2599 }
2600 try {
2601 namenode.setQuota(src, namespaceQuota, diskspaceQuota);
2602 } catch(RemoteException re) {
2603 throw re.unwrapRemoteException(AccessControlException.class,
2604 FileNotFoundException.class,
2605 NSQuotaExceededException.class,
2606 DSQuotaExceededException.class,
2607 UnresolvedPathException.class,
2608 SnapshotAccessControlException.class);
2609 }
2610 }
2611
2612 /**
2613 * set the modification and access time of a file
2614 *
2615 * @see ClientProtocol#setTimes(String, long, long)
2616 */
2617 public void setTimes(String src, long mtime, long atime) throws IOException {
2618 checkOpen();
2619 try {
2620 namenode.setTimes(src, mtime, atime);
2621 } catch(RemoteException re) {
2622 throw re.unwrapRemoteException(AccessControlException.class,
2623 FileNotFoundException.class,
2624 UnresolvedPathException.class,
2625 SnapshotAccessControlException.class);
2626 }
2627 }
2628
2629 /**
2630 * @deprecated use {@link HdfsDataInputStream} instead.
2631 */
2632 @Deprecated
2633 public static class DFSDataInputStream extends HdfsDataInputStream {
2634
2635 public DFSDataInputStream(DFSInputStream in) throws IOException {
2636 super(in);
2637 }
2638 }
2639
2640 void reportChecksumFailure(String file, ExtendedBlock blk, DatanodeInfo dn) {
2641 DatanodeInfo [] dnArr = { dn };
2642 LocatedBlock [] lblocks = { new LocatedBlock(blk, dnArr) };
2643 reportChecksumFailure(file, lblocks);
2644 }
2645
2646 // just reports checksum failure and ignores any exception during the report.
2647 void reportChecksumFailure(String file, LocatedBlock lblocks[]) {
2648 try {
2649 reportBadBlocks(lblocks);
2650 } catch (IOException ie) {
2651 LOG.info("Found corruption while reading " + file
2652 + ". Error repairing corrupt blocks. Bad blocks remain.", ie);
2653 }
2654 }
2655
2656 @Override
2657 public String toString() {
2658 return getClass().getSimpleName() + "[clientName=" + clientName
2659 + ", ugi=" + ugi + "]";
2660 }
2661
2662 public CachingStrategy getDefaultReadCachingStrategy() {
2663 return defaultReadCachingStrategy;
2664 }
2665
2666 public CachingStrategy getDefaultWriteCachingStrategy() {
2667 return defaultWriteCachingStrategy;
2668 }
2669
2670 public ClientContext getClientContext() {
2671 return clientContext;
2672 }
2673
2674 public void modifyAclEntries(String src, List<AclEntry> aclSpec)
2675 throws IOException {
2676 checkOpen();
2677 try {
2678 namenode.modifyAclEntries(src, aclSpec);
2679 } catch(RemoteException re) {
2680 throw re.unwrapRemoteException(AccessControlException.class,
2681 AclException.class,
2682 FileNotFoundException.class,
2683 NSQuotaExceededException.class,
2684 SafeModeException.class,
2685 SnapshotAccessControlException.class,
2686 UnresolvedPathException.class);
2687 }
2688 }
2689
2690 public void removeAclEntries(String src, List<AclEntry> aclSpec)
2691 throws IOException {
2692 checkOpen();
2693 try {
2694 namenode.removeAclEntries(src, aclSpec);
2695 } catch(RemoteException re) {
2696 throw re.unwrapRemoteException(AccessControlException.class,
2697 AclException.class,
2698 FileNotFoundException.class,
2699 NSQuotaExceededException.class,
2700 SafeModeException.class,
2701 SnapshotAccessControlException.class,
2702 UnresolvedPathException.class);
2703 }
2704 }
2705
2706 public void removeDefaultAcl(String src) throws IOException {
2707 checkOpen();
2708 try {
2709 namenode.removeDefaultAcl(src);
2710 } catch(RemoteException re) {
2711 throw re.unwrapRemoteException(AccessControlException.class,
2712 AclException.class,
2713 FileNotFoundException.class,
2714 NSQuotaExceededException.class,
2715 SafeModeException.class,
2716 SnapshotAccessControlException.class,
2717 UnresolvedPathException.class);
2718 }
2719 }
2720
2721 public void removeAcl(String src) throws IOException {
2722 checkOpen();
2723 try {
2724 namenode.removeAcl(src);
2725 } catch(RemoteException re) {
2726 throw re.unwrapRemoteException(AccessControlException.class,
2727 AclException.class,
2728 FileNotFoundException.class,
2729 NSQuotaExceededException.class,
2730 SafeModeException.class,
2731 SnapshotAccessControlException.class,
2732 UnresolvedPathException.class);
2733 }
2734 }
2735
2736 public void setAcl(String src, List<AclEntry> aclSpec) throws IOException {
2737 checkOpen();
2738 try {
2739 namenode.setAcl(src, aclSpec);
2740 } catch(RemoteException re) {
2741 throw re.unwrapRemoteException(AccessControlException.class,
2742 AclException.class,
2743 FileNotFoundException.class,
2744 NSQuotaExceededException.class,
2745 SafeModeException.class,
2746 SnapshotAccessControlException.class,
2747 UnresolvedPathException.class);
2748 }
2749 }
2750
2751 public AclStatus getAclStatus(String src) throws IOException {
2752 checkOpen();
2753 try {
2754 return namenode.getAclStatus(src);
2755 } catch(RemoteException re) {
2756 throw re.unwrapRemoteException(AccessControlException.class,
2757 AclException.class,
2758 FileNotFoundException.class,
2759 UnresolvedPathException.class);
2760 }
2761 }
2762
2763 @Override // RemotePeerFactory
2764 public Peer newConnectedPeer(InetSocketAddress addr) throws IOException {
2765 Peer peer = null;
2766 boolean success = false;
2767 Socket sock = null;
2768 try {
2769 sock = socketFactory.createSocket();
2770 NetUtils.connect(sock, addr,
2771 getRandomLocalInterfaceAddr(),
2772 dfsClientConf.socketTimeout);
2773 peer = TcpPeerServer.peerFromSocketAndKey(sock,
2774 getDataEncryptionKey());
2775 success = true;
2776 return peer;
2777 } finally {
2778 if (!success) {
2779 IOUtils.cleanup(LOG, peer);
2780 IOUtils.closeSocket(sock);
2781 }
2782 }
2783 }
2784
2785 /**
2786 * Create hedged reads thread pool, HEDGED_READ_THREAD_POOL, if
2787 * it does not already exist.
2788 * @param num Number of threads for hedged reads thread pool.
2789 * If zero, skip hedged reads thread pool creation.
2790 */
2791 private synchronized void initThreadsNumForHedgedReads(int num) {
2792 if (num <= 0 || HEDGED_READ_THREAD_POOL != null) return;
2793 HEDGED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
2794 TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
2795 new Daemon.DaemonFactory() {
2796 private final AtomicInteger threadIndex =
2797 new AtomicInteger(0);
2798 @Override
2799 public Thread newThread(Runnable r) {
2800 Thread t = super.newThread(r);
2801 t.setName("hedgedRead-" +
2802 threadIndex.getAndIncrement());
2803 return t;
2804 }
2805 },
2806 new ThreadPoolExecutor.CallerRunsPolicy() {
2807
2808 @Override
2809 public void rejectedExecution(Runnable runnable,
2810 ThreadPoolExecutor e) {
2811 LOG.info("Execution rejected, Executing in current thread");
2812 HEDGED_READ_METRIC.incHedgedReadOpsInCurThread();
2813 // will run in the current thread
2814 super.rejectedExecution(runnable, e);
2815 }
2816 });
2817 HEDGED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
2818 if (LOG.isDebugEnabled()) {
2819 LOG.debug("Using hedged reads; pool threads=" + num);
2820 }
2821 }
2822
2823 long getHedgedReadTimeout() {
2824 return this.hedgedReadThresholdMillis;
2825 }
2826
2827 @VisibleForTesting
2828 void setHedgedReadTimeout(long timeoutMillis) {
2829 this.hedgedReadThresholdMillis = timeoutMillis;
2830 }
2831
2832 ThreadPoolExecutor getHedgedReadsThreadPool() {
2833 return HEDGED_READ_THREAD_POOL;
2834 }
2835
2836 boolean isHedgedReadsEnabled() {
2837 return (HEDGED_READ_THREAD_POOL != null) &&
2838 HEDGED_READ_THREAD_POOL.getMaximumPoolSize() > 0;
2839 }
2840
2841 DFSHedgedReadMetrics getHedgedReadMetrics() {
2842 return HEDGED_READ_METRIC;
2843 }
2844 }