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