You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by wh...@apache.org on 2015/09/26 20:17:10 UTC
[01/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and
related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Repository: hadoop
Updated Branches:
refs/heads/branch-2 b46e4ceaf -> 94cbb6d16
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
deleted file mode 100644
index 745ca7e..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
+++ /dev/null
@@ -1,112 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.client;
-
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.EnumSet;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.crypto.CryptoOutputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hdfs.DFSOutputStream;
-
-import com.google.common.base.Preconditions;
-
-/**
- * The Hdfs implementation of {@link FSDataOutputStream}.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class HdfsDataOutputStream extends FSDataOutputStream {
- public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats,
- long startPosition) throws IOException {
- super(out, stats, startPosition);
- }
-
- public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats
- ) throws IOException {
- this(out, stats, 0L);
- }
-
- public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats,
- long startPosition) throws IOException {
- super(out, stats, startPosition);
- Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream,
- "CryptoOutputStream should wrap a DFSOutputStream");
- }
-
- public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats)
- throws IOException {
- this(out, stats, 0L);
- }
-
- /**
- * Get the actual number of replicas of the current block.
- *
- * This can be different from the designated replication factor of the file
- * because the namenode does not maintain replication for the blocks which are
- * currently being written to. Depending on the configuration, the client may
- * continue to write to a block even if a few datanodes in the write pipeline
- * have failed, or the client may add a new datanodes once a datanode has
- * failed.
- *
- * @return the number of valid replicas of the current block
- */
- public synchronized int getCurrentBlockReplication() throws IOException {
- OutputStream wrappedStream = getWrappedStream();
- if (wrappedStream instanceof CryptoOutputStream) {
- wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
- }
- return ((DFSOutputStream) wrappedStream).getCurrentBlockReplication();
- }
-
- /**
- * Sync buffered data to DataNodes (flush to disk devices).
- *
- * @param syncFlags
- * Indicate the detailed semantic and actions of the hsync.
- * @throws IOException
- * @see FSDataOutputStream#hsync()
- */
- public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
- OutputStream wrappedStream = getWrappedStream();
- if (wrappedStream instanceof CryptoOutputStream) {
- ((CryptoOutputStream) wrappedStream).flush();
- wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
- }
- ((DFSOutputStream) wrappedStream).hsync(syncFlags);
- }
-
- public static enum SyncFlag {
-
- /**
- * When doing sync to DataNodes, also update the metadata (block length) in
- * the NameNode.
- */
- UPDATE_LENGTH,
-
- /**
- * Sync the data to DataNode, close the current block, and allocate a new
- * block
- */
- END_BLOCK;
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
deleted file mode 100644
index b41e2c3..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
+++ /dev/null
@@ -1,524 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.client.impl;
-
-import java.io.IOException;
-import java.net.SocketTimeoutException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.DFSClient;
-import org.apache.hadoop.hdfs.DFSOutputStream;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.Time;
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * <p>
- * Used by {@link org.apache.hadoop.hdfs.DFSClient} for renewing file-being-written leases
- * on the namenode.
- * When a file is opened for write (create or append),
- * namenode stores a file lease for recording the identity of the writer.
- * The writer (i.e. the DFSClient) is required to renew the lease periodically.
- * When the lease is not renewed before it expires,
- * the namenode considers the writer as failed and then it may either let
- * another writer to obtain the lease or close the file.
- * </p>
- * <p>
- * This class also provides the following functionality:
- * <ul>
- * <li>
- * It maintains a map from (namenode, user) pairs to lease renewers.
- * The same {@link LeaseRenewer} instance is used for renewing lease
- * for all the {@link org.apache.hadoop.hdfs.DFSClient} to the same namenode and the same user.
- * </li>
- * <li>
- * Each renewer maintains a list of {@link org.apache.hadoop.hdfs.DFSClient}.
- * Periodically the leases for all the clients are renewed.
- * A client is removed from the list when the client is closed.
- * </li>
- * <li>
- * A thread per namenode per user is used by the {@link LeaseRenewer}
- * to renew the leases.
- * </li>
- * </ul>
- * </p>
- */
-@InterfaceAudience.Private
-public class LeaseRenewer {
- static final Log LOG = LogFactory.getLog(LeaseRenewer.class);
-
- static final long LEASE_RENEWER_GRACE_DEFAULT = 60*1000L;
- static final long LEASE_RENEWER_SLEEP_DEFAULT = 1000L;
-
- /** Get a {@link LeaseRenewer} instance */
- public static LeaseRenewer getInstance(final String authority,
- final UserGroupInformation ugi, final DFSClient dfsc) throws IOException {
- final LeaseRenewer r = Factory.INSTANCE.get(authority, ugi);
- r.addClient(dfsc);
- return r;
- }
-
- /**
- * A factory for sharing {@link LeaseRenewer} objects
- * among {@link DFSClient} instances
- * so that there is only one renewer per authority per user.
- */
- private static class Factory {
- private static final Factory INSTANCE = new Factory();
-
- private static class Key {
- /** Namenode info */
- final String authority;
- /** User info */
- final UserGroupInformation ugi;
-
- private Key(final String authority, final UserGroupInformation ugi) {
- if (authority == null) {
- throw new HadoopIllegalArgumentException("authority == null");
- } else if (ugi == null) {
- throw new HadoopIllegalArgumentException("ugi == null");
- }
-
- this.authority = authority;
- this.ugi = ugi;
- }
-
- @Override
- public int hashCode() {
- return authority.hashCode() ^ ugi.hashCode();
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj == this) {
- return true;
- }
- if (obj != null && obj instanceof Key) {
- final Key that = (Key)obj;
- return this.authority.equals(that.authority)
- && this.ugi.equals(that.ugi);
- }
- return false;
- }
-
- @Override
- public String toString() {
- return ugi.getShortUserName() + "@" + authority;
- }
- }
-
- /** A map for per user per namenode renewers. */
- private final Map<Key, LeaseRenewer> renewers = new HashMap<Key, LeaseRenewer>();
-
- /** Get a renewer. */
- private synchronized LeaseRenewer get(final String authority,
- final UserGroupInformation ugi) {
- final Key k = new Key(authority, ugi);
- LeaseRenewer r = renewers.get(k);
- if (r == null) {
- r = new LeaseRenewer(k);
- renewers.put(k, r);
- }
- return r;
- }
-
- /** Remove the given renewer. */
- private synchronized void remove(final LeaseRenewer r) {
- final LeaseRenewer stored = renewers.get(r.factorykey);
- //Since a renewer may expire, the stored renewer can be different.
- if (r == stored) {
- if (!r.clientsRunning()) {
- renewers.remove(r.factorykey);
- }
- }
- }
- }
-
- /** The time in milliseconds that the map became empty. */
- private long emptyTime = Long.MAX_VALUE;
- /** A fixed lease renewal time period in milliseconds */
- private long renewal = HdfsConstants.LEASE_SOFTLIMIT_PERIOD / 2;
-
- /** A daemon for renewing lease */
- private Daemon daemon = null;
- /** Only the daemon with currentId should run. */
- private int currentId = 0;
-
- /**
- * A period in milliseconds that the lease renewer thread should run
- * after the map became empty.
- * In other words,
- * if the map is empty for a time period longer than the grace period,
- * the renewer should terminate.
- */
- private long gracePeriod;
- /**
- * The time period in milliseconds
- * that the renewer sleeps for each iteration.
- */
- private long sleepPeriod;
-
- private final Factory.Key factorykey;
-
- /** A list of clients corresponding to this renewer. */
- private final List<DFSClient> dfsclients = new ArrayList<DFSClient>();
-
- /**
- * A stringified stack trace of the call stack when the Lease Renewer
- * was instantiated. This is only generated if trace-level logging is
- * enabled on this class.
- */
- private final String instantiationTrace;
-
- private LeaseRenewer(Factory.Key factorykey) {
- this.factorykey = factorykey;
- unsyncSetGraceSleepPeriod(LEASE_RENEWER_GRACE_DEFAULT);
-
- if (LOG.isTraceEnabled()) {
- instantiationTrace = StringUtils.stringifyException(
- new Throwable("TRACE"));
- } else {
- instantiationTrace = null;
- }
- }
-
- /** @return the renewal time in milliseconds. */
- private synchronized long getRenewalTime() {
- return renewal;
- }
-
- /** Used for testing only. */
- @VisibleForTesting
- public synchronized void setRenewalTime(final long renewal) {
- this.renewal = renewal;
- }
-
- /** Add a client. */
- private synchronized void addClient(final DFSClient dfsc) {
- for(DFSClient c : dfsclients) {
- if (c == dfsc) {
- //client already exists, nothing to do.
- return;
- }
- }
- //client not found, add it
- dfsclients.add(dfsc);
-
- //update renewal time
- final int hdfsTimeout = dfsc.getConf().getHdfsTimeout();
- if (hdfsTimeout > 0) {
- final long half = hdfsTimeout/2;
- if (half < renewal) {
- this.renewal = half;
- }
- }
- }
-
- private synchronized boolean clientsRunning() {
- for(Iterator<DFSClient> i = dfsclients.iterator(); i.hasNext(); ) {
- if (!i.next().isClientRunning()) {
- i.remove();
- }
- }
- return !dfsclients.isEmpty();
- }
-
- private synchronized long getSleepPeriod() {
- return sleepPeriod;
- }
-
- /** Set the grace period and adjust the sleep period accordingly. */
- synchronized void setGraceSleepPeriod(final long gracePeriod) {
- unsyncSetGraceSleepPeriod(gracePeriod);
- }
-
- private void unsyncSetGraceSleepPeriod(final long gracePeriod) {
- if (gracePeriod < 100L) {
- throw new HadoopIllegalArgumentException(gracePeriod
- + " = gracePeriod < 100ms is too small.");
- }
- this.gracePeriod = gracePeriod;
- final long half = gracePeriod/2;
- this.sleepPeriod = half < LEASE_RENEWER_SLEEP_DEFAULT?
- half: LEASE_RENEWER_SLEEP_DEFAULT;
- }
-
- /** Is the daemon running? */
- synchronized boolean isRunning() {
- return daemon != null && daemon.isAlive();
- }
-
- /** Does this renewer have nothing to renew? */
- public boolean isEmpty() {
- return dfsclients.isEmpty();
- }
-
- /** Used only by tests */
- synchronized String getDaemonName() {
- return daemon.getName();
- }
-
- /** Is the empty period longer than the grace period? */
- private synchronized boolean isRenewerExpired() {
- return emptyTime != Long.MAX_VALUE
- && Time.monotonicNow() - emptyTime > gracePeriod;
- }
-
- public synchronized void put(final long inodeId, final DFSOutputStream out,
- final DFSClient dfsc) {
- if (dfsc.isClientRunning()) {
- if (!isRunning() || isRenewerExpired()) {
- //start a new deamon with a new id.
- final int id = ++currentId;
- daemon = new Daemon(new Runnable() {
- @Override
- public void run() {
- try {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Lease renewer daemon for " + clientsString()
- + " with renew id " + id + " started");
- }
- LeaseRenewer.this.run(id);
- } catch(InterruptedException e) {
- if (LOG.isDebugEnabled()) {
- LOG.debug(LeaseRenewer.this.getClass().getSimpleName()
- + " is interrupted.", e);
- }
- } finally {
- synchronized(LeaseRenewer.this) {
- Factory.INSTANCE.remove(LeaseRenewer.this);
- }
- if (LOG.isDebugEnabled()) {
- LOG.debug("Lease renewer daemon for " + clientsString()
- + " with renew id " + id + " exited");
- }
- }
- }
-
- @Override
- public String toString() {
- return String.valueOf(LeaseRenewer.this);
- }
- });
- daemon.start();
- }
- dfsc.putFileBeingWritten(inodeId, out);
- emptyTime = Long.MAX_VALUE;
- }
- }
-
- @VisibleForTesting
- synchronized void setEmptyTime(long time) {
- emptyTime = time;
- }
-
- /** Close a file. */
- public void closeFile(final long inodeId, final DFSClient dfsc) {
- dfsc.removeFileBeingWritten(inodeId);
-
- synchronized(this) {
- if (dfsc.isFilesBeingWrittenEmpty()) {
- dfsclients.remove(dfsc);
- }
- //update emptyTime if necessary
- if (emptyTime == Long.MAX_VALUE) {
- for(DFSClient c : dfsclients) {
- if (!c.isFilesBeingWrittenEmpty()) {
- //found a non-empty file-being-written map
- return;
- }
- }
- //discover the first time that all file-being-written maps are empty.
- emptyTime = Time.monotonicNow();
- }
- }
- }
-
- /** Close the given client. */
- public synchronized void closeClient(final DFSClient dfsc) {
- dfsclients.remove(dfsc);
- if (dfsclients.isEmpty()) {
- if (!isRunning() || isRenewerExpired()) {
- Factory.INSTANCE.remove(LeaseRenewer.this);
- return;
- }
- if (emptyTime == Long.MAX_VALUE) {
- //discover the first time that the client list is empty.
- emptyTime = Time.monotonicNow();
- }
- }
-
- //update renewal time
- if (renewal == dfsc.getConf().getHdfsTimeout()/2) {
- long min = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
- for(DFSClient c : dfsclients) {
- final int timeout = c.getConf().getHdfsTimeout();
- if (timeout > 0 && timeout < min) {
- min = timeout;
- }
- }
- renewal = min/2;
- }
- }
-
- public void interruptAndJoin() throws InterruptedException {
- Daemon daemonCopy = null;
- synchronized (this) {
- if (isRunning()) {
- daemon.interrupt();
- daemonCopy = daemon;
- }
- }
-
- if (daemonCopy != null) {
- if(LOG.isDebugEnabled()) {
- LOG.debug("Wait for lease checker to terminate");
- }
- daemonCopy.join();
- }
- }
-
- private void renew() throws IOException {
- final List<DFSClient> copies;
- synchronized(this) {
- copies = new ArrayList<DFSClient>(dfsclients);
- }
- //sort the client names for finding out repeated names.
- Collections.sort(copies, new Comparator<DFSClient>() {
- @Override
- public int compare(final DFSClient left, final DFSClient right) {
- return left.getClientName().compareTo(right.getClientName());
- }
- });
- String previousName = "";
- for(int i = 0; i < copies.size(); i++) {
- final DFSClient c = copies.get(i);
- //skip if current client name is the same as the previous name.
- if (!c.getClientName().equals(previousName)) {
- if (!c.renewLease()) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Did not renew lease for client " +
- c);
- }
- continue;
- }
- previousName = c.getClientName();
- if (LOG.isDebugEnabled()) {
- LOG.debug("Lease renewed for client " + previousName);
- }
- }
- }
- }
-
- /**
- * Periodically check in with the namenode and renew all the leases
- * when the lease period is half over.
- */
- private void run(final int id) throws InterruptedException {
- for(long lastRenewed = Time.monotonicNow(); !Thread.interrupted();
- Thread.sleep(getSleepPeriod())) {
- final long elapsed = Time.monotonicNow() - lastRenewed;
- if (elapsed >= getRenewalTime()) {
- try {
- renew();
- if (LOG.isDebugEnabled()) {
- LOG.debug("Lease renewer daemon for " + clientsString()
- + " with renew id " + id + " executed");
- }
- lastRenewed = Time.monotonicNow();
- } catch (SocketTimeoutException ie) {
- LOG.warn("Failed to renew lease for " + clientsString() + " for "
- + (elapsed/1000) + " seconds. Aborting ...", ie);
- synchronized (this) {
- while (!dfsclients.isEmpty()) {
- DFSClient dfsClient = dfsclients.get(0);
- dfsClient.closeAllFilesBeingWritten(true);
- closeClient(dfsClient);
- }
- //Expire the current LeaseRenewer thread.
- emptyTime = 0;
- }
- break;
- } catch (IOException ie) {
- LOG.warn("Failed to renew lease for " + clientsString() + " for "
- + (elapsed/1000) + " seconds. Will retry shortly ...", ie);
- }
- }
-
- synchronized(this) {
- if (id != currentId || isRenewerExpired()) {
- if (LOG.isDebugEnabled()) {
- if (id != currentId) {
- LOG.debug("Lease renewer daemon for " + clientsString()
- + " with renew id " + id + " is not current");
- } else {
- LOG.debug("Lease renewer daemon for " + clientsString()
- + " with renew id " + id + " expired");
- }
- }
- //no longer the current daemon or expired
- return;
- }
-
- // if no clients are in running state or there is no more clients
- // registered with this renewer, stop the daemon after the grace
- // period.
- if (!clientsRunning() && emptyTime == Long.MAX_VALUE) {
- emptyTime = Time.monotonicNow();
- }
- }
- }
- }
-
- @Override
- public String toString() {
- String s = getClass().getSimpleName() + ":" + factorykey;
- if (LOG.isTraceEnabled()) {
- return s + ", clients=" + clientsString()
- + ", created at " + instantiationTrace;
- }
- return s;
- }
-
- /** Get the names of all clients */
- private synchronized String clientsString() {
- if (dfsclients.isEmpty()) {
- return "[]";
- } else {
- final StringBuilder b = new StringBuilder("[").append(
- dfsclients.get(0).getClientName());
- for(int i = 1; i < dfsclients.size(); i++) {
- b.append(", ").append(dfsclients.get(i).getClientName());
- }
- return b.append("]").toString();
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java
deleted file mode 100644
index e4b51c5..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.inotify;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class MissingEventsException extends Exception {
- private static final long serialVersionUID = 1L;
-
- private long expectedTxid;
- private long actualTxid;
-
- public MissingEventsException() {}
-
- public MissingEventsException(long expectedTxid, long actualTxid) {
- this.expectedTxid = expectedTxid;
- this.actualTxid = actualTxid;
- }
-
- public long getExpectedTxid() {
- return expectedTxid;
- }
-
- public long getActualTxid() {
- return actualTxid;
- }
-
- @Override
- public String toString() {
- return "We expected the next batch of events to start with transaction ID "
- + expectedTxid + ", but it instead started with transaction ID " +
- actualTxid + ". Most likely the intervening transactions were cleaned "
- + "up as part of checkpointing.";
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java
deleted file mode 100644
index 1210999..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * Indicates a failure manipulating an ACL.
- */
-@InterfaceAudience.Private
-public class AclException extends IOException {
- private static final long serialVersionUID = 1L;
-
- /**
- * Creates a new AclException.
- *
- * @param message String message
- */
- public AclException(String message) {
- super(message);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
deleted file mode 100644
index 923cdb4..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.BatchedRemoteIterator;
-import org.apache.hadoop.fs.InvalidRequestException;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-import com.google.common.base.Preconditions;
-
-/**
- * CacheDirectiveIterator is a remote iterator that iterates cache directives.
- * It supports retrying in case of namenode failover.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class CacheDirectiveIterator
- extends BatchedRemoteIterator<Long, CacheDirectiveEntry> {
-
- private CacheDirectiveInfo filter;
- private final ClientProtocol namenode;
- private final Sampler<?> traceSampler;
-
- public CacheDirectiveIterator(ClientProtocol namenode,
- CacheDirectiveInfo filter, Sampler<?> traceSampler) {
- super(0L);
- this.namenode = namenode;
- this.filter = filter;
- this.traceSampler = traceSampler;
- }
-
- private static CacheDirectiveInfo removeIdFromFilter(CacheDirectiveInfo filter) {
- CacheDirectiveInfo.Builder builder = new CacheDirectiveInfo.Builder(filter);
- builder.setId(null);
- return builder.build();
- }
-
- /**
- * Used for compatibility when communicating with a server version that
- * does not support filtering directives by ID.
- */
- private static class SingleEntry implements
- BatchedEntries<CacheDirectiveEntry> {
-
- private final CacheDirectiveEntry entry;
-
- public SingleEntry(final CacheDirectiveEntry entry) {
- this.entry = entry;
- }
-
- @Override
- public CacheDirectiveEntry get(int i) {
- if (i > 0) {
- return null;
- }
- return entry;
- }
-
- @Override
- public int size() {
- return 1;
- }
-
- @Override
- public boolean hasMore() {
- return false;
- }
- }
-
- @Override
- public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
- throws IOException {
- BatchedEntries<CacheDirectiveEntry> entries = null;
- TraceScope scope = Trace.startSpan("listCacheDirectives", traceSampler);
- try {
- entries = namenode.listCacheDirectives(prevKey, filter);
- } catch (IOException e) {
- if (e.getMessage().contains("Filtering by ID is unsupported")) {
- // Retry case for old servers, do the filtering client-side
- long id = filter.getId();
- filter = removeIdFromFilter(filter);
- // Using id - 1 as prevId should get us a window containing the id
- // This is somewhat brittle, since it depends on directives being
- // returned in order of ascending ID.
- entries = namenode.listCacheDirectives(id - 1, filter);
- for (int i=0; i<entries.size(); i++) {
- CacheDirectiveEntry entry = entries.get(i);
- if (entry.getInfo().getId().equals((Long)id)) {
- return new SingleEntry(entry);
- }
- }
- throw new RemoteException(InvalidRequestException.class.getName(),
- "Did not find requested id " + id);
- }
- throw e;
- } finally {
- scope.close();
- }
- Preconditions.checkNotNull(entries);
- return entries;
- }
-
- @Override
- public Long elementToPrevKey(CacheDirectiveEntry entry) {
- return entry.getInfo().getId();
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
deleted file mode 100644
index e9481f7..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.BatchedRemoteIterator;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-/**
- * CachePoolIterator is a remote iterator that iterates cache pools.
- * It supports retrying in case of namenode failover.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class CachePoolIterator
- extends BatchedRemoteIterator<String, CachePoolEntry> {
-
- private final ClientProtocol namenode;
- private final Sampler traceSampler;
-
- public CachePoolIterator(ClientProtocol namenode, Sampler traceSampler) {
- super("");
- this.namenode = namenode;
- this.traceSampler = traceSampler;
- }
-
- @Override
- public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
- throws IOException {
- TraceScope scope = Trace.startSpan("listCachePools", traceSampler);
- try {
- return namenode.listCachePools(prevKey);
- } finally {
- scope.close();
- }
- }
-
- @Override
- public String elementToPrevKey(CachePoolEntry entry) {
- return entry.getInfo().getPoolName();
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
deleted file mode 100644
index 0141215..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.BatchedRemoteIterator;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-/**
- * EncryptionZoneIterator is a remote iterator that iterates over encryption
- * zones. It supports retrying in case of namenode failover.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class EncryptionZoneIterator
- extends BatchedRemoteIterator<Long, EncryptionZone> {
-
- private final ClientProtocol namenode;
- private final Sampler<?> traceSampler;
-
- public EncryptionZoneIterator(ClientProtocol namenode,
- Sampler<?> traceSampler) {
- super(Long.valueOf(0));
- this.namenode = namenode;
- this.traceSampler = traceSampler;
- }
-
- @Override
- public BatchedEntries<EncryptionZone> makeRequest(Long prevId)
- throws IOException {
- TraceScope scope = Trace.startSpan("listEncryptionZones", traceSampler);
- try {
- return namenode.listEncryptionZones(prevId);
- } finally {
- scope.close();
- }
- }
-
- @Override
- public Long elementToPrevKey(EncryptionZone entry) {
- return entry.getId();
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java
deleted file mode 100644
index 25084c7..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.protocol;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.StorageType;
-
-import static org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix.long2String;
-
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class QuotaByStorageTypeExceededException extends QuotaExceededException {
- protected static final long serialVersionUID = 1L;
- protected StorageType type;
-
- public QuotaByStorageTypeExceededException() {}
-
- public QuotaByStorageTypeExceededException(String msg) {
- super(msg);
- }
-
- public QuotaByStorageTypeExceededException(long quota, long count, StorageType type) {
- super(quota, count);
- this.type = type;
- }
-
- @Override
- public String getMessage() {
- String msg = super.getMessage();
- if (msg == null) {
- return "Quota by storage type : " + type.toString() +
- " on path : " + (pathName==null ? "": pathName) +
- " is exceeded. quota = " + long2String(quota, "B", 2) +
- " but space consumed = " + long2String(count, "B", 2);
- } else {
- return msg;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
deleted file mode 100644
index 03fb704..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.protocol;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.Path;
-
-/**
- * Thrown when a symbolic link is encountered in a path.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public final class UnresolvedPathException extends UnresolvedLinkException {
- private static final long serialVersionUID = 1L;
- private String path; // The path containing the link
- private String preceding; // The path part preceding the link
- private String remainder; // The path part following the link
- private String linkTarget; // The link's target
-
- /**
- * Used by RemoteException to instantiate an UnresolvedPathException.
- */
- public UnresolvedPathException(String msg) {
- super(msg);
- }
-
- public UnresolvedPathException(String path, String preceding,
- String remainder, String linkTarget) {
- this.path = path;
- this.preceding = preceding;
- this.remainder = remainder;
- this.linkTarget = linkTarget;
- }
-
- /**
- * Return a path with the link resolved with the target.
- */
- public Path getResolvedPath() throws IOException {
- // If the path is absolute we cam throw out the preceding part and
- // just append the remainder to the target, otherwise append each
- // piece to resolve the link in path.
- boolean noRemainder = (remainder == null || "".equals(remainder));
- Path target = new Path(linkTarget);
- if (target.isUriPathAbsolute()) {
- return noRemainder ? target : new Path(target, remainder);
- } else {
- return noRemainder
- ? new Path(preceding, target)
- : new Path(new Path(preceding, linkTarget), remainder);
- }
- }
-
- @Override
- public String getMessage() {
- String msg = super.getMessage();
- if (msg != null) {
- return msg;
- }
- String myMsg = "Unresolved path " + path;
- try {
- return getResolvedPath().toString();
- } catch (IOException e) {
- // Ignore
- }
- return myMsg;
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
deleted file mode 100644
index c69986a..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
+++ /dev/null
@@ -1,200 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.protocol.datatransfer;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-
-/**
- * The setting of replace-datanode-on-failure feature.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class ReplaceDatanodeOnFailure {
- /** The replacement policies */
- public enum Policy {
- /** The feature is disabled in the entire site. */
- DISABLE(Condition.FALSE),
- /** Never add a new datanode. */
- NEVER(Condition.FALSE),
- /** @see ReplaceDatanodeOnFailure.Condition#DEFAULT */
- DEFAULT(Condition.DEFAULT),
- /** Always add a new datanode when an existing datanode is removed. */
- ALWAYS(Condition.TRUE);
-
- private final Condition condition;
-
- private Policy(Condition condition) {
- this.condition = condition;
- }
-
- Condition getCondition() {
- return condition;
- }
- }
-
- /** Datanode replacement condition */
- private static interface Condition {
- /** Return true unconditionally. */
- static final Condition TRUE = new Condition() {
- @Override
- public boolean satisfy(short replication, DatanodeInfo[] existings,
- int nExistings, boolean isAppend, boolean isHflushed) {
- return true;
- }
- };
-
- /** Return false unconditionally. */
- static final Condition FALSE = new Condition() {
- @Override
- public boolean satisfy(short replication, DatanodeInfo[] existings,
- int nExistings, boolean isAppend, boolean isHflushed) {
- return false;
- }
- };
-
- /**
- * DEFAULT condition:
- * Let r be the replication number.
- * Let n be the number of existing datanodes.
- * Add a new datanode only if r >= 3 and either
- * (1) floor(r/2) >= n; or
- * (2) r > n and the block is hflushed/appended.
- */
- static final Condition DEFAULT = new Condition() {
- @Override
- public boolean satisfy(final short replication,
- final DatanodeInfo[] existings, final int n, final boolean isAppend,
- final boolean isHflushed) {
- if (replication < 3) {
- return false;
- } else {
- if (n <= (replication/2)) {
- return true;
- } else {
- return isAppend || isHflushed;
- }
- }
- }
- };
-
- /** Is the condition satisfied? */
- public boolean satisfy(short replication, DatanodeInfo[] existings,
- int nExistings, boolean isAppend, boolean isHflushed);
- }
-
- private final Policy policy;
- private final boolean bestEffort;
-
- public ReplaceDatanodeOnFailure(Policy policy, boolean bestEffort) {
- this.policy = policy;
- this.bestEffort = bestEffort;
- }
-
- /** Check if the feature is enabled. */
- public void checkEnabled() {
- if (policy == Policy.DISABLE) {
- throw new UnsupportedOperationException(
- "This feature is disabled. Please refer to "
- + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY
- + " configuration property.");
- }
- }
-
- /**
- * Best effort means that the client will try to replace the failed datanode
- * (provided that the policy is satisfied), however, it will continue the
- * write operation in case that the datanode replacement also fails.
- *
- * @return Suppose the datanode replacement fails.
- * false: An exception should be thrown so that the write will fail.
- * true : The write should be resumed with the remaining datandoes.
- */
- public boolean isBestEffort() {
- return bestEffort;
- }
-
- /** Does it need a replacement according to the policy? */
- public boolean satisfy(
- final short replication, final DatanodeInfo[] existings,
- final boolean isAppend, final boolean isHflushed) {
- final int n = existings == null? 0: existings.length;
- if (n == 0 || n >= replication) {
- //don't need to add datanode for any policy.
- return false;
- } else {
- return policy.getCondition().satisfy(
- replication, existings, n, isAppend, isHflushed);
- }
- }
-
- @Override
- public String toString() {
- return policy.toString();
- }
-
- /** Get the setting from configuration. */
- public static ReplaceDatanodeOnFailure get(final Configuration conf) {
- final Policy policy = getPolicy(conf);
- final boolean bestEffort = conf.getBoolean(
- HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
- HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_DEFAULT);
-
- return new ReplaceDatanodeOnFailure(policy, bestEffort);
- }
-
- private static Policy getPolicy(final Configuration conf) {
- final boolean enabled = conf.getBoolean(
- HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY,
- HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_DEFAULT);
- if (!enabled) {
- return Policy.DISABLE;
- }
-
- final String policy = conf.get(
- HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY,
- HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_DEFAULT);
- for(int i = 1; i < Policy.values().length; i++) {
- final Policy p = Policy.values()[i];
- if (p.name().equalsIgnoreCase(policy)) {
- return p;
- }
- }
- throw new HadoopIllegalArgumentException("Illegal configuration value for "
- + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY
- + ": " + policy);
- }
-
- /** Write the setting to configuration. */
- public static void write(final Policy policy,
- final boolean bestEffort, final Configuration conf) {
- conf.setBoolean(
- HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY,
- policy != Policy.DISABLE);
- conf.set(
- HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY,
- policy.name());
- conf.setBoolean(
- HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
- bestEffort);
- }
-}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
index dc967ff..1afe0af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java
@@ -49,6 +49,7 @@ import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.DistributedFileSystem;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -879,7 +880,7 @@ public class Dispatcher {
this.saslClient = new SaslDataTransferClient(conf,
DataTransferSaslUtil.getSaslPropertiesResolver(conf),
TrustedChannelResolver.getInstance(conf), nnc.fallbackToSimpleAuth);
- this.ioFileBufferSize = DFSUtil.getIoFileBufferSize(conf);
+ this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
}
public DistributedFileSystem getDistributedFileSystem() {
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
index 957b2c7..eec2b2d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java
@@ -38,7 +38,7 @@ import org.apache.commons.logging.Log;
import org.apache.hadoop.fs.ChecksumException;
import org.apache.hadoop.fs.FSOutputSummer;
import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
@@ -248,7 +248,7 @@ class BlockReceiver implements Closeable {
out.getClass());
}
this.checksumOut = new DataOutputStream(new BufferedOutputStream(
- streams.getChecksumOut(), DFSUtil.getSmallBufferSize(
+ streams.getChecksumOut(), DFSUtilClient.getSmallBufferSize(
datanode.getConf())));
// write data chunk header if creating a new replica
if (isCreate) {
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
index fb8b132..d7e62bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java
@@ -34,7 +34,7 @@ import java.util.Arrays;
import org.apache.commons.logging.Log;
import org.apache.hadoop.fs.ChecksumException;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
@@ -111,7 +111,7 @@ class BlockSender implements java.io.Closeable {
private static final int IO_FILE_BUFFER_SIZE;
static {
HdfsConfiguration conf = new HdfsConfiguration();
- IO_FILE_BUFFER_SIZE = DFSUtil.getIoFileBufferSize(conf);
+ IO_FILE_BUFFER_SIZE = DFSUtilClient.getIoFileBufferSize(conf);
}
private static final int TRANSFERTO_BUFFER_SIZE = Math.max(
IO_FILE_BUFFER_SIZE, MIN_BUFFER_WITH_TRANSFERTO);
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
index d20c444..e2d6490 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java
@@ -109,6 +109,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.hdfs.DFSUtil.ConfiguredNNAddress;
+import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.client.BlockReportOptions;
@@ -2186,7 +2187,7 @@ public class DataNode extends ReconfigurableBase
unbufIn = saslStreams.in;
out = new DataOutputStream(new BufferedOutputStream(unbufOut,
- DFSUtil.getSmallBufferSize(conf)));
+ DFSUtilClient.getSmallBufferSize(conf)));
in = new DataInputStream(unbufIn);
blockSender = new BlockSender(b, 0, b.getNumBytes(),
false, false, true, DataNode.this, null, cachingStrategy);
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
index 8c4e38a..665432e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java
@@ -49,7 +49,7 @@ import java.util.Arrays;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.ExtendedBlockId;
import org.apache.hadoop.hdfs.net.Peer;
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
@@ -134,8 +134,8 @@ class DataXceiver extends Receiver implements Runnable {
this.datanode = datanode;
this.dataXceiverServer = dataXceiverServer;
this.connectToDnViaHostname = datanode.getDnConf().connectToDnViaHostname;
- this.ioFileBufferSize = DFSUtil.getIoFileBufferSize(datanode.getConf());
- this.smallBufferSize = DFSUtil.getSmallBufferSize(datanode.getConf());
+ this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(datanode.getConf());
+ this.smallBufferSize = DFSUtilClient.getSmallBufferSize(datanode.getConf());
remoteAddress = peer.getRemoteAddressString();
final int colonIdx = remoteAddress.indexOf(':');
remoteAddressWithoutPort =
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
deleted file mode 100644
index b159d3a..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.hadoop.hdfs.server.datanode;
-
-import java.io.IOException;
-
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-
-/**
- * Exception indicating that DataNode does not have a replica
- * that matches the target block.
- */
-public class ReplicaNotFoundException extends IOException {
- private static final long serialVersionUID = 1L;
- public final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
- public final static String UNFINALIZED_REPLICA =
- "Cannot append to an unfinalized replica ";
- public final static String UNFINALIZED_AND_NONRBW_REPLICA =
- "Cannot recover append/close to a replica that's not FINALIZED and not RBW ";
- public final static String NON_EXISTENT_REPLICA =
- "Cannot append to a non-existent replica ";
- public final static String UNEXPECTED_GS_REPLICA =
- "Cannot append to a replica with unexpected generation stamp ";
-
- public ReplicaNotFoundException() {
- super();
- }
-
- public ReplicaNotFoundException(ExtendedBlock b) {
- super("Replica not found for " + b);
- }
-
- public ReplicaNotFoundException(String msg) {
- super(msg);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
index 667a0b2..e12d69a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/BlockPoolSlice.java
@@ -38,7 +38,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.DU;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.protocol.Block;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
import org.apache.hadoop.hdfs.protocol.BlockListAsLongs.BlockReportReplica;
@@ -111,7 +111,7 @@ class BlockPoolSlice {
}
}
- this.ioFileBufferSize = DFSUtil.getIoFileBufferSize(conf);
+ this.ioFileBufferSize = DFSUtilClient.getIoFileBufferSize(conf);
this.deleteDuplicateReplicas = conf.getBoolean(
DFSConfigKeys.DFS_DATANODE_DUPLICATE_REPLICA_DELETION,
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
index 0d931c4..37b6b9e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java
@@ -59,7 +59,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.ExtendedBlockId;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.protocol.Block;
@@ -268,8 +268,8 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
this.datanode = datanode;
this.dataStorage = storage;
this.conf = conf;
- this.smallBufferSize = DFSUtil.getSmallBufferSize(conf);
- // The number of volumes required for operation is the total number
+ this.smallBufferSize = DFSUtilClient.getSmallBufferSize(conf);
+ // The number of volumes required for operation is the total number
// of volumes minus the number of failed volumes we can tolerate.
volFailuresTolerated =
conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
@@ -961,7 +961,7 @@ class FsDatasetImpl implements FsDatasetSpi<FsVolumeImpl> {
File blockFile, int smallBufferSize, final Configuration conf)
throws IOException {
final DataChecksum checksum = BlockMetadataHeader.readDataChecksum(srcMeta,
- DFSUtil.getIoFileBufferSize(conf));
+ DFSUtilClient.getIoFileBufferSize(conf));
final byte[] data = new byte[1 << 16];
final byte[] crcs = new byte[checksum.getChecksumSize(data.length)];
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
index bf1fd98..476f596 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/RamDiskAsyncLazyPersistService.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hdfs.server.datanode.fsdataset.impl;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeReference;
@@ -240,7 +240,7 @@ class RamDiskAsyncLazyPersistService {
boolean succeeded = false;
final FsDatasetImpl dataset = (FsDatasetImpl)datanode.getFSDataset();
try (FsVolumeReference ref = this.targetVolume) {
- int smallBufferSize = DFSUtil.getSmallBufferSize(EMPTY_HDFS_CONF);
+ int smallBufferSize = DFSUtilClient.getSmallBufferSize(EMPTY_HDFS_CONF);
// No FsDatasetImpl lock for the file copy
File targetFiles[] = FsDatasetImpl.copyBlockFiles(
blockId, genStamp, metaFile, blockFile, lazyPersistDir, true,
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
deleted file mode 100644
index 0bdd2a5..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.server.namenode;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-@InterfaceAudience.Private
-public class RetryStartFileException extends IOException {
- private static final long serialVersionUID = 1L;
-
- public RetryStartFileException() {
- super("Preconditions for creating a file failed because of a " +
- "transient error, retry create later.");
- }
-
- public RetryStartFileException(String s) {
- super(s);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
index 9783cca..f1b74f4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.DFSUtilClient;
import org.apache.hadoop.hdfs.HdfsConfiguration;
import org.apache.hadoop.hdfs.server.common.Storage;
import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -89,7 +89,7 @@ public class TransferFsImage {
connectionFactory = URLConnectionFactory
.newDefaultURLConnectionFactory(conf);
isSpnegoEnabled = UserGroupInformation.isSecurityEnabled();
- IO_FILE_BUFFER_SIZE = DFSUtil.getIoFileBufferSize(conf);
+ IO_FILE_BUFFER_SIZE = DFSUtilClient.getIoFileBufferSize(conf);
}
private static final Log LOG = LogFactory.getLog(TransferFsImage.class);
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
index 0b0da21..52cd9c1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/aop/org/apache/hadoop/hdfs/server/datanode/TestFiDataTransferProtocol2.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.datanode;
import java.io.IOException;
import java.util.Random;
-import org.apache.commons.logging.impl.Log4JLogger;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fi.DataTransferTestUtil;
import org.apache.hadoop.fi.DataTransferTestUtil.CountdownDoosAction;
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index e324557..b236bff 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -990,7 +990,7 @@ public class DFSTestUtil {
final long writeTimeout = dfsClient.getDatanodeWriteTimeout(datanodes.length);
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
NetUtils.getOutputStream(s, writeTimeout),
- DFSUtil.getSmallBufferSize(dfsClient.getConfiguration())));
+ DFSUtilClient.getSmallBufferSize(dfsClient.getConfiguration())));
final DataInputStream in = new DataInputStream(NetUtils.getInputStream(s));
// send the request
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
index b9682a1..5c011e3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSClusterWithNodeGroup.java
@@ -28,8 +28,8 @@ import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter;
-import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources;
+import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolumeImpl;
import org.apache.hadoop.net.NetUtils;
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index c3f89ea..53b8038 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -36,7 +36,6 @@ import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.hdfs.protocol.*;
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
import org.apache.hadoop.hdfs.server.blockmanagement.*;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
import org.apache.hadoop.hdfs.server.datanode.DataNode;
import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
import org.apache.hadoop.hdfs.server.namenode.NameNode;
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
index ca98441..b83157d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClientProtocolForPipelineRecovery.java
@@ -124,8 +124,8 @@ public class TestClientProtocolForPipelineRecovery {
public void testPipelineRecoveryForLastBlock() throws IOException {
DFSClientFaultInjector faultInjector
= Mockito.mock(DFSClientFaultInjector.class);
- DFSClientFaultInjector oldInjector = DFSClientFaultInjector.instance;
- DFSClientFaultInjector.instance = faultInjector;
+ DFSClientFaultInjector oldInjector = DFSClientFaultInjector.get();
+ DFSClientFaultInjector.set(faultInjector);
Configuration conf = new HdfsConfiguration();
conf.setInt(HdfsClientConfigKeys.BlockWrite.LOCATEFOLLOWINGBLOCK_RETRIES_KEY, 3);
@@ -153,7 +153,7 @@ public class TestClientProtocolForPipelineRecovery {
+ " corrupt replicas.");
}
} finally {
- DFSClientFaultInjector.instance = oldInjector;
+ DFSClientFaultInjector.set(oldInjector);
if (cluster != null) {
cluster.shutdown();
}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java
index 2b11fbd..3850ff2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestCrcCorruption.java
@@ -76,7 +76,7 @@ public class TestCrcCorruption {
@Before
public void setUp() throws IOException {
faultInjector = Mockito.mock(DFSClientFaultInjector.class);
- DFSClientFaultInjector.instance = faultInjector;
+ DFSClientFaultInjector.set(faultInjector);
}
/**
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
index a821c30..3435b7f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUtil.java
@@ -903,16 +903,16 @@ public class TestDFSUtil {
Configuration conf = new Configuration(false);
conf.unset(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI);
assertFalse("encryption enabled on no provider key",
- DFSUtil.isHDFSEncryptionEnabled(conf));
+ DFSUtilClient.isHDFSEncryptionEnabled(conf));
conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "");
assertFalse("encryption enabled on empty provider key",
- DFSUtil.isHDFSEncryptionEnabled(conf));
+ DFSUtilClient.isHDFSEncryptionEnabled(conf));
conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "\n\t\n");
assertFalse("encryption enabled on whitespace provider key",
- DFSUtil.isHDFSEncryptionEnabled(conf));
+ DFSUtilClient.isHDFSEncryptionEnabled(conf));
conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "http://hadoop.apache.org");
assertTrue("encryption disabled on valid provider key",
- DFSUtil.isHDFSEncryptionEnabled(conf));
+ DFSUtilClient.isHDFSEncryptionEnabled(conf));
}
}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
index acb179c..e3dcd8c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatus.java
@@ -90,7 +90,7 @@ public class TestFileStatus {
int fileSize, int blockSize) throws IOException {
// Create and write a file that contains three blocks of data
FSDataOutputStream stm = fileSys.create(name, true,
- DFSUtil.getIoFileBufferSize(conf), (short)repl, (long)blockSize);
+ DFSUtilClient.getIoFileBufferSize(conf), (short)repl, (long)blockSize);
byte[] buffer = new byte[fileSize];
Random rand = new Random(seed);
rand.nextBytes(buffer);
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
index ee3ef89..cc0fb92 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPread.java
@@ -295,9 +295,8 @@ public class TestPread {
hedgedReadTimeoutMillis);
conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 0);
// Set up the InjectionHandler
- DFSClientFaultInjector.instance = Mockito
- .mock(DFSClientFaultInjector.class);
- DFSClientFaultInjector injector = DFSClientFaultInjector.instance;
+ DFSClientFaultInjector.set(Mockito.mock(DFSClientFaultInjector.class));
+ DFSClientFaultInjector injector = DFSClientFaultInjector.get();
final int sleepMs = 100;
Mockito.doAnswer(new Answer<Void>() {
@Override
@@ -371,9 +370,8 @@ public class TestPread {
initialHedgedReadTimeoutMillis);
// Set up the InjectionHandler
- DFSClientFaultInjector.instance = Mockito
- .mock(DFSClientFaultInjector.class);
- DFSClientFaultInjector injector = DFSClientFaultInjector.instance;
+ DFSClientFaultInjector.set(Mockito.mock(DFSClientFaultInjector.class));
+ DFSClientFaultInjector injector = DFSClientFaultInjector.get();
// make preads sleep for 50ms
Mockito.doAnswer(new Answer<Void>() {
@Override
[06/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and
related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
deleted file mode 100644
index c9add53..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
+++ /dev/null
@@ -1,892 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
-
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.net.InetSocketAddress;
-import java.util.List;
-
-import com.google.common.io.ByteArrayDataOutput;
-import com.google.common.io.ByteStreams;
-import org.apache.commons.lang.mutable.MutableBoolean;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
-import org.apache.hadoop.hdfs.net.DomainPeer;
-import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
-import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.shortcircuit.DomainSocketFactory;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.ShortCircuitReplicaCreator;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplicaInfo;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
-import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.unix.DomainSocket;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.PerformanceAdvisory;
-import org.apache.hadoop.util.Time;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-
-/**
- * Utility class to create BlockReader implementations.
- */
-@InterfaceAudience.Private
-public class BlockReaderFactory implements ShortCircuitReplicaCreator {
- static final Log LOG = LogFactory.getLog(BlockReaderFactory.class);
-
- public static class FailureInjector {
- public void injectRequestFileDescriptorsFailure() throws IOException {
- // do nothing
- }
- public boolean getSupportsReceiptVerification() {
- return true;
- }
- }
-
- @VisibleForTesting
- static ShortCircuitReplicaCreator
- createShortCircuitReplicaInfoCallback = null;
-
- private final DfsClientConf conf;
-
- /**
- * Injects failures into specific operations during unit tests.
- */
- private static FailureInjector failureInjector = new FailureInjector();
-
- /**
- * The file name, for logging and debugging purposes.
- */
- private String fileName;
-
- /**
- * The block ID and block pool ID to use.
- */
- private ExtendedBlock block;
-
- /**
- * The block token to use for security purposes.
- */
- private Token<BlockTokenIdentifier> token;
-
- /**
- * The offset within the block to start reading at.
- */
- private long startOffset;
-
- /**
- * If false, we won't try to verify the block checksum.
- */
- private boolean verifyChecksum;
-
- /**
- * The name of this client.
- */
- private String clientName;
-
- /**
- * The DataNode we're talking to.
- */
- private DatanodeInfo datanode;
-
- /**
- * StorageType of replica on DataNode.
- */
- private StorageType storageType;
-
- /**
- * If false, we won't try short-circuit local reads.
- */
- private boolean allowShortCircuitLocalReads;
-
- /**
- * The ClientContext to use for things like the PeerCache.
- */
- private ClientContext clientContext;
-
- /**
- * Number of bytes to read. -1 indicates no limit.
- */
- private long length = -1;
-
- /**
- * Caching strategy to use when reading the block.
- */
- private CachingStrategy cachingStrategy;
-
- /**
- * Socket address to use to connect to peer.
- */
- private InetSocketAddress inetSocketAddress;
-
- /**
- * Remote peer factory to use to create a peer, if needed.
- */
- private RemotePeerFactory remotePeerFactory;
-
- /**
- * UserGroupInformation to use for legacy block reader local objects, if needed.
- */
- private UserGroupInformation userGroupInformation;
-
- /**
- * Configuration to use for legacy block reader local objects, if needed.
- */
- private Configuration configuration;
-
- /**
- * Information about the domain socket path we should use to connect to the
- * local peer-- or null if we haven't examined the local domain socket.
- */
- private DomainSocketFactory.PathInfo pathInfo;
-
- /**
- * The remaining number of times that we'll try to pull a socket out of the
- * cache.
- */
- private int remainingCacheTries;
-
- public BlockReaderFactory(DfsClientConf conf) {
- this.conf = conf;
- this.remainingCacheTries = conf.getNumCachedConnRetry();
- }
-
- public BlockReaderFactory setFileName(String fileName) {
- this.fileName = fileName;
- return this;
- }
-
- public BlockReaderFactory setBlock(ExtendedBlock block) {
- this.block = block;
- return this;
- }
-
- public BlockReaderFactory setBlockToken(Token<BlockTokenIdentifier> token) {
- this.token = token;
- return this;
- }
-
- public BlockReaderFactory setStartOffset(long startOffset) {
- this.startOffset = startOffset;
- return this;
- }
-
- public BlockReaderFactory setVerifyChecksum(boolean verifyChecksum) {
- this.verifyChecksum = verifyChecksum;
- return this;
- }
-
- public BlockReaderFactory setClientName(String clientName) {
- this.clientName = clientName;
- return this;
- }
-
- public BlockReaderFactory setDatanodeInfo(DatanodeInfo datanode) {
- this.datanode = datanode;
- return this;
- }
-
- public BlockReaderFactory setStorageType(StorageType storageType) {
- this.storageType = storageType;
- return this;
- }
-
- public BlockReaderFactory setAllowShortCircuitLocalReads(
- boolean allowShortCircuitLocalReads) {
- this.allowShortCircuitLocalReads = allowShortCircuitLocalReads;
- return this;
- }
-
- public BlockReaderFactory setClientCacheContext(
- ClientContext clientContext) {
- this.clientContext = clientContext;
- return this;
- }
-
- public BlockReaderFactory setLength(long length) {
- this.length = length;
- return this;
- }
-
- public BlockReaderFactory setCachingStrategy(
- CachingStrategy cachingStrategy) {
- this.cachingStrategy = cachingStrategy;
- return this;
- }
-
- public BlockReaderFactory setInetSocketAddress (
- InetSocketAddress inetSocketAddress) {
- this.inetSocketAddress = inetSocketAddress;
- return this;
- }
-
- public BlockReaderFactory setUserGroupInformation(
- UserGroupInformation userGroupInformation) {
- this.userGroupInformation = userGroupInformation;
- return this;
- }
-
- public BlockReaderFactory setRemotePeerFactory(
- RemotePeerFactory remotePeerFactory) {
- this.remotePeerFactory = remotePeerFactory;
- return this;
- }
-
- public BlockReaderFactory setConfiguration(
- Configuration configuration) {
- this.configuration = configuration;
- return this;
- }
-
- @VisibleForTesting
- public static void setFailureInjectorForTesting(FailureInjector injector) {
- failureInjector = injector;
- }
-
- /**
- * Build a BlockReader with the given options.
- *
- * This function will do the best it can to create a block reader that meets
- * all of our requirements. We prefer short-circuit block readers
- * (BlockReaderLocal and BlockReaderLocalLegacy) over remote ones, since the
- * former avoid the overhead of socket communication. If short-circuit is
- * unavailable, our next fallback is data transfer over UNIX domain sockets,
- * if dfs.client.domain.socket.data.traffic has been enabled. If that doesn't
- * work, we will try to create a remote block reader that operates over TCP
- * sockets.
- *
- * There are a few caches that are important here.
- *
- * The ShortCircuitCache stores file descriptor objects which have been passed
- * from the DataNode.
- *
- * The DomainSocketFactory stores information about UNIX domain socket paths
- * that we not been able to use in the past, so that we don't waste time
- * retrying them over and over. (Like all the caches, it does have a timeout,
- * though.)
- *
- * The PeerCache stores peers that we have used in the past. If we can reuse
- * one of these peers, we avoid the overhead of re-opening a socket. However,
- * if the socket has been timed out on the remote end, our attempt to reuse
- * the socket may end with an IOException. For that reason, we limit our
- * attempts at socket reuse to dfs.client.cached.conn.retry times. After
- * that, we create new sockets. This avoids the problem where a thread tries
- * to talk to a peer that it hasn't talked to in a while, and has to clean out
- * every entry in a socket cache full of stale entries.
- *
- * @return The new BlockReader. We will not return null.
- *
- * @throws InvalidToken
- * If the block token was invalid.
- * InvalidEncryptionKeyException
- * If the encryption key was invalid.
- * Other IOException
- * If there was another problem.
- */
- public BlockReader build() throws IOException {
- Preconditions.checkNotNull(configuration);
- BlockReader reader = tryToCreateExternalBlockReader();
- if (reader != null) {
- return reader;
- }
- final ShortCircuitConf scConf = conf.getShortCircuitConf();
- if (scConf.isShortCircuitLocalReads() && allowShortCircuitLocalReads) {
- if (clientContext.getUseLegacyBlockReaderLocal()) {
- reader = getLegacyBlockReaderLocal();
- if (reader != null) {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": returning new legacy block reader local.");
- }
- return reader;
- }
- } else {
- reader = getBlockReaderLocal();
- if (reader != null) {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": returning new block reader local.");
- }
- return reader;
- }
- }
- }
- if (scConf.isDomainSocketDataTraffic()) {
- reader = getRemoteBlockReaderFromDomain();
- if (reader != null) {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": returning new remote block reader using " +
- "UNIX domain socket on " + pathInfo.getPath());
- }
- return reader;
- }
- }
- Preconditions.checkState(!DFSInputStream.tcpReadsDisabledForTesting,
- "TCP reads were disabled for testing, but we failed to " +
- "do a non-TCP read.");
- return getRemoteBlockReaderFromTcp();
- }
-
- private BlockReader tryToCreateExternalBlockReader() {
- List<Class<? extends ReplicaAccessorBuilder>> clses =
- conf.getReplicaAccessorBuilderClasses();
- for (Class<? extends ReplicaAccessorBuilder> cls : clses) {
- try {
- ByteArrayDataOutput bado = ByteStreams.newDataOutput();
- token.write(bado);
- byte tokenBytes[] = bado.toByteArray();
-
- Constructor<? extends ReplicaAccessorBuilder> ctor =
- cls.getConstructor();
- ReplicaAccessorBuilder builder = ctor.newInstance();
- ReplicaAccessor accessor = builder.
- setAllowShortCircuitReads(allowShortCircuitLocalReads).
- setBlock(block.getBlockId(), block.getBlockPoolId()).
- setGenerationStamp(block.getGenerationStamp()).
- setBlockAccessToken(tokenBytes).
- setClientName(clientName).
- setConfiguration(configuration).
- setFileName(fileName).
- setVerifyChecksum(verifyChecksum).
- setVisibleLength(length).
- build();
- if (accessor == null) {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": No ReplicaAccessor created by " +
- cls.getName());
- }
- } else {
- return new ExternalBlockReader(accessor, length, startOffset);
- }
- } catch (Throwable t) {
- LOG.warn("Failed to construct new object of type " +
- cls.getName(), t);
- }
- }
- return null;
- }
-
-
- /**
- * Get {@link BlockReaderLocalLegacy} for short circuited local reads.
- * This block reader implements the path-based style of local reads
- * first introduced in HDFS-2246.
- */
- private BlockReader getLegacyBlockReaderLocal() throws IOException {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": trying to construct BlockReaderLocalLegacy");
- }
- if (!DFSUtilClient.isLocalAddress(inetSocketAddress)) {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": can't construct BlockReaderLocalLegacy because " +
- "the address " + inetSocketAddress + " is not local");
- }
- return null;
- }
- if (clientContext.getDisableLegacyBlockReaderLocal()) {
- PerformanceAdvisory.LOG.debug("{}: can't construct " +
- "BlockReaderLocalLegacy because " +
- "disableLegacyBlockReaderLocal is set.", this);
- return null;
- }
- IOException ioe;
- try {
- return BlockReaderLocalLegacy.newBlockReader(conf,
- userGroupInformation, configuration, fileName, block, token,
- datanode, startOffset, length, storageType);
- } catch (RemoteException remoteException) {
- ioe = remoteException.unwrapRemoteException(
- InvalidToken.class, AccessControlException.class);
- } catch (IOException e) {
- ioe = e;
- }
- if ((!(ioe instanceof AccessControlException)) &&
- isSecurityException(ioe)) {
- // Handle security exceptions.
- // We do not handle AccessControlException here, since
- // BlockReaderLocalLegacy#newBlockReader uses that exception to indicate
- // that the user is not in dfs.block.local-path-access.user, a condition
- // which requires us to disable legacy SCR.
- throw ioe;
- }
- LOG.warn(this + ": error creating legacy BlockReaderLocal. " +
- "Disabling legacy local reads.", ioe);
- clientContext.setDisableLegacyBlockReaderLocal();
- return null;
- }
-
- private BlockReader getBlockReaderLocal() throws InvalidToken {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": trying to construct a BlockReaderLocal " +
- "for short-circuit reads.");
- }
- if (pathInfo == null) {
- pathInfo = clientContext.getDomainSocketFactory()
- .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
- }
- if (!pathInfo.getPathState().getUsableForShortCircuit()) {
- PerformanceAdvisory.LOG.debug("{}: {} is not usable for short circuit; " +
- "giving up on BlockReaderLocal.", this, pathInfo);
- return null;
- }
- ShortCircuitCache cache = clientContext.getShortCircuitCache();
- ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
- ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
- InvalidToken exc = info.getInvalidTokenException();
- if (exc != null) {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": got InvalidToken exception while trying to " +
- "construct BlockReaderLocal via " + pathInfo.getPath());
- }
- throw exc;
- }
- if (info.getReplica() == null) {
- PerformanceAdvisory.LOG.debug("{}: failed to get " +
- "ShortCircuitReplica. Cannot construct " +
- "BlockReaderLocal via {}", this, pathInfo.getPath());
- return null;
- }
- return new BlockReaderLocal.Builder(conf.getShortCircuitConf()).
- setFilename(fileName).
- setBlock(block).
- setStartOffset(startOffset).
- setShortCircuitReplica(info.getReplica()).
- setVerifyChecksum(verifyChecksum).
- setCachingStrategy(cachingStrategy).
- setStorageType(storageType).
- build();
- }
-
- /**
- * Fetch a pair of short-circuit block descriptors from a local DataNode.
- *
- * @return Null if we could not communicate with the datanode,
- * a new ShortCircuitReplicaInfo object otherwise.
- * ShortCircuitReplicaInfo objects may contain either an InvalidToken
- * exception, or a ShortCircuitReplica object ready to use.
- */
- @Override
- public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
- if (createShortCircuitReplicaInfoCallback != null) {
- ShortCircuitReplicaInfo info =
- createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
- if (info != null) return info;
- }
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": trying to create ShortCircuitReplicaInfo.");
- }
- BlockReaderPeer curPeer;
- while (true) {
- curPeer = nextDomainPeer();
- if (curPeer == null) break;
- if (curPeer.fromCache) remainingCacheTries--;
- DomainPeer peer = (DomainPeer)curPeer.peer;
- Slot slot = null;
- ShortCircuitCache cache = clientContext.getShortCircuitCache();
- try {
- MutableBoolean usedPeer = new MutableBoolean(false);
- slot = cache.allocShmSlot(datanode, peer, usedPeer,
- new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()),
- clientName);
- if (usedPeer.booleanValue()) {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": allocShmSlot used up our previous socket " +
- peer.getDomainSocket() + ". Allocating a new one...");
- }
- curPeer = nextDomainPeer();
- if (curPeer == null) break;
- peer = (DomainPeer)curPeer.peer;
- }
- ShortCircuitReplicaInfo info = requestFileDescriptors(peer, slot);
- clientContext.getPeerCache().put(datanode, peer);
- return info;
- } catch (IOException e) {
- if (slot != null) {
- cache.freeSlot(slot);
- }
- if (curPeer.fromCache) {
- // Handle an I/O error we got when using a cached socket.
- // These are considered less serious, because the socket may be stale.
- if (LOG.isDebugEnabled()) {
- LOG.debug(this + ": closing stale domain peer " + peer, e);
- }
- IOUtils.cleanup(LOG, peer);
- } else {
- // Handle an I/O error we got when using a newly created socket.
- // We temporarily disable the domain socket path for a few minutes in
- // this case, to prevent wasting more time on it.
- LOG.warn(this + ": I/O error requesting file descriptors. " +
- "Disabling domain socket " + peer.getDomainSocket(), e);
- IOUtils.cleanup(LOG, peer);
- clientContext.getDomainSocketFactory()
- .disableDomainSocketPath(pathInfo.getPath());
- return null;
- }
- }
- }
- return null;
- }
-
- /**
- * Request file descriptors from a DomainPeer.
- *
- * @param peer The peer to use for communication.
- * @param slot If non-null, the shared memory slot to associate with the
- * new ShortCircuitReplica.
- *
- * @return A ShortCircuitReplica object if we could communicate with the
- * datanode; null, otherwise.
- * @throws IOException If we encountered an I/O exception while communicating
- * with the datanode.
- */
- private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer,
- Slot slot) throws IOException {
- ShortCircuitCache cache = clientContext.getShortCircuitCache();
- final DataOutputStream out =
- new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
- SlotId slotId = slot == null ? null : slot.getSlotId();
- new Sender(out).requestShortCircuitFds(block, token, slotId, 1,
- failureInjector.getSupportsReceiptVerification());
- DataInputStream in = new DataInputStream(peer.getInputStream());
- BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
- PBHelperClient.vintPrefixed(in));
- DomainSocket sock = peer.getDomainSocket();
- failureInjector.injectRequestFileDescriptorsFailure();
- switch (resp.getStatus()) {
- case SUCCESS:
- byte buf[] = new byte[1];
- FileInputStream fis[] = new FileInputStream[2];
- sock.recvFileInputStreams(fis, buf, 0, buf.length);
- ShortCircuitReplica replica = null;
- try {
- ExtendedBlockId key =
- new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
- if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
- LOG.trace("Sending receipt verification byte for slot " + slot);
- sock.getOutputStream().write(0);
- }
- replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
- Time.monotonicNow(), slot);
- return new ShortCircuitReplicaInfo(replica);
- } catch (IOException e) {
- // This indicates an error reading from disk, or a format error. Since
- // it's not a socket communication problem, we return null rather than
- // throwing an exception.
- LOG.warn(this + ": error creating ShortCircuitReplica.", e);
- return null;
- } finally {
- if (replica == null) {
- IOUtils.cleanup(DFSClient.LOG, fis[0], fis[1]);
- }
- }
- case ERROR_UNSUPPORTED:
- if (!resp.hasShortCircuitAccessVersion()) {
- LOG.warn("short-circuit read access is disabled for " +
- "DataNode " + datanode + ". reason: " + resp.getMessage());
- clientContext.getDomainSocketFactory()
- .disableShortCircuitForPath(pathInfo.getPath());
- } else {
- LOG.warn("short-circuit read access for the file " +
- fileName + " is disabled for DataNode " + datanode +
- ". reason: " + resp.getMessage());
- }
- return null;
- case ERROR_ACCESS_TOKEN:
- String msg = "access control error while " +
- "attempting to set up short-circuit access to " +
- fileName + resp.getMessage();
- if (LOG.isDebugEnabled()) {
- LOG.debug(this + ":" + msg);
- }
- return new ShortCircuitReplicaInfo(new InvalidToken(msg));
- default:
- LOG.warn(this + ": unknown response code " + resp.getStatus() +
- " while attempting to set up short-circuit access. " +
- resp.getMessage());
- clientContext.getDomainSocketFactory()
- .disableShortCircuitForPath(pathInfo.getPath());
- return null;
- }
- }
-
- /**
- * Get a RemoteBlockReader that communicates over a UNIX domain socket.
- *
- * @return The new BlockReader, or null if we failed to create the block
- * reader.
- *
- * @throws InvalidToken If the block token was invalid.
- * Potentially other security-related execptions.
- */
- private BlockReader getRemoteBlockReaderFromDomain() throws IOException {
- if (pathInfo == null) {
- pathInfo = clientContext.getDomainSocketFactory()
- .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
- }
- if (!pathInfo.getPathState().getUsableForDataTransfer()) {
- PerformanceAdvisory.LOG.debug("{}: not trying to create a " +
- "remote block reader because the UNIX domain socket at {}" +
- " is not usable.", this, pathInfo);
- return null;
- }
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": trying to create a remote block reader from the " +
- "UNIX domain socket at " + pathInfo.getPath());
- }
-
- while (true) {
- BlockReaderPeer curPeer = nextDomainPeer();
- if (curPeer == null) break;
- if (curPeer.fromCache) remainingCacheTries--;
- DomainPeer peer = (DomainPeer)curPeer.peer;
- BlockReader blockReader = null;
- try {
- blockReader = getRemoteBlockReader(peer);
- return blockReader;
- } catch (IOException ioe) {
- IOUtils.cleanup(LOG, peer);
- if (isSecurityException(ioe)) {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": got security exception while constructing " +
- "a remote block reader from the unix domain socket at " +
- pathInfo.getPath(), ioe);
- }
- throw ioe;
- }
- if (curPeer.fromCache) {
- // Handle an I/O error we got when using a cached peer. These are
- // considered less serious, because the underlying socket may be stale.
- if (LOG.isDebugEnabled()) {
- LOG.debug("Closed potentially stale domain peer " + peer, ioe);
- }
- } else {
- // Handle an I/O error we got when using a newly created domain peer.
- // We temporarily disable the domain socket path for a few minutes in
- // this case, to prevent wasting more time on it.
- LOG.warn("I/O error constructing remote block reader. Disabling " +
- "domain socket " + peer.getDomainSocket(), ioe);
- clientContext.getDomainSocketFactory()
- .disableDomainSocketPath(pathInfo.getPath());
- return null;
- }
- } finally {
- if (blockReader == null) {
- IOUtils.cleanup(LOG, peer);
- }
- }
- }
- return null;
- }
-
- /**
- * Get a RemoteBlockReader that communicates over a TCP socket.
- *
- * @return The new BlockReader. We will not return null, but instead throw
- * an exception if this fails.
- *
- * @throws InvalidToken
- * If the block token was invalid.
- * InvalidEncryptionKeyException
- * If the encryption key was invalid.
- * Other IOException
- * If there was another problem.
- */
- private BlockReader getRemoteBlockReaderFromTcp() throws IOException {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": trying to create a remote block reader from a " +
- "TCP socket");
- }
- BlockReader blockReader = null;
- while (true) {
- BlockReaderPeer curPeer = null;
- Peer peer = null;
- try {
- curPeer = nextTcpPeer();
- if (curPeer.fromCache) remainingCacheTries--;
- peer = curPeer.peer;
- blockReader = getRemoteBlockReader(peer);
- return blockReader;
- } catch (IOException ioe) {
- if (isSecurityException(ioe)) {
- if (LOG.isTraceEnabled()) {
- LOG.trace(this + ": got security exception while constructing " +
- "a remote block reader from " + peer, ioe);
- }
- throw ioe;
- }
- if ((curPeer != null) && curPeer.fromCache) {
- // Handle an I/O error we got when using a cached peer. These are
- // considered less serious, because the underlying socket may be
- // stale.
- if (LOG.isDebugEnabled()) {
- LOG.debug("Closed potentially stale remote peer " + peer, ioe);
- }
- } else {
- // Handle an I/O error we got when using a newly created peer.
- LOG.warn("I/O error constructing remote block reader.", ioe);
- throw ioe;
- }
- } finally {
- if (blockReader == null) {
- IOUtils.cleanup(LOG, peer);
- }
- }
- }
- }
-
- public static class BlockReaderPeer {
- final Peer peer;
- final boolean fromCache;
-
- BlockReaderPeer(Peer peer, boolean fromCache) {
- this.peer = peer;
- this.fromCache = fromCache;
- }
- }
-
- /**
- * Get the next DomainPeer-- either from the cache or by creating it.
- *
- * @return the next DomainPeer, or null if we could not construct one.
- */
- private BlockReaderPeer nextDomainPeer() {
- if (remainingCacheTries > 0) {
- Peer peer = clientContext.getPeerCache().get(datanode, true);
- if (peer != null) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("nextDomainPeer: reusing existing peer " + peer);
- }
- return new BlockReaderPeer(peer, true);
- }
- }
- DomainSocket sock = clientContext.getDomainSocketFactory().
- createSocket(pathInfo, conf.getSocketTimeout());
- if (sock == null) return null;
- return new BlockReaderPeer(new DomainPeer(sock), false);
- }
-
- /**
- * Get the next TCP-based peer-- either from the cache or by creating it.
- *
- * @return the next Peer, or null if we could not construct one.
- *
- * @throws IOException If there was an error while constructing the peer
- * (such as an InvalidEncryptionKeyException)
- */
- private BlockReaderPeer nextTcpPeer() throws IOException {
- if (remainingCacheTries > 0) {
- Peer peer = clientContext.getPeerCache().get(datanode, false);
- if (peer != null) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("nextTcpPeer: reusing existing peer " + peer);
- }
- return new BlockReaderPeer(peer, true);
- }
- }
- try {
- Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress, token,
- datanode);
- if (LOG.isTraceEnabled()) {
- LOG.trace("nextTcpPeer: created newConnectedPeer " + peer);
- }
- return new BlockReaderPeer(peer, false);
- } catch (IOException e) {
- if (LOG.isTraceEnabled()) {
- LOG.trace("nextTcpPeer: failed to create newConnectedPeer " +
- "connected to " + datanode);
- }
- throw e;
- }
- }
-
- /**
- * Determine if an exception is security-related.
- *
- * We need to handle these exceptions differently than other IOExceptions.
- * They don't indicate a communication problem. Instead, they mean that there
- * is some action the client needs to take, such as refetching block tokens,
- * renewing encryption keys, etc.
- *
- * @param ioe The exception
- * @return True only if the exception is security-related.
- */
- private static boolean isSecurityException(IOException ioe) {
- return (ioe instanceof InvalidToken) ||
- (ioe instanceof InvalidEncryptionKeyException) ||
- (ioe instanceof InvalidBlockTokenException) ||
- (ioe instanceof AccessControlException);
- }
-
- @SuppressWarnings("deprecation")
- private BlockReader getRemoteBlockReader(Peer peer) throws IOException {
- if (conf.getShortCircuitConf().isUseLegacyBlockReader()) {
- return RemoteBlockReader.newBlockReader(fileName,
- block, token, startOffset, length, conf.getIoBufferSize(),
- verifyChecksum, clientName, peer, datanode,
- clientContext.getPeerCache(), cachingStrategy);
- } else {
- return RemoteBlockReader2.newBlockReader(
- fileName, block, token, startOffset, length,
- verifyChecksum, clientName, peer, datanode,
- clientContext.getPeerCache(), cachingStrategy);
- }
- }
-
- @Override
- public String toString() {
- return "BlockReaderFactory(fileName=" + fileName + ", block=" + block + ")";
- }
-
- /**
- * File name to print when accessing a block directly (from servlets)
- * @param s Address of the block location
- * @param poolId Block pool ID of the block
- * @param blockId Block ID of the block
- * @return string that has a file name for debug purposes
- */
- public static String getFileName(final InetSocketAddress s,
- final String poolId, final long blockId) {
- return s.toString() + ":" + poolId + ":" + blockId;
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
deleted file mode 100644
index cac5366..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
+++ /dev/null
@@ -1,369 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Future;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.BlockStorageLocation;
-import org.apache.hadoop.fs.HdfsVolumeId;
-import org.apache.hadoop.fs.VolumeId;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.security.token.Token;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-class BlockStorageLocationUtil {
-
- static final Log LOG = LogFactory
- .getLog(BlockStorageLocationUtil.class);
-
- /**
- * Create a list of {@link VolumeBlockLocationCallable} corresponding to a set
- * of datanodes and blocks. The blocks must all correspond to the same
- * block pool.
- *
- * @param datanodeBlocks
- * Map of datanodes to block replicas at each datanode
- * @return callables Used to query each datanode for location information on
- * the block replicas at the datanode
- */
- private static List<VolumeBlockLocationCallable> createVolumeBlockLocationCallables(
- Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
- int timeout, boolean connectToDnViaHostname, Span parent) {
-
- if (datanodeBlocks.isEmpty()) {
- return Lists.newArrayList();
- }
-
- // Construct the callables, one per datanode
- List<VolumeBlockLocationCallable> callables =
- new ArrayList<VolumeBlockLocationCallable>();
- for (Map.Entry<DatanodeInfo, List<LocatedBlock>> entry : datanodeBlocks
- .entrySet()) {
- // Construct RPC parameters
- DatanodeInfo datanode = entry.getKey();
- List<LocatedBlock> locatedBlocks = entry.getValue();
- if (locatedBlocks.isEmpty()) {
- continue;
- }
-
- // Ensure that the blocks all are from the same block pool.
- String poolId = locatedBlocks.get(0).getBlock().getBlockPoolId();
- for (LocatedBlock lb : locatedBlocks) {
- if (!poolId.equals(lb.getBlock().getBlockPoolId())) {
- throw new IllegalArgumentException(
- "All blocks to be queried must be in the same block pool: " +
- locatedBlocks.get(0).getBlock() + " and " + lb +
- " are from different pools.");
- }
- }
-
- long[] blockIds = new long[locatedBlocks.size()];
- int i = 0;
- List<Token<BlockTokenIdentifier>> dnTokens =
- new ArrayList<Token<BlockTokenIdentifier>>(
- locatedBlocks.size());
- for (LocatedBlock b : locatedBlocks) {
- blockIds[i++] = b.getBlock().getBlockId();
- dnTokens.add(b.getBlockToken());
- }
- VolumeBlockLocationCallable callable = new VolumeBlockLocationCallable(
- conf, datanode, poolId, blockIds, dnTokens, timeout,
- connectToDnViaHostname, parent);
- callables.add(callable);
- }
- return callables;
- }
-
- /**
- * Queries datanodes for the blocks specified in <code>datanodeBlocks</code>,
- * making one RPC to each datanode. These RPCs are made in parallel using a
- * threadpool.
- *
- * @param datanodeBlocks
- * Map of datanodes to the blocks present on the DN
- * @return metadatas Map of datanodes to block metadata of the DN
- * @throws InvalidBlockTokenException
- * if client does not have read access on a requested block
- */
- static Map<DatanodeInfo, HdfsBlocksMetadata> queryDatanodesForHdfsBlocksMetadata(
- Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
- int poolsize, int timeoutMs, boolean connectToDnViaHostname)
- throws InvalidBlockTokenException {
-
- List<VolumeBlockLocationCallable> callables =
- createVolumeBlockLocationCallables(conf, datanodeBlocks, timeoutMs,
- connectToDnViaHostname, Trace.currentSpan());
-
- // Use a thread pool to execute the Callables in parallel
- List<Future<HdfsBlocksMetadata>> futures =
- new ArrayList<Future<HdfsBlocksMetadata>>();
- ExecutorService executor = new ScheduledThreadPoolExecutor(poolsize);
- try {
- futures = executor.invokeAll(callables, timeoutMs,
- TimeUnit.MILLISECONDS);
- } catch (InterruptedException e) {
- // Swallow the exception here, because we can return partial results
- }
- executor.shutdown();
-
- Map<DatanodeInfo, HdfsBlocksMetadata> metadatas =
- Maps.newHashMapWithExpectedSize(datanodeBlocks.size());
- // Fill in metadatas with results from DN RPCs, where possible
- for (int i = 0; i < futures.size(); i++) {
- VolumeBlockLocationCallable callable = callables.get(i);
- DatanodeInfo datanode = callable.getDatanodeInfo();
- Future<HdfsBlocksMetadata> future = futures.get(i);
- try {
- HdfsBlocksMetadata metadata = future.get();
- metadatas.put(callable.getDatanodeInfo(), metadata);
- } catch (CancellationException e) {
- LOG.info("Cancelled while waiting for datanode "
- + datanode.getIpcAddr(false) + ": " + e.toString());
- } catch (ExecutionException e) {
- Throwable t = e.getCause();
- if (t instanceof InvalidBlockTokenException) {
- LOG.warn("Invalid access token when trying to retrieve "
- + "information from datanode " + datanode.getIpcAddr(false));
- throw (InvalidBlockTokenException) t;
- }
- else if (t instanceof UnsupportedOperationException) {
- LOG.info("Datanode " + datanode.getIpcAddr(false) + " does not support"
- + " required #getHdfsBlocksMetadata() API");
- throw (UnsupportedOperationException) t;
- } else {
- LOG.info("Failed to query block locations on datanode " +
- datanode.getIpcAddr(false) + ": " + t);
- }
- if (LOG.isDebugEnabled()) {
- LOG.debug("Could not fetch information from datanode", t);
- }
- } catch (InterruptedException e) {
- // Shouldn't happen, because invokeAll waits for all Futures to be ready
- LOG.info("Interrupted while fetching HdfsBlocksMetadata");
- }
- }
-
- return metadatas;
- }
-
- /**
- * Group the per-replica {@link VolumeId} info returned from
- * {@link DFSClient#queryDatanodesForHdfsBlocksMetadata(Map)} to be
- * associated
- * with the corresponding {@link LocatedBlock}.
- *
- * @param blocks
- * Original LocatedBlock array
- * @param metadatas
- * VolumeId information for the replicas on each datanode
- * @return blockVolumeIds per-replica VolumeId information associated with the
- * parent LocatedBlock
- */
- static Map<LocatedBlock, List<VolumeId>> associateVolumeIdsWithBlocks(
- List<LocatedBlock> blocks,
- Map<DatanodeInfo, HdfsBlocksMetadata> metadatas) {
-
- // Initialize mapping of ExtendedBlock to LocatedBlock.
- // Used to associate results from DN RPCs to the parent LocatedBlock
- Map<Long, LocatedBlock> blockIdToLocBlock =
- new HashMap<Long, LocatedBlock>();
- for (LocatedBlock b : blocks) {
- blockIdToLocBlock.put(b.getBlock().getBlockId(), b);
- }
-
- // Initialize the mapping of blocks -> list of VolumeIds, one per replica
- // This is filled out with real values from the DN RPCs
- Map<LocatedBlock, List<VolumeId>> blockVolumeIds =
- new HashMap<LocatedBlock, List<VolumeId>>();
- for (LocatedBlock b : blocks) {
- ArrayList<VolumeId> l = new ArrayList<VolumeId>(b.getLocations().length);
- for (int i = 0; i < b.getLocations().length; i++) {
- l.add(null);
- }
- blockVolumeIds.put(b, l);
- }
-
- // Iterate through the list of metadatas (one per datanode).
- // For each metadata, if it's valid, insert its volume location information
- // into the Map returned to the caller
- for (Map.Entry<DatanodeInfo, HdfsBlocksMetadata> entry : metadatas.entrySet()) {
- DatanodeInfo datanode = entry.getKey();
- HdfsBlocksMetadata metadata = entry.getValue();
- // Check if metadata is valid
- if (metadata == null) {
- continue;
- }
- long[] metaBlockIds = metadata.getBlockIds();
- List<byte[]> metaVolumeIds = metadata.getVolumeIds();
- List<Integer> metaVolumeIndexes = metadata.getVolumeIndexes();
- // Add VolumeId for each replica in the HdfsBlocksMetadata
- for (int j = 0; j < metaBlockIds.length; j++) {
- int volumeIndex = metaVolumeIndexes.get(j);
- long blockId = metaBlockIds[j];
- // Skip if block wasn't found, or not a valid index into metaVolumeIds
- // Also skip if the DN responded with a block we didn't ask for
- if (volumeIndex == Integer.MAX_VALUE
- || volumeIndex >= metaVolumeIds.size()
- || !blockIdToLocBlock.containsKey(blockId)) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("No data for block " + blockId);
- }
- continue;
- }
- // Get the VolumeId by indexing into the list of VolumeIds
- // provided by the datanode
- byte[] volumeId = metaVolumeIds.get(volumeIndex);
- HdfsVolumeId id = new HdfsVolumeId(volumeId);
- // Find out which index we are in the LocatedBlock's replicas
- LocatedBlock locBlock = blockIdToLocBlock.get(blockId);
- DatanodeInfo[] dnInfos = locBlock.getLocations();
- int index = -1;
- for (int k = 0; k < dnInfos.length; k++) {
- if (dnInfos[k].equals(datanode)) {
- index = k;
- break;
- }
- }
- if (index < 0) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Datanode responded with a block volume id we did" +
- " not request, omitting.");
- }
- continue;
- }
- // Place VolumeId at the same index as the DN's index in the list of
- // replicas
- List<VolumeId> volumeIds = blockVolumeIds.get(locBlock);
- volumeIds.set(index, id);
- }
- }
- return blockVolumeIds;
- }
-
- /**
- * Helper method to combine a list of {@link LocatedBlock} with associated
- * {@link VolumeId} information to form a list of {@link BlockStorageLocation}
- * .
- */
- static BlockStorageLocation[] convertToVolumeBlockLocations(
- List<LocatedBlock> blocks,
- Map<LocatedBlock, List<VolumeId>> blockVolumeIds) throws IOException {
- // Construct the final return value of VolumeBlockLocation[]
- BlockLocation[] locations = DFSUtilClient.locatedBlocks2Locations(blocks);
- List<BlockStorageLocation> volumeBlockLocs =
- new ArrayList<BlockStorageLocation>(locations.length);
- for (int i = 0; i < locations.length; i++) {
- LocatedBlock locBlock = blocks.get(i);
- List<VolumeId> volumeIds = blockVolumeIds.get(locBlock);
- BlockStorageLocation bsLoc = new BlockStorageLocation(locations[i],
- volumeIds.toArray(new VolumeId[0]));
- volumeBlockLocs.add(bsLoc);
- }
- return volumeBlockLocs.toArray(new BlockStorageLocation[] {});
- }
-
- /**
- * Callable that sets up an RPC proxy to a datanode and queries it for
- * volume location information for a list of ExtendedBlocks.
- */
- private static class VolumeBlockLocationCallable implements
- Callable<HdfsBlocksMetadata> {
-
- private final Configuration configuration;
- private final int timeout;
- private final DatanodeInfo datanode;
- private final String poolId;
- private final long[] blockIds;
- private final List<Token<BlockTokenIdentifier>> dnTokens;
- private final boolean connectToDnViaHostname;
- private final Span parentSpan;
-
- VolumeBlockLocationCallable(Configuration configuration,
- DatanodeInfo datanode, String poolId, long []blockIds,
- List<Token<BlockTokenIdentifier>> dnTokens, int timeout,
- boolean connectToDnViaHostname, Span parentSpan) {
- this.configuration = configuration;
- this.timeout = timeout;
- this.datanode = datanode;
- this.poolId = poolId;
- this.blockIds = blockIds;
- this.dnTokens = dnTokens;
- this.connectToDnViaHostname = connectToDnViaHostname;
- this.parentSpan = parentSpan;
- }
-
- public DatanodeInfo getDatanodeInfo() {
- return datanode;
- }
-
- @Override
- public HdfsBlocksMetadata call() throws Exception {
- HdfsBlocksMetadata metadata = null;
- // Create the RPC proxy and make the RPC
- ClientDatanodeProtocol cdp = null;
- TraceScope scope =
- Trace.startSpan("getHdfsBlocksMetadata", parentSpan);
- try {
- cdp = DFSUtilClient.createClientDatanodeProtocolProxy(
- datanode, configuration,
- timeout, connectToDnViaHostname);
- metadata = cdp.getHdfsBlocksMetadata(poolId, blockIds, dnTokens);
- } catch (IOException e) {
- // Bubble this up to the caller, handle with the Future
- throw e;
- } finally {
- scope.close();
- if (cdp != null) {
- RPC.stopProxy(cdp);
- }
- }
- return metadata;
- }
- }
-}
[08/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and
related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
new file mode 100644
index 0000000..fb57825
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -0,0 +1,1904 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.io.OutputStream;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
+import org.apache.hadoop.hdfs.util.ByteArrayManager;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
+import org.apache.htrace.NullScope;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceInfo;
+import org.apache.htrace.TraceScope;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/*********************************************************************
+ *
+ * The DataStreamer class is responsible for sending data packets to the
+ * datanodes in the pipeline. It retrieves a new blockid and block locations
+ * from the namenode, and starts streaming packets to the pipeline of
+ * Datanodes. Every packet has a sequence number associated with
+ * it. When all the packets for a block are sent out and acks for each
+ * if them are received, the DataStreamer closes the current block.
+ *
+ * The DataStreamer thread picks up packets from the dataQueue, sends it to
+ * the first datanode in the pipeline and moves it from the dataQueue to the
+ * ackQueue. The ResponseProcessor receives acks from the datanodes. When an
+ * successful ack for a packet is received from all datanodes, the
+ * ResponseProcessor removes the corresponding packet from the ackQueue.
+ *
+ * In case of error, all outstanding packets are moved from ackQueue. A new
+ * pipeline is setup by eliminating the bad datanode from the original
+ * pipeline. The DataStreamer now starts sending packets from the dataQueue.
+ *
+ *********************************************************************/
+
+@InterfaceAudience.Private
+class DataStreamer extends Daemon {
+ static final Logger LOG = LoggerFactory.getLogger(DataStreamer.class);
+
+ /**
+ * Create a socket for a write pipeline
+ *
+ * @param first the first datanode
+ * @param length the pipeline length
+ * @param client client
+ * @return the socket connected to the first datanode
+ */
+ static Socket createSocketForPipeline(final DatanodeInfo first,
+ final int length, final DFSClient client) throws IOException {
+ final DfsClientConf conf = client.getConf();
+ final String dnAddr = first.getXferAddr(conf.isConnectToDnViaHostname());
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Connecting to datanode " + dnAddr);
+ }
+ final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
+ final Socket sock = client.socketFactory.createSocket();
+ final int timeout = client.getDatanodeReadTimeout(length);
+ NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout());
+ sock.setSoTimeout(timeout);
+ sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Send buf size " + sock.getSendBufferSize());
+ }
+ return sock;
+ }
+
+ /**
+ * if this file is lazy persist
+ *
+ * @param stat the HdfsFileStatus of a file
+ * @return if this file is lazy persist
+ */
+ static boolean isLazyPersist(HdfsFileStatus stat) {
+ return stat.getStoragePolicy() == HdfsConstants.MEMORY_STORAGE_POLICY_ID;
+ }
+
+ /**
+ * release a list of packets to ByteArrayManager
+ *
+ * @param packets packets to be release
+ * @param bam ByteArrayManager
+ */
+ private static void releaseBuffer(List<DFSPacket> packets, ByteArrayManager bam) {
+ for(DFSPacket p : packets) {
+ p.releaseBuffer(bam);
+ }
+ packets.clear();
+ }
+
+ static class LastExceptionInStreamer {
+ private IOException thrown;
+
+ synchronized void set(Throwable t) {
+ assert t != null;
+ this.thrown = t instanceof IOException ?
+ (IOException) t : new IOException(t);
+ }
+
+ synchronized void clear() {
+ thrown = null;
+ }
+
+ /** Check if there already is an exception. */
+ synchronized void check(boolean resetToNull) throws IOException {
+ if (thrown != null) {
+ if (LOG.isTraceEnabled()) {
+ // wrap and print the exception to know when the check is called
+ LOG.trace("Got Exception while checking", new Throwable(thrown));
+ }
+ final IOException e = thrown;
+ if (resetToNull) {
+ thrown = null;
+ }
+ throw e;
+ }
+ }
+
+ synchronized void throwException4Close() throws IOException {
+ check(false);
+ throw new ClosedChannelException();
+ }
+ }
+
+ static class ErrorState {
+ private boolean error = false;
+ private int badNodeIndex = -1;
+ private int restartingNodeIndex = -1;
+ private long restartingNodeDeadline = 0;
+ private final long datanodeRestartTimeout;
+
+ ErrorState(long datanodeRestartTimeout) {
+ this.datanodeRestartTimeout = datanodeRestartTimeout;
+ }
+
+ synchronized void reset() {
+ error = false;
+ badNodeIndex = -1;
+ restartingNodeIndex = -1;
+ restartingNodeDeadline = 0;
+ }
+
+ synchronized boolean hasError() {
+ return error;
+ }
+
+ synchronized boolean hasDatanodeError() {
+ return error && isNodeMarked();
+ }
+
+ synchronized void setError(boolean err) {
+ this.error = err;
+ }
+
+ synchronized void setBadNodeIndex(int index) {
+ this.badNodeIndex = index;
+ }
+
+ synchronized int getBadNodeIndex() {
+ return badNodeIndex;
+ }
+
+ synchronized int getRestartingNodeIndex() {
+ return restartingNodeIndex;
+ }
+
+ synchronized void initRestartingNode(int i, String message) {
+ restartingNodeIndex = i;
+ restartingNodeDeadline = Time.monotonicNow() + datanodeRestartTimeout;
+ // If the data streamer has already set the primary node
+ // bad, clear it. It is likely that the write failed due to
+ // the DN shutdown. Even if it was a real failure, the pipeline
+ // recovery will take care of it.
+ badNodeIndex = -1;
+ LOG.info(message);
+ }
+
+ synchronized boolean isRestartingNode() {
+ return restartingNodeIndex >= 0;
+ }
+
+ synchronized boolean isNodeMarked() {
+ return badNodeIndex >= 0 || isRestartingNode();
+ }
+
+ /**
+ * This method is used when no explicit error report was received, but
+ * something failed. The first node is a suspect or unsure about the cause
+ * so that it is marked as failed.
+ */
+ synchronized void markFirstNodeIfNotMarked() {
+ // There should be no existing error and no ongoing restart.
+ if (!isNodeMarked()) {
+ badNodeIndex = 0;
+ }
+ }
+
+ synchronized void adjustState4RestartingNode() {
+ // Just took care of a node error while waiting for a node restart
+ if (restartingNodeIndex >= 0) {
+ // If the error came from a node further away than the restarting
+ // node, the restart must have been complete.
+ if (badNodeIndex > restartingNodeIndex) {
+ restartingNodeIndex = -1;
+ } else if (badNodeIndex < restartingNodeIndex) {
+ // the node index has shifted.
+ restartingNodeIndex--;
+ } else {
+ throw new IllegalStateException("badNodeIndex = " + badNodeIndex
+ + " = restartingNodeIndex = " + restartingNodeIndex);
+ }
+ }
+
+ if (!isRestartingNode()) {
+ error = false;
+ }
+ badNodeIndex = -1;
+ }
+
+ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) {
+ if (restartingNodeIndex >= 0) {
+ if (!error) {
+ throw new IllegalStateException("error=false while checking" +
+ " restarting node deadline");
+ }
+
+ // check badNodeIndex
+ if (badNodeIndex == restartingNodeIndex) {
+ // ignore, if came from the restarting node
+ badNodeIndex = -1;
+ }
+ // not within the deadline
+ if (Time.monotonicNow() >= restartingNodeDeadline) {
+ // expired. declare the restarting node dead
+ restartingNodeDeadline = 0;
+ final int i = restartingNodeIndex;
+ restartingNodeIndex = -1;
+ LOG.warn("Datanode " + i + " did not restart within "
+ + datanodeRestartTimeout + "ms: " + nodes[i]);
+ // Mark the restarting node as failed. If there is any other failed
+ // node during the last pipeline construction attempt, it will not be
+ // overwritten/dropped. In this case, the restarting node will get
+ // excluded in the following attempt, if it still does not come up.
+ if (badNodeIndex == -1) {
+ badNodeIndex = i;
+ }
+ }
+ }
+ }
+ }
+
+ private volatile boolean streamerClosed = false;
+ private ExtendedBlock block; // its length is number of bytes acked
+ private Token<BlockTokenIdentifier> accessToken;
+ private DataOutputStream blockStream;
+ private DataInputStream blockReplyStream;
+ private ResponseProcessor response = null;
+ private volatile DatanodeInfo[] nodes = null; // list of targets for current block
+ private volatile StorageType[] storageTypes = null;
+ private volatile String[] storageIDs = null;
+ private final ErrorState errorState;
+
+ private BlockConstructionStage stage; // block construction stage
+ private long bytesSent = 0; // number of bytes that've been sent
+ private final boolean isLazyPersistFile;
+
+ /** Nodes have been used in the pipeline before and have failed. */
+ private final List<DatanodeInfo> failed = new ArrayList<>();
+ /** The last ack sequence number before pipeline failure. */
+ private long lastAckedSeqnoBeforeFailure = -1;
+ private int pipelineRecoveryCount = 0;
+ /** Has the current block been hflushed? */
+ private boolean isHflushed = false;
+ /** Append on an existing block? */
+ private final boolean isAppend;
+
+ private long currentSeqno = 0;
+ private long lastQueuedSeqno = -1;
+ private long lastAckedSeqno = -1;
+ private long bytesCurBlock = 0; // bytes written in current block
+ private final LastExceptionInStreamer lastException = new LastExceptionInStreamer();
+ private Socket s;
+
+ private final DFSClient dfsClient;
+ private final String src;
+ /** Only for DataTransferProtocol.writeBlock(..) */
+ private final DataChecksum checksum4WriteBlock;
+ private final Progressable progress;
+ private final HdfsFileStatus stat;
+ // appending to existing partial block
+ private volatile boolean appendChunk = false;
+ // both dataQueue and ackQueue are protected by dataQueue lock
+ private final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
+ private final LinkedList<DFSPacket> ackQueue = new LinkedList<>();
+ private final AtomicReference<CachingStrategy> cachingStrategy;
+ private final ByteArrayManager byteArrayManager;
+ //persist blocks on namenode
+ private final AtomicBoolean persistBlocks = new AtomicBoolean(false);
+ private boolean failPacket = false;
+ private final long dfsclientSlowLogThresholdMs;
+ private long artificialSlowdown = 0;
+ // List of congested data nodes. The stream will back off if the DataNodes
+ // are congested
+ private final List<DatanodeInfo> congestedNodes = new ArrayList<>();
+ private static final int CONGESTION_BACKOFF_MEAN_TIME_IN_MS = 5000;
+ private static final int CONGESTION_BACK_OFF_MAX_TIME_IN_MS =
+ CONGESTION_BACKOFF_MEAN_TIME_IN_MS * 10;
+ private int lastCongestionBackoffTime;
+
+ private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
+ private final String[] favoredNodes;
+
+ private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src,
+ Progressable progress, DataChecksum checksum,
+ AtomicReference<CachingStrategy> cachingStrategy,
+ ByteArrayManager byteArrayManage,
+ boolean isAppend, String[] favoredNodes) {
+ this.dfsClient = dfsClient;
+ this.src = src;
+ this.progress = progress;
+ this.stat = stat;
+ this.checksum4WriteBlock = checksum;
+ this.cachingStrategy = cachingStrategy;
+ this.byteArrayManager = byteArrayManage;
+ this.isLazyPersistFile = isLazyPersist(stat);
+ this.isAppend = isAppend;
+ this.favoredNodes = favoredNodes;
+
+ final DfsClientConf conf = dfsClient.getConf();
+ this.dfsclientSlowLogThresholdMs = conf.getSlowIoWarningThresholdMs();
+ this.excludedNodes = initExcludedNodes(conf.getExcludedNodesCacheExpiry());
+ this.errorState = new ErrorState(conf.getDatanodeRestartTimeout());
+ }
+
+ /**
+ * construction with tracing info
+ */
+ DataStreamer(HdfsFileStatus stat, ExtendedBlock block, DFSClient dfsClient,
+ String src, Progressable progress, DataChecksum checksum,
+ AtomicReference<CachingStrategy> cachingStrategy,
+ ByteArrayManager byteArrayManage, String[] favoredNodes) {
+ this(stat, dfsClient, src, progress, checksum, cachingStrategy,
+ byteArrayManage, false, favoredNodes);
+ this.block = block;
+ stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
+ }
+
+ /**
+ * Construct a data streamer for appending to the last partial block
+ * @param lastBlock last block of the file to be appended
+ * @param stat status of the file to be appended
+ * @throws IOException if error occurs
+ */
+ DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, DFSClient dfsClient,
+ String src, Progressable progress, DataChecksum checksum,
+ AtomicReference<CachingStrategy> cachingStrategy,
+ ByteArrayManager byteArrayManage) throws IOException {
+ this(stat, dfsClient, src, progress, checksum, cachingStrategy,
+ byteArrayManage, true, null);
+ stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
+ block = lastBlock.getBlock();
+ bytesSent = block.getNumBytes();
+ accessToken = lastBlock.getBlockToken();
+ }
+
+ /**
+ * Set pipeline in construction
+ *
+ * @param lastBlock the last block of a file
+ * @throws IOException
+ */
+ void setPipelineInConstruction(LocatedBlock lastBlock) throws IOException{
+ // setup pipeline to append to the last block XXX retries??
+ setPipeline(lastBlock);
+ if (nodes.length < 1) {
+ throw new IOException("Unable to retrieve blocks locations " +
+ " for last block " + block +
+ "of file " + src);
+ }
+ }
+
+ private void setPipeline(LocatedBlock lb) {
+ setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs());
+ }
+
+ private void setPipeline(DatanodeInfo[] nodes, StorageType[] storageTypes,
+ String[] storageIDs) {
+ this.nodes = nodes;
+ this.storageTypes = storageTypes;
+ this.storageIDs = storageIDs;
+ }
+
+ /**
+ * Initialize for data streaming
+ */
+ private void initDataStreaming() {
+ this.setName("DataStreamer for file " + src +
+ " block " + block);
+ response = new ResponseProcessor(nodes);
+ response.start();
+ stage = BlockConstructionStage.DATA_STREAMING;
+ }
+
+ private void endBlock() {
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("Closing old block " + block);
+ }
+ this.setName("DataStreamer for file " + src);
+ closeResponder();
+ closeStream();
+ setPipeline(null, null, null);
+ stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
+ }
+
+ private boolean shouldStop() {
+ return streamerClosed || errorState.hasError() || !dfsClient.clientRunning;
+ }
+
+ /*
+ * streamer thread is the only thread that opens streams to datanode,
+ * and closes them. Any error recovery is also done by this thread.
+ */
+ @Override
+ public void run() {
+ long lastPacket = Time.monotonicNow();
+ TraceScope scope = NullScope.INSTANCE;
+ while (!streamerClosed && dfsClient.clientRunning) {
+ // if the Responder encountered an error, shutdown Responder
+ if (errorState.hasError() && response != null) {
+ try {
+ response.close();
+ response.join();
+ response = null;
+ } catch (InterruptedException e) {
+ LOG.warn("Caught exception", e);
+ }
+ }
+
+ DFSPacket one;
+ try {
+ // process datanode IO errors if any
+ boolean doSleep = processDatanodeError();
+
+ final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2;
+ synchronized (dataQueue) {
+ // wait for a packet to be sent.
+ long now = Time.monotonicNow();
+ while ((!shouldStop() && dataQueue.size() == 0 &&
+ (stage != BlockConstructionStage.DATA_STREAMING ||
+ stage == BlockConstructionStage.DATA_STREAMING &&
+ now - lastPacket < halfSocketTimeout)) || doSleep ) {
+ long timeout = halfSocketTimeout - (now-lastPacket);
+ timeout = timeout <= 0 ? 1000 : timeout;
+ timeout = (stage == BlockConstructionStage.DATA_STREAMING)?
+ timeout : 1000;
+ try {
+ dataQueue.wait(timeout);
+ } catch (InterruptedException e) {
+ LOG.warn("Caught exception", e);
+ }
+ doSleep = false;
+ now = Time.monotonicNow();
+ }
+ if (shouldStop()) {
+ continue;
+ }
+ // get packet to be sent.
+ if (dataQueue.isEmpty()) {
+ one = createHeartbeatPacket();
+ } else {
+ try {
+ backOffIfNecessary();
+ } catch (InterruptedException e) {
+ LOG.warn("Caught exception", e);
+ }
+ one = dataQueue.getFirst(); // regular data packet
+ long parents[] = one.getTraceParents();
+ if (parents.length > 0) {
+ scope = Trace.startSpan("dataStreamer", new TraceInfo(0, parents[0]));
+ // TODO: use setParents API once it's available from HTrace 3.2
+ // scope = Trace.startSpan("dataStreamer", Sampler.ALWAYS);
+ // scope.getSpan().setParents(parents);
+ }
+ }
+ }
+
+ // get new block from namenode.
+ if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("Allocating new block");
+ }
+ setPipeline(nextBlockOutputStream());
+ initDataStreaming();
+ } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("Append to block " + block);
+ }
+ setupPipelineForAppendOrRecovery();
+ if (streamerClosed) {
+ continue;
+ }
+ initDataStreaming();
+ }
+
+ long lastByteOffsetInBlock = one.getLastByteOffsetBlock();
+ if (lastByteOffsetInBlock > stat.getBlockSize()) {
+ throw new IOException("BlockSize " + stat.getBlockSize() +
+ " is smaller than data size. " +
+ " Offset of packet in block " +
+ lastByteOffsetInBlock +
+ " Aborting file " + src);
+ }
+
+ if (one.isLastPacketInBlock()) {
+ // wait for all data packets have been successfully acked
+ synchronized (dataQueue) {
+ while (!shouldStop() && ackQueue.size() != 0) {
+ try {
+ // wait for acks to arrive from datanodes
+ dataQueue.wait(1000);
+ } catch (InterruptedException e) {
+ LOG.warn("Caught exception", e);
+ }
+ }
+ }
+ if (shouldStop()) {
+ continue;
+ }
+ stage = BlockConstructionStage.PIPELINE_CLOSE;
+ }
+
+ // send the packet
+ Span span = null;
+ synchronized (dataQueue) {
+ // move packet from dataQueue to ackQueue
+ if (!one.isHeartbeatPacket()) {
+ span = scope.detach();
+ one.setTraceSpan(span);
+ dataQueue.removeFirst();
+ ackQueue.addLast(one);
+ dataQueue.notifyAll();
+ }
+ }
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("DataStreamer block " + block +
+ " sending packet " + one);
+ }
+
+ // write out data to remote datanode
+ TraceScope writeScope = Trace.startSpan("writeTo", span);
+ try {
+ one.writeTo(blockStream);
+ blockStream.flush();
+ } catch (IOException e) {
+ // HDFS-3398 treat primary DN is down since client is unable to
+ // write to primary DN. If a failed or restarting node has already
+ // been recorded by the responder, the following call will have no
+ // effect. Pipeline recovery can handle only one node error at a
+ // time. If the primary node fails again during the recovery, it
+ // will be taken out then.
+ errorState.markFirstNodeIfNotMarked();
+ throw e;
+ } finally {
+ writeScope.close();
+ }
+ lastPacket = Time.monotonicNow();
+
+ // update bytesSent
+ long tmpBytesSent = one.getLastByteOffsetBlock();
+ if (bytesSent < tmpBytesSent) {
+ bytesSent = tmpBytesSent;
+ }
+
+ if (shouldStop()) {
+ continue;
+ }
+
+ // Is this block full?
+ if (one.isLastPacketInBlock()) {
+ // wait for the close packet has been acked
+ synchronized (dataQueue) {
+ while (!shouldStop() && ackQueue.size() != 0) {
+ dataQueue.wait(1000);// wait for acks to arrive from datanodes
+ }
+ }
+ if (shouldStop()) {
+ continue;
+ }
+
+ endBlock();
+ }
+ if (progress != null) { progress.progress(); }
+
+ // This is used by unit test to trigger race conditions.
+ if (artificialSlowdown != 0 && dfsClient.clientRunning) {
+ Thread.sleep(artificialSlowdown);
+ }
+ } catch (Throwable e) {
+ // Log warning if there was a real error.
+ if (!errorState.isRestartingNode()) {
+ // Since their messages are descriptive enough, do not always
+ // log a verbose stack-trace WARN for quota exceptions.
+ if (e instanceof QuotaExceededException) {
+ LOG.debug("DataStreamer Quota Exception", e);
+ } else {
+ LOG.warn("DataStreamer Exception", e);
+ }
+ }
+ lastException.set(e);
+ assert !(e instanceof NullPointerException);
+ errorState.setError(true);
+ if (!errorState.isNodeMarked()) {
+ // Not a datanode issue
+ streamerClosed = true;
+ }
+ } finally {
+ scope.close();
+ }
+ }
+ closeInternal();
+ }
+
+ private void closeInternal() {
+ closeResponder(); // close and join
+ closeStream();
+ streamerClosed = true;
+ release();
+ synchronized (dataQueue) {
+ dataQueue.notifyAll();
+ }
+ }
+
+ /**
+ * release the DFSPackets in the two queues
+ *
+ */
+ void release() {
+ synchronized (dataQueue) {
+ releaseBuffer(dataQueue, byteArrayManager);
+ releaseBuffer(ackQueue, byteArrayManager);
+ }
+ }
+
+ /**
+ * wait for the ack of seqno
+ *
+ * @param seqno the sequence number to be acked
+ * @throws IOException
+ */
+ void waitForAckedSeqno(long seqno) throws IOException {
+ TraceScope scope = Trace.startSpan("waitForAckedSeqno", Sampler.NEVER);
+ try {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Waiting for ack for: " + seqno);
+ }
+ long begin = Time.monotonicNow();
+ try {
+ synchronized (dataQueue) {
+ while (!streamerClosed) {
+ checkClosed();
+ if (lastAckedSeqno >= seqno) {
+ break;
+ }
+ try {
+ dataQueue.wait(1000); // when we receive an ack, we notify on
+ // dataQueue
+ } catch (InterruptedException ie) {
+ throw new InterruptedIOException(
+ "Interrupted while waiting for data to be acknowledged by pipeline");
+ }
+ }
+ }
+ checkClosed();
+ } catch (ClosedChannelException e) {
+ }
+ long duration = Time.monotonicNow() - begin;
+ if (duration > dfsclientSlowLogThresholdMs) {
+ LOG.warn("Slow waitForAckedSeqno took " + duration
+ + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
+ }
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * wait for space of dataQueue and queue the packet
+ *
+ * @param packet the DFSPacket to be queued
+ * @throws IOException
+ */
+ void waitAndQueuePacket(DFSPacket packet) throws IOException {
+ synchronized (dataQueue) {
+ try {
+ // If queue is full, then wait till we have enough space
+ boolean firstWait = true;
+ try {
+ while (!streamerClosed && dataQueue.size() + ackQueue.size() >
+ dfsClient.getConf().getWriteMaxPackets()) {
+ if (firstWait) {
+ Span span = Trace.currentSpan();
+ if (span != null) {
+ span.addTimelineAnnotation("dataQueue.wait");
+ }
+ firstWait = false;
+ }
+ try {
+ dataQueue.wait();
+ } catch (InterruptedException e) {
+ // If we get interrupted while waiting to queue data, we still need to get rid
+ // of the current packet. This is because we have an invariant that if
+ // currentPacket gets full, it will get queued before the next writeChunk.
+ //
+ // Rather than wait around for space in the queue, we should instead try to
+ // return to the caller as soon as possible, even though we slightly overrun
+ // the MAX_PACKETS length.
+ Thread.currentThread().interrupt();
+ break;
+ }
+ }
+ } finally {
+ Span span = Trace.currentSpan();
+ if ((span != null) && (!firstWait)) {
+ span.addTimelineAnnotation("end.wait");
+ }
+ }
+ checkClosed();
+ queuePacket(packet);
+ } catch (ClosedChannelException e) {
+ }
+ }
+ }
+
+ /*
+ * close the streamer, should be called only by an external thread
+ * and only after all data to be sent has been flushed to datanode.
+ *
+ * Interrupt this data streamer if force is true
+ *
+ * @param force if this data stream is forced to be closed
+ */
+ void close(boolean force) {
+ streamerClosed = true;
+ synchronized (dataQueue) {
+ dataQueue.notifyAll();
+ }
+ if (force) {
+ this.interrupt();
+ }
+ }
+
+
+ private void checkClosed() throws IOException {
+ if (streamerClosed) {
+ lastException.throwException4Close();
+ }
+ }
+
+ private void closeResponder() {
+ if (response != null) {
+ try {
+ response.close();
+ response.join();
+ } catch (InterruptedException e) {
+ LOG.warn("Caught exception", e);
+ } finally {
+ response = null;
+ }
+ }
+ }
+
+ private void closeStream() {
+ final MultipleIOException.Builder b = new MultipleIOException.Builder();
+
+ if (blockStream != null) {
+ try {
+ blockStream.close();
+ } catch (IOException e) {
+ b.add(e);
+ } finally {
+ blockStream = null;
+ }
+ }
+ if (blockReplyStream != null) {
+ try {
+ blockReplyStream.close();
+ } catch (IOException e) {
+ b.add(e);
+ } finally {
+ blockReplyStream = null;
+ }
+ }
+ if (null != s) {
+ try {
+ s.close();
+ } catch (IOException e) {
+ b.add(e);
+ } finally {
+ s = null;
+ }
+ }
+
+ final IOException ioe = b.build();
+ if (ioe != null) {
+ lastException.set(ioe);
+ }
+ }
+
+ /**
+ * Examine whether it is worth waiting for a node to restart.
+ * @param index the node index
+ */
+ boolean shouldWaitForRestart(int index) {
+ // Only one node in the pipeline.
+ if (nodes.length == 1) {
+ return true;
+ }
+
+ // Is it a local node?
+ InetAddress addr = null;
+ try {
+ addr = InetAddress.getByName(nodes[index].getIpAddr());
+ } catch (java.net.UnknownHostException e) {
+ // we are passing an ip address. this should not happen.
+ assert false;
+ }
+
+ if (addr != null && NetUtils.isLocalAddress(addr)) {
+ return true;
+ }
+ return false;
+ }
+
+ //
+ // Processes responses from the datanodes. A packet is removed
+ // from the ackQueue when its response arrives.
+ //
+ private class ResponseProcessor extends Daemon {
+
+ private volatile boolean responderClosed = false;
+ private DatanodeInfo[] targets = null;
+ private boolean isLastPacketInBlock = false;
+
+ ResponseProcessor (DatanodeInfo[] targets) {
+ this.targets = targets;
+ }
+
+ @Override
+ public void run() {
+
+ setName("ResponseProcessor for block " + block);
+ PipelineAck ack = new PipelineAck();
+
+ TraceScope scope = NullScope.INSTANCE;
+ while (!responderClosed && dfsClient.clientRunning && !isLastPacketInBlock) {
+ // process responses from datanodes.
+ try {
+ // read an ack from the pipeline
+ long begin = Time.monotonicNow();
+ ack.readFields(blockReplyStream);
+ long duration = Time.monotonicNow() - begin;
+ if (duration > dfsclientSlowLogThresholdMs
+ && ack.getSeqno() != DFSPacket.HEART_BEAT_SEQNO) {
+ LOG.warn("Slow ReadProcessor read fields took " + duration
+ + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
+ + ack + ", targets: " + Arrays.asList(targets));
+ } else if (LOG.isDebugEnabled()) {
+ LOG.debug("DFSClient " + ack);
+ }
+
+ long seqno = ack.getSeqno();
+ // processes response status from datanodes.
+ ArrayList<DatanodeInfo> congestedNodesFromAck = new ArrayList<>();
+ for (int i = ack.getNumOfReplies()-1; i >=0 && dfsClient.clientRunning; i--) {
+ final Status reply = PipelineAck.getStatusFromHeader(ack
+ .getHeaderFlag(i));
+ if (PipelineAck.getECNFromHeader(ack.getHeaderFlag(i)) ==
+ PipelineAck.ECN.CONGESTED) {
+ congestedNodesFromAck.add(targets[i]);
+ }
+ // Restart will not be treated differently unless it is
+ // the local node or the only one in the pipeline.
+ if (PipelineAck.isRestartOOBStatus(reply) &&
+ shouldWaitForRestart(i)) {
+ final String message = "Datanode " + i + " is restarting: "
+ + targets[i];
+ errorState.initRestartingNode(i, message);
+ throw new IOException(message);
+ }
+ // node error
+ if (reply != SUCCESS) {
+ errorState.setBadNodeIndex(i); // mark bad datanode
+ throw new IOException("Bad response " + reply +
+ " for " + block + " from datanode " + targets[i]);
+ }
+ }
+
+ if (!congestedNodesFromAck.isEmpty()) {
+ synchronized (congestedNodes) {
+ congestedNodes.clear();
+ congestedNodes.addAll(congestedNodesFromAck);
+ }
+ } else {
+ synchronized (congestedNodes) {
+ congestedNodes.clear();
+ lastCongestionBackoffTime = 0;
+ }
+ }
+
+ assert seqno != PipelineAck.UNKOWN_SEQNO :
+ "Ack for unknown seqno should be a failed ack: " + ack;
+ if (seqno == DFSPacket.HEART_BEAT_SEQNO) { // a heartbeat ack
+ continue;
+ }
+
+ // a success ack for a data packet
+ DFSPacket one;
+ synchronized (dataQueue) {
+ one = ackQueue.getFirst();
+ }
+ if (one.getSeqno() != seqno) {
+ throw new IOException("ResponseProcessor: Expecting seqno " +
+ " for block " + block +
+ one.getSeqno() + " but received " + seqno);
+ }
+ isLastPacketInBlock = one.isLastPacketInBlock();
+
+ // Fail the packet write for testing in order to force a
+ // pipeline recovery.
+ if (DFSClientFaultInjector.get().failPacket() &&
+ isLastPacketInBlock) {
+ failPacket = true;
+ throw new IOException(
+ "Failing the last packet for testing.");
+ }
+
+ // update bytesAcked
+ block.setNumBytes(one.getLastByteOffsetBlock());
+
+ synchronized (dataQueue) {
+ scope = Trace.continueSpan(one.getTraceSpan());
+ one.setTraceSpan(null);
+ lastAckedSeqno = seqno;
+ ackQueue.removeFirst();
+ dataQueue.notifyAll();
+
+ one.releaseBuffer(byteArrayManager);
+ }
+ } catch (Exception e) {
+ if (!responderClosed) {
+ lastException.set(e);
+ errorState.setError(true);
+ errorState.markFirstNodeIfNotMarked();
+ synchronized (dataQueue) {
+ dataQueue.notifyAll();
+ }
+ if (!errorState.isRestartingNode()) {
+ LOG.warn("Exception for " + block, e);
+ }
+ responderClosed = true;
+ }
+ } finally {
+ scope.close();
+ }
+ }
+ }
+
+ void close() {
+ responderClosed = true;
+ this.interrupt();
+ }
+ }
+
+ /**
+ * If this stream has encountered any errors, shutdown threads
+ * and mark the stream as closed.
+ *
+ * @return true if it should sleep for a while after returning.
+ */
+ private boolean processDatanodeError() throws IOException {
+ if (!errorState.hasDatanodeError()) {
+ return false;
+ }
+ if (response != null) {
+ LOG.info("Error Recovery for " + block +
+ " waiting for responder to exit. ");
+ return true;
+ }
+ closeStream();
+
+ // move packets from ack queue to front of the data queue
+ synchronized (dataQueue) {
+ dataQueue.addAll(0, ackQueue);
+ ackQueue.clear();
+ }
+
+ // Record the new pipeline failure recovery.
+ if (lastAckedSeqnoBeforeFailure != lastAckedSeqno) {
+ lastAckedSeqnoBeforeFailure = lastAckedSeqno;
+ pipelineRecoveryCount = 1;
+ } else {
+ // If we had to recover the pipeline five times in a row for the
+ // same packet, this client likely has corrupt data or corrupting
+ // during transmission.
+ if (++pipelineRecoveryCount > 5) {
+ LOG.warn("Error recovering pipeline for writing " +
+ block + ". Already retried 5 times for the same packet.");
+ lastException.set(new IOException("Failing write. Tried pipeline " +
+ "recovery 5 times without success."));
+ streamerClosed = true;
+ return false;
+ }
+ }
+ boolean doSleep = setupPipelineForAppendOrRecovery();
+
+ if (!streamerClosed && dfsClient.clientRunning) {
+ if (stage == BlockConstructionStage.PIPELINE_CLOSE) {
+
+ // If we had an error while closing the pipeline, we go through a fast-path
+ // where the BlockReceiver does not run. Instead, the DataNode just finalizes
+ // the block immediately during the 'connect ack' process. So, we want to pull
+ // the end-of-block packet from the dataQueue, since we don't actually have
+ // a true pipeline to send it over.
+ //
+ // We also need to set lastAckedSeqno to the end-of-block Packet's seqno, so that
+ // a client waiting on close() will be aware that the flush finished.
+ synchronized (dataQueue) {
+ DFSPacket endOfBlockPacket = dataQueue.remove(); // remove the end of block packet
+ Span span = endOfBlockPacket.getTraceSpan();
+ if (span != null) {
+ // Close any trace span associated with this Packet
+ TraceScope scope = Trace.continueSpan(span);
+ scope.close();
+ }
+ assert endOfBlockPacket.isLastPacketInBlock();
+ assert lastAckedSeqno == endOfBlockPacket.getSeqno() - 1;
+ lastAckedSeqno = endOfBlockPacket.getSeqno();
+ dataQueue.notifyAll();
+ }
+ endBlock();
+ } else {
+ initDataStreaming();
+ }
+ }
+
+ return doSleep;
+ }
+
+ void setHflush() {
+ isHflushed = true;
+ }
+
+ private int findNewDatanode(final DatanodeInfo[] original
+ ) throws IOException {
+ if (nodes.length != original.length + 1) {
+ throw new IOException(
+ new StringBuilder()
+ .append("Failed to replace a bad datanode on the existing pipeline ")
+ .append("due to no more good datanodes being available to try. ")
+ .append("(Nodes: current=").append(Arrays.asList(nodes))
+ .append(", original=").append(Arrays.asList(original)).append("). ")
+ .append("The current failed datanode replacement policy is ")
+ .append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ")
+ .append("a client may configure this via '")
+ .append(BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY)
+ .append("' in its configuration.")
+ .toString());
+ }
+ for(int i = 0; i < nodes.length; i++) {
+ int j = 0;
+ for(; j < original.length && !nodes[i].equals(original[j]); j++);
+ if (j == original.length) {
+ return i;
+ }
+ }
+ throw new IOException("Failed: new datanode not found: nodes="
+ + Arrays.asList(nodes) + ", original=" + Arrays.asList(original));
+ }
+
+ private void addDatanode2ExistingPipeline() throws IOException {
+ if (DataTransferProtocol.LOG.isDebugEnabled()) {
+ DataTransferProtocol.LOG.debug("lastAckedSeqno = " + lastAckedSeqno);
+ }
+ /*
+ * Is data transfer necessary? We have the following cases.
+ *
+ * Case 1: Failure in Pipeline Setup
+ * - Append
+ * + Transfer the stored replica, which may be a RBW or a finalized.
+ * - Create
+ * + If no data, then no transfer is required.
+ * + If there are data written, transfer RBW. This case may happens
+ * when there are streaming failure earlier in this pipeline.
+ *
+ * Case 2: Failure in Streaming
+ * - Append/Create:
+ * + transfer RBW
+ *
+ * Case 3: Failure in Close
+ * - Append/Create:
+ * + no transfer, let NameNode replicates the block.
+ */
+ if (!isAppend && lastAckedSeqno < 0
+ && stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
+ //no data have been written
+ return;
+ } else if (stage == BlockConstructionStage.PIPELINE_CLOSE
+ || stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
+ //pipeline is closing
+ return;
+ }
+
+ //get a new datanode
+ final DatanodeInfo[] original = nodes;
+ final LocatedBlock lb = dfsClient.namenode.getAdditionalDatanode(
+ src, stat.getFileId(), block, nodes, storageIDs,
+ failed.toArray(new DatanodeInfo[failed.size()]),
+ 1, dfsClient.clientName);
+ setPipeline(lb);
+
+ //find the new datanode
+ final int d = findNewDatanode(original);
+
+ //transfer replica
+ final DatanodeInfo src = d == 0? nodes[1]: nodes[d - 1];
+ final DatanodeInfo[] targets = {nodes[d]};
+ final StorageType[] targetStorageTypes = {storageTypes[d]};
+ transfer(src, targets, targetStorageTypes, lb.getBlockToken());
+ }
+
+ private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
+ final StorageType[] targetStorageTypes,
+ final Token<BlockTokenIdentifier> blockToken) throws IOException {
+ //transfer replica to the new datanode
+ Socket sock = null;
+ DataOutputStream out = null;
+ DataInputStream in = null;
+ try {
+ sock = createSocketForPipeline(src, 2, dfsClient);
+ final long writeTimeout = dfsClient.getDatanodeWriteTimeout(2);
+ final long readTimeout = dfsClient.getDatanodeReadTimeout(2);
+
+ OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
+ InputStream unbufIn = NetUtils.getInputStream(sock, readTimeout);
+ IOStreamPair saslStreams = dfsClient.saslClient.socketSend(sock,
+ unbufOut, unbufIn, dfsClient, blockToken, src);
+ unbufOut = saslStreams.out;
+ unbufIn = saslStreams.in;
+ out = new DataOutputStream(new BufferedOutputStream(unbufOut,
+ DFSUtilClient.getSmallBufferSize(dfsClient.getConfiguration())));
+ in = new DataInputStream(unbufIn);
+
+ //send the TRANSFER_BLOCK request
+ new Sender(out).transferBlock(block, blockToken, dfsClient.clientName,
+ targets, targetStorageTypes);
+ out.flush();
+
+ //ack
+ BlockOpResponseProto response =
+ BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
+ if (SUCCESS != response.getStatus()) {
+ throw new IOException("Failed to add a datanode");
+ }
+ } finally {
+ IOUtils.closeStream(in);
+ IOUtils.closeStream(out);
+ IOUtils.closeSocket(sock);
+ }
+ }
+
+ /**
+ * Open a DataStreamer to a DataNode pipeline so that
+ * it can be written to.
+ * This happens when a file is appended or data streaming fails
+ * It keeps on trying until a pipeline is setup
+ */
+ private boolean setupPipelineForAppendOrRecovery() throws IOException {
+ // check number of datanodes
+ if (nodes == null || nodes.length == 0) {
+ String msg = "Could not get block locations. " + "Source file \""
+ + src + "\" - Aborting...";
+ LOG.warn(msg);
+ lastException.set(new IOException(msg));
+ streamerClosed = true;
+ return false;
+ }
+
+ boolean success = false;
+ long newGS = 0L;
+ while (!success && !streamerClosed && dfsClient.clientRunning) {
+ if (!handleRestartingDatanode()) {
+ return false;
+ }
+
+ final boolean isRecovery = errorState.hasError();
+ if (!handleBadDatanode()) {
+ return false;
+ }
+
+ handleDatanodeReplacement();
+
+ // get a new generation stamp and an access token
+ final LocatedBlock lb = updateBlockForPipeline();
+ newGS = lb.getBlock().getGenerationStamp();
+ accessToken = lb.getBlockToken();
+
+ // set up the pipeline again with the remaining nodes
+ success = createBlockOutputStream(nodes, storageTypes, newGS, isRecovery);
+
+ failPacket4Testing();
+
+ errorState.checkRestartingNodeDeadline(nodes);
+ } // while
+
+ if (success) {
+ block = updatePipeline(newGS);
+ }
+ return false; // do not sleep, continue processing
+ }
+
+ /**
+ * Sleep if a node is restarting.
+ * This process is repeated until the deadline or the node starts back up.
+ * @return true if it should continue.
+ */
+ private boolean handleRestartingDatanode() {
+ if (errorState.isRestartingNode()) {
+ // 4 seconds or the configured deadline period, whichever is shorter.
+ // This is the retry interval and recovery will be retried in this
+ // interval until timeout or success.
+ final long delay = Math.min(errorState.datanodeRestartTimeout, 4000L);
+ try {
+ Thread.sleep(delay);
+ } catch (InterruptedException ie) {
+ lastException.set(new IOException(
+ "Interrupted while waiting for restarting "
+ + nodes[errorState.getRestartingNodeIndex()]));
+ streamerClosed = true;
+ return false;
+ }
+ }
+ return true;
+ }
+
+ /**
+ * Remove bad node from list of nodes if badNodeIndex was set.
+ * @return true if it should continue.
+ */
+ private boolean handleBadDatanode() {
+ final int badNodeIndex = errorState.getBadNodeIndex();
+ if (badNodeIndex >= 0) {
+ if (nodes.length <= 1) {
+ lastException.set(new IOException("All datanodes "
+ + Arrays.toString(nodes) + " are bad. Aborting..."));
+ streamerClosed = true;
+ return false;
+ }
+
+ LOG.warn("Error Recovery for " + block + " in pipeline "
+ + Arrays.toString(nodes) + ": datanode " + badNodeIndex
+ + "("+ nodes[badNodeIndex] + ") is bad.");
+ failed.add(nodes[badNodeIndex]);
+
+ DatanodeInfo[] newnodes = new DatanodeInfo[nodes.length-1];
+ arraycopy(nodes, newnodes, badNodeIndex);
+
+ final StorageType[] newStorageTypes = new StorageType[newnodes.length];
+ arraycopy(storageTypes, newStorageTypes, badNodeIndex);
+
+ final String[] newStorageIDs = new String[newnodes.length];
+ arraycopy(storageIDs, newStorageIDs, badNodeIndex);
+
+ setPipeline(newnodes, newStorageTypes, newStorageIDs);
+
+ errorState.adjustState4RestartingNode();
+ lastException.clear();
+ }
+ return true;
+ }
+
+ /** Add a datanode if replace-datanode policy is satisfied. */
+ private void handleDatanodeReplacement() throws IOException {
+ if (dfsClient.dtpReplaceDatanodeOnFailure.satisfy(stat.getReplication(),
+ nodes, isAppend, isHflushed)) {
+ try {
+ addDatanode2ExistingPipeline();
+ } catch(IOException ioe) {
+ if (!dfsClient.dtpReplaceDatanodeOnFailure.isBestEffort()) {
+ throw ioe;
+ }
+ LOG.warn("Failed to replace datanode."
+ + " Continue with the remaining datanodes since "
+ + BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY
+ + " is set to true.", ioe);
+ }
+ }
+ }
+
+ private void failPacket4Testing() {
+ if (failPacket) { // for testing
+ failPacket = false;
+ try {
+ // Give DNs time to send in bad reports. In real situations,
+ // good reports should follow bad ones, if client committed
+ // with those nodes.
+ Thread.sleep(2000);
+ } catch (InterruptedException ie) {}
+ }
+ }
+
+ LocatedBlock updateBlockForPipeline() throws IOException {
+ return dfsClient.namenode.updateBlockForPipeline(
+ block, dfsClient.clientName);
+ }
+
+ /** update pipeline at the namenode */
+ ExtendedBlock updatePipeline(long newGS) throws IOException {
+ final ExtendedBlock newBlock = new ExtendedBlock(
+ block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
+ dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
+ nodes, storageIDs);
+ return newBlock;
+ }
+
+ /**
+ * Open a DataStreamer to a DataNode so that it can be written to.
+ * This happens when a file is created and each time a new block is allocated.
+ * Must get block ID and the IDs of the destinations from the namenode.
+ * Returns the list of target datanodes.
+ */
+ private LocatedBlock nextBlockOutputStream() throws IOException {
+ LocatedBlock lb = null;
+ DatanodeInfo[] nodes = null;
+ StorageType[] storageTypes = null;
+ int count = dfsClient.getConf().getNumBlockWriteRetry();
+ boolean success = false;
+ ExtendedBlock oldBlock = block;
+ do {
+ errorState.reset();
+ lastException.clear();
+ success = false;
+
+ DatanodeInfo[] excluded =
+ excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
+ .keySet()
+ .toArray(new DatanodeInfo[0]);
+ block = oldBlock;
+ lb = locateFollowingBlock(excluded.length > 0 ? excluded : null);
+ block = lb.getBlock();
+ block.setNumBytes(0);
+ bytesSent = 0;
+ accessToken = lb.getBlockToken();
+ nodes = lb.getLocations();
+ storageTypes = lb.getStorageTypes();
+
+ //
+ // Connect to first DataNode in the list.
+ //
+ success = createBlockOutputStream(nodes, storageTypes, 0L, false);
+
+ if (!success) {
+ LOG.info("Abandoning " + block);
+ dfsClient.namenode.abandonBlock(block, stat.getFileId(), src,
+ dfsClient.clientName);
+ block = null;
+ final DatanodeInfo badNode = nodes[errorState.getBadNodeIndex()];
+ LOG.info("Excluding datanode " + badNode);
+ excludedNodes.put(badNode, badNode);
+ }
+ } while (!success && --count >= 0);
+
+ if (!success) {
+ throw new IOException("Unable to create new block.");
+ }
+ return lb;
+ }
+
+ // connects to the first datanode in the pipeline
+ // Returns true if success, otherwise return failure.
+ //
+ private boolean createBlockOutputStream(DatanodeInfo[] nodes,
+ StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
+ if (nodes.length == 0) {
+ LOG.info("nodes are empty for write pipeline of " + block);
+ return false;
+ }
+ Status pipelineStatus = SUCCESS;
+ String firstBadLink = "";
+ boolean checkRestart = false;
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("pipeline = " + Arrays.asList(nodes));
+ }
+
+ // persist blocks on namenode on next flush
+ persistBlocks.set(true);
+
+ int refetchEncryptionKey = 1;
+ while (true) {
+ boolean result = false;
+ DataOutputStream out = null;
+ try {
+ assert null == s : "Previous socket unclosed";
+ assert null == blockReplyStream : "Previous blockReplyStream unclosed";
+ s = createSocketForPipeline(nodes[0], nodes.length, dfsClient);
+ long writeTimeout = dfsClient.getDatanodeWriteTimeout(nodes.length);
+ long readTimeout = dfsClient.getDatanodeReadTimeout(nodes.length);
+
+ OutputStream unbufOut = NetUtils.getOutputStream(s, writeTimeout);
+ InputStream unbufIn = NetUtils.getInputStream(s, readTimeout);
+ IOStreamPair saslStreams = dfsClient.saslClient.socketSend(s,
+ unbufOut, unbufIn, dfsClient, accessToken, nodes[0]);
+ unbufOut = saslStreams.out;
+ unbufIn = saslStreams.in;
+ out = new DataOutputStream(new BufferedOutputStream(unbufOut,
+ DFSUtilClient.getSmallBufferSize(dfsClient.getConfiguration())));
+ blockReplyStream = new DataInputStream(unbufIn);
+
+ //
+ // Xmit header info to datanode
+ //
+
+ BlockConstructionStage bcs = recoveryFlag? stage.getRecoveryStage(): stage;
+
+ // We cannot change the block length in 'block' as it counts the number
+ // of bytes ack'ed.
+ ExtendedBlock blockCopy = new ExtendedBlock(block);
+ blockCopy.setNumBytes(stat.getBlockSize());
+
+ boolean[] targetPinnings = getPinnings(nodes, true);
+ // send the request
+ new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
+ dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
+ nodes.length, block.getNumBytes(), bytesSent, newGS,
+ checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
+ (targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
+
+ // receive ack for connect
+ BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
+ PBHelperClient.vintPrefixed(blockReplyStream));
+ pipelineStatus = resp.getStatus();
+ firstBadLink = resp.getFirstBadLink();
+
+ // Got an restart OOB ack.
+ // If a node is already restarting, this status is not likely from
+ // the same node. If it is from a different node, it is not
+ // from the local datanode. Thus it is safe to treat this as a
+ // regular node error.
+ if (PipelineAck.isRestartOOBStatus(pipelineStatus) &&
+ !errorState.isRestartingNode()) {
+ checkRestart = true;
+ throw new IOException("A datanode is restarting.");
+ }
+
+ String logInfo = "ack with firstBadLink as " + firstBadLink;
+ DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
+
+ assert null == blockStream : "Previous blockStream unclosed";
+ blockStream = out;
+ result = true; // success
+ errorState.reset();
+ } catch (IOException ie) {
+ if (!errorState.isRestartingNode()) {
+ LOG.info("Exception in createBlockOutputStream", ie);
+ }
+ if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
+ LOG.info("Will fetch a new encryption key and retry, "
+ + "encryption key was invalid when connecting to "
+ + nodes[0] + " : " + ie);
+ // The encryption key used is invalid.
+ refetchEncryptionKey--;
+ dfsClient.clearDataEncryptionKey();
+ // Don't close the socket/exclude this node just yet. Try again with
+ // a new encryption key.
+ continue;
+ }
+
+ // find the datanode that matches
+ if (firstBadLink.length() != 0) {
+ for (int i = 0; i < nodes.length; i++) {
+ // NB: Unconditionally using the xfer addr w/o hostname
+ if (firstBadLink.equals(nodes[i].getXferAddr())) {
+ errorState.setBadNodeIndex(i);
+ break;
+ }
+ }
+ } else {
+ assert checkRestart == false;
+ errorState.setBadNodeIndex(0);
+ }
+
+ final int i = errorState.getBadNodeIndex();
+ // Check whether there is a restart worth waiting for.
+ if (checkRestart && shouldWaitForRestart(i)) {
+ errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
+ }
+ errorState.setError(true);
+ lastException.set(ie);
+ result = false; // error
+ } finally {
+ if (!result) {
+ IOUtils.closeSocket(s);
+ s = null;
+ IOUtils.closeStream(out);
+ out = null;
+ IOUtils.closeStream(blockReplyStream);
+ blockReplyStream = null;
+ }
+ }
+ return result;
+ }
+ }
+
+ private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) {
+ if (favoredNodes == null) {
+ return null;
+ } else {
+ boolean[] pinnings = new boolean[nodes.length];
+ HashSet<String> favoredSet =
+ new HashSet<String>(Arrays.asList(favoredNodes));
+ for (int i = 0; i < nodes.length; i++) {
+ pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(nodes[i].getXferAddrWithHostname() +
+ " was chosen by name node (favored=" + pinnings[i] + ").");
+ }
+ }
+ if (shouldLog && !favoredSet.isEmpty()) {
+ // There is one or more favored nodes that were not allocated.
+ LOG.warn("These favored nodes were specified but not chosen: "
+ + favoredSet + " Specified favored nodes: "
+ + Arrays.toString(favoredNodes));
+
+ }
+ return pinnings;
+ }
+ }
+
+ protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
+ throws IOException {
+ final DfsClientConf conf = dfsClient.getConf();
+ int retries = conf.getNumBlockWriteLocateFollowingRetry();
+ long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
+ while (true) {
+ long localstart = Time.monotonicNow();
+ while (true) {
+ try {
+ return dfsClient.namenode.addBlock(src, dfsClient.clientName,
+ block, excludedNodes, stat.getFileId(), favoredNodes);
+ } catch (RemoteException e) {
+ IOException ue =
+ e.unwrapRemoteException(FileNotFoundException.class,
+ AccessControlException.class,
+ NSQuotaExceededException.class,
+ DSQuotaExceededException.class,
+ QuotaByStorageTypeExceededException.class,
+ UnresolvedPathException.class);
+ if (ue != e) {
+ throw ue; // no need to retry these exceptions
+ }
+
+
+ if (NotReplicatedYetException.class.getName().
+ equals(e.getClassName())) {
+ if (retries == 0) {
+ throw e;
+ } else {
+ --retries;
+ LOG.info("Exception while adding a block", e);
+ long elapsed = Time.monotonicNow() - localstart;
+ if (elapsed > 5000) {
+ LOG.info("Waiting for replication for "
+ + (elapsed / 1000) + " seconds");
+ }
+ try {
+ LOG.warn("NotReplicatedYetException sleeping " + src
+ + " retries left " + retries);
+ Thread.sleep(sleeptime);
+ sleeptime *= 2;
+ } catch (InterruptedException ie) {
+ LOG.warn("Caught exception", ie);
+ }
+ }
+ } else {
+ throw e;
+ }
+
+ }
+ }
+ }
+ }
+
+ /**
+ * This function sleeps for a certain amount of time when the writing
+ * pipeline is congested. The function calculates the time based on a
+ * decorrelated filter.
+ *
+ * @see
+ * <a href="http://www.awsarchitectureblog.com/2015/03/backoff.html">
+ * http://www.awsarchitectureblog.com/2015/03/backoff.html</a>
+ */
+ private void backOffIfNecessary() throws InterruptedException {
+ int t = 0;
+ synchronized (congestedNodes) {
+ if (!congestedNodes.isEmpty()) {
+ StringBuilder sb = new StringBuilder("DataNode");
+ for (DatanodeInfo i : congestedNodes) {
+ sb.append(' ').append(i);
+ }
+ int range = Math.abs(lastCongestionBackoffTime * 3 -
+ CONGESTION_BACKOFF_MEAN_TIME_IN_MS);
+ int base = Math.min(lastCongestionBackoffTime * 3,
+ CONGESTION_BACKOFF_MEAN_TIME_IN_MS);
+ t = Math.min(CONGESTION_BACK_OFF_MAX_TIME_IN_MS,
+ (int)(base + Math.random() * range));
+ lastCongestionBackoffTime = t;
+ sb.append(" are congested. Backing off for ").append(t).append(" ms");
+ LOG.info(sb.toString());
+ congestedNodes.clear();
+ }
+ }
+ if (t != 0) {
+ Thread.sleep(t);
+ }
+ }
+
+ /**
+ * get the block this streamer is writing to
+ *
+ * @return the block this streamer is writing to
+ */
+ ExtendedBlock getBlock() {
+ return block;
+ }
+
+ /**
+ * return the target datanodes in the pipeline
+ *
+ * @return the target datanodes in the pipeline
+ */
+ DatanodeInfo[] getNodes() {
+ return nodes;
+ }
+
+ /**
+ * return the token of the block
+ *
+ * @return the token of the block
+ */
+ Token<BlockTokenIdentifier> getBlockToken() {
+ return accessToken;
+ }
+
+ /**
+ * Put a packet to the data queue
+ *
+ * @param packet the packet to be put into the data queued
+ */
+ void queuePacket(DFSPacket packet) {
+ synchronized (dataQueue) {
+ if (packet == null) return;
+ packet.addTraceParent(Trace.currentSpan());
+ dataQueue.addLast(packet);
+ lastQueuedSeqno = packet.getSeqno();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Queued packet " + packet.getSeqno());
+ }
+ dataQueue.notifyAll();
+ }
+ }
+
+ /**
+ * For heartbeat packets, create buffer directly by new byte[]
+ * since heartbeats should not be blocked.
+ */
+ private DFSPacket createHeartbeatPacket() throws InterruptedIOException {
+ final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
+ return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false);
+ }
+
+ private static LoadingCache<DatanodeInfo, DatanodeInfo> initExcludedNodes(
+ long excludedNodesCacheExpiry) {
+ return CacheBuilder.newBuilder()
+ .expireAfterWrite(excludedNodesCacheExpiry, TimeUnit.MILLISECONDS)
+ .removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
+ @Override
+ public void onRemoval(
+ RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
+ LOG.info("Removing node " + notification.getKey()
+ + " from the excluded nodes list");
+ }
+ }).build(new CacheLoader<DatanodeInfo, DatanodeInfo>() {
+ @Override
+ public DatanodeInfo load(DatanodeInfo key) throws Exception {
+ return key;
+ }
+ });
+ }
+
+ private static <T> void arraycopy(T[] srcs, T[] dsts, int skipIndex) {
+ System.arraycopy(srcs, 0, dsts, 0, skipIndex);
+ System.arraycopy(srcs, skipIndex+1, dsts, skipIndex, dsts.length-skipIndex);
+ }
+
+ /**
+ * check if to persist blocks on namenode
+ *
+ * @return if to persist blocks on namenode
+ */
+ AtomicBoolean getPersistBlocks(){
+ return persistBlocks;
+ }
+
+ /**
+ * check if to append a chunk
+ *
+ * @param appendChunk if to append a chunk
+ */
+ void setAppendChunk(boolean appendChunk){
+ this.appendChunk = appendChunk;
+ }
+
+ /**
+ * get if to append a chunk
+ *
+ * @return if to append a chunk
+ */
+ boolean getAppendChunk(){
+ return appendChunk;
+ }
+
+ /**
+ * @return the last exception
+ */
+ LastExceptionInStreamer getLastException(){
+ return lastException;
+ }
+
+ /**
+ * set socket to null
+ */
+ void setSocketToNull() {
+ this.s = null;
+ }
+
+ /**
+ * return current sequence number and then increase it by 1
+ *
+ * @return current sequence number before increasing
+ */
+ long getAndIncCurrentSeqno() {
+ long old = this.currentSeqno;
+ this.currentSeqno++;
+ return old;
+ }
+
+ /**
+ * get last queued sequence number
+ *
+ * @return last queued sequence number
+ */
+ long getLastQueuedSeqno() {
+ return lastQueuedSeqno;
+ }
+
+ /**
+ * get the number of bytes of current block
+ *
+ * @return the number of bytes of current block
+ */
+ long getBytesCurBlock() {
+ return bytesCurBlock;
+ }
+
+ /**
+ * set the bytes of current block that have been written
+ *
+ * @param bytesCurBlock bytes of current block that have been written
+ */
+ void setBytesCurBlock(long bytesCurBlock) {
+ this.bytesCurBlock = bytesCurBlock;
+ }
+
+ /**
+ * increase bytes of current block by len.
+ *
+ * @param len how many bytes to increase to current block
+ */
+ void incBytesCurBlock(long len) {
+ this.bytesCurBlock += len;
+ }
+
+ /**
+ * set artificial slow down for unit test
+ *
+ * @param period artificial slow down
+ */
+ void setArtificialSlowdown(long period) {
+ this.artificialSlowdown = period;
+ }
+
+ /**
+ * if this streamer is to terminate
+ *
+ * @return if this streamer is to terminate
+ */
+ boolean streamerClosed(){
+ return streamerClosed;
+ }
+
+ void closeSocket() throws IOException {
+ if (s != null) {
+ s.close();
+ }
+ }
+
+ @Override
+ public String toString() {
+ return (block == null? null: block.getLocalBlock())
+ + "@" + Arrays.toString(getNodes());
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfigurationLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfigurationLoader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfigurationLoader.java
new file mode 100644
index 0000000..4a84f06
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/HdfsConfigurationLoader.java
@@ -0,0 +1,44 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * Load default HDFS configuration resources.
+ */
+@InterfaceAudience.Private
+class HdfsConfigurationLoader {
+
+ static {
+ // adds the default resources
+ Configuration.addDefaultResource("hdfs-default.xml");
+ Configuration.addDefaultResource("hdfs-site.xml");
+ }
+
+ /**
+ * This method is here so that when invoked, default resources are added if
+ * they haven't already been previously loaded. Upon loading this class, the
+ * static initializer block above will be executed to add the default
+ * resources. It is safe for this method to be called multiple times
+ * as the static initializer block will only get invoked once.
+ */
+ public static void init() {
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
new file mode 100644
index 0000000..f03e179
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
@@ -0,0 +1,43 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+@InterfaceAudience.Private
+public interface RemotePeerFactory {
+ /**
+ * @param addr The address to connect to.
+ * @param blockToken Token used during optional SASL negotiation
+ * @param datanodeId ID of destination DataNode
+ * @return A new Peer connected to the address.
+ *
+ * @throws IOException If there was an error connecting or creating
+ * the remote socket, encrypted stream, etc.
+ */
+ Peer newConnectedPeer(InetSocketAddress addr,
+ Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId)
+ throws IOException;
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java
new file mode 100644
index 0000000..ec17bb8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java
@@ -0,0 +1,35 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Thrown when an unknown cipher suite is encountered.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Stable
+public class UnknownCipherSuiteException extends IOException {
+ public UnknownCipherSuiteException(String msg) {
+ super(msg);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java
new file mode 100644
index 0000000..0aac8c8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java
@@ -0,0 +1,38 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class UnknownCryptoProtocolVersionException extends IOException {
+ private static final long serialVersionUID = 8957192l;
+
+ public UnknownCryptoProtocolVersionException() {
+ super();
+ }
+
+ public UnknownCryptoProtocolVersionException(String msg) {
+ super(msg);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
new file mode 100644
index 0000000..2655c40
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttr.NameSpace;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+@InterfaceAudience.Private
+public class XAttrHelper {
+
+ /**
+ * Build <code>XAttr</code> from xattr name with prefix.
+ */
+ public static XAttr buildXAttr(String name) {
+ return buildXAttr(name, null);
+ }
+
+ /**
+ * Build <code>XAttr</code> from name with prefix and value.
+ * Name can not be null. Value can be null. The name and prefix
+ * are validated.
+ * Both name and namespace are case sensitive.
+ */
+ public static XAttr buildXAttr(String name, byte[] value) {
+ Preconditions.checkNotNull(name, "XAttr name cannot be null.");
+
+ final int prefixIndex = name.indexOf(".");
+ if (prefixIndex < 3) {// Prefix length is at least 3.
+ throw new HadoopIllegalArgumentException("An XAttr name must be " +
+ "prefixed with user/trusted/security/system/raw, followed by a '.'");
+ } else if (prefixIndex == name.length() - 1) {
+ throw new HadoopIllegalArgumentException("XAttr name cannot be empty.");
+ }
+
+ NameSpace ns;
+ final String prefix = name.substring(0, prefixIndex);
+ if (StringUtils.equalsIgnoreCase(prefix, NameSpace.USER.toString())) {
+ ns = NameSpace.USER;
+ } else if (
+ StringUtils.equalsIgnoreCase(prefix, NameSpace.TRUSTED.toString())) {
+ ns = NameSpace.TRUSTED;
+ } else if (
+ StringUtils.equalsIgnoreCase(prefix, NameSpace.SYSTEM.toString())) {
+ ns = NameSpace.SYSTEM;
+ } else if (
+ StringUtils.equalsIgnoreCase(prefix, NameSpace.SECURITY.toString())) {
+ ns = NameSpace.SECURITY;
+ } else if (
+ StringUtils.equalsIgnoreCase(prefix, NameSpace.RAW.toString())) {
+ ns = NameSpace.RAW;
+ } else {
+ throw new HadoopIllegalArgumentException("An XAttr name must be " +
+ "prefixed with user/trusted/security/system/raw, followed by a '.'");
+ }
+ XAttr xAttr = (new XAttr.Builder()).setNameSpace(ns).setName(name.
+ substring(prefixIndex + 1)).setValue(value).build();
+
+ return xAttr;
+ }
+
+ /**
+ * Build xattr name with prefix as <code>XAttr</code> list.
+ */
+ public static List<XAttr> buildXAttrAsList(String name) {
+ XAttr xAttr = buildXAttr(name);
+ List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
+ xAttrs.add(xAttr);
+
+ return xAttrs;
+ }
+
+ /**
+ * Get value of first xattr from <code>XAttr</code> list
+ */
+ public static byte[] getFirstXAttrValue(List<XAttr> xAttrs) {
+ byte[] value = null;
+ XAttr xAttr = getFirstXAttr(xAttrs);
+ if (xAttr != null) {
+ value = xAttr.getValue();
+ if (value == null) {
+ value = new byte[0]; // xattr exists, but no value.
+ }
+ }
+ return value;
+ }
+
+ /**
+ * Get first xattr from <code>XAttr</code> list
+ */
+ public static XAttr getFirstXAttr(List<XAttr> xAttrs) {
+ if (xAttrs != null && !xAttrs.isEmpty()) {
+ return xAttrs.get(0);
+ }
+
+ return null;
+ }
+
+ /**
+ * Build xattr map from <code>XAttr</code> list, the key is
+ * xattr name with prefix, and value is xattr value.
+ */
+ public static Map<String, byte[]> buildXAttrMap(List<XAttr> xAttrs) {
+ if (xAttrs == null) {
+ return null;
+ }
+ Map<String, byte[]> xAttrMap = Maps.newHashMap();
+ for (XAttr xAttr : xAttrs) {
+ String name = getPrefixedName(xAttr);
+ byte[] value = xAttr.getValue();
+ if (value == null) {
+ value = new byte[0];
+ }
+ xAttrMap.put(name, value);
+ }
+
+ return xAttrMap;
+ }
+
+ /**
+ * Get name with prefix from <code>XAttr</code>
+ */
+ public static String getPrefixedName(XAttr xAttr) {
+ if (xAttr == null) {
+ return null;
+ }
+
+ return getPrefixedName(xAttr.getNameSpace(), xAttr.getName());
+ }
+
+ public static String getPrefixedName(XAttr.NameSpace ns, String name) {
+ return StringUtils.toLowerCase(ns.toString()) + "." + name;
+ }
+
+ /**
+ * Build <code>XAttr</code> list from xattr name list.
+ */
+ public static List<XAttr> buildXAttrs(List<String> names) {
+ if (names == null || names.isEmpty()) {
+ throw new HadoopIllegalArgumentException("XAttr names can not be " +
+ "null or empty.");
+ }
+
+ List<XAttr> xAttrs = Lists.newArrayListWithCapacity(names.size());
+ for (String name : names) {
+ xAttrs.add(buildXAttr(name, null));
+ }
+ return xAttrs;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
new file mode 100644
index 0000000..e8ac686
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
@@ -0,0 +1,113 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.client;
+
+import java.io.InputStream;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.crypto.CryptoInputStream;
+import org.apache.hadoop.hdfs.DFSInputStream;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * The Hdfs implementation of {@link FSDataInputStream}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class HdfsDataInputStream extends FSDataInputStream {
+ public HdfsDataInputStream(DFSInputStream in) throws IOException {
+ super(in);
+ }
+
+ public HdfsDataInputStream(CryptoInputStream in) throws IOException {
+ super(in);
+ Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream,
+ "CryptoInputStream should wrap a DFSInputStream");
+ }
+
+ private DFSInputStream getDFSInputStream() {
+ if (in instanceof CryptoInputStream) {
+ return (DFSInputStream) ((CryptoInputStream) in).getWrappedStream();
+ }
+ return (DFSInputStream) in;
+ }
+
+ /**
+ * Get a reference to the wrapped output stream. We always want to return the
+ * actual underlying InputStream, even when we're using a CryptoStream. e.g.
+ * in the delegated methods below.
+ *
+ * @return the underlying output stream
+ */
+ public InputStream getWrappedStream() {
+ return in;
+ }
+
+ /**
+ * Get the datanode from which the stream is currently reading.
+ */
+ public DatanodeInfo getCurrentDatanode() {
+ return getDFSInputStream().getCurrentDatanode();
+ }
+
+ /**
+ * Get the block containing the target position.
+ */
+ public ExtendedBlock getCurrentBlock() {
+ return getDFSInputStream().getCurrentBlock();
+ }
+
+ /**
+ * Get the collection of blocks that has already been located.
+ */
+ public List<LocatedBlock> getAllBlocks() throws IOException {
+ return getDFSInputStream().getAllBlocks();
+ }
+
+ /**
+ * Get the visible length of the file. It will include the length of the last
+ * block even if that is in UnderConstruction state.
+ *
+ * @return The visible length of the file.
+ */
+ public long getVisibleLength() throws IOException {
+ return getDFSInputStream().getFileLength();
+ }
+
+ /**
+ * Get statistics about the reads which this DFSInputStream has done.
+ * Note that because HdfsDataInputStream is buffered, these stats may
+ * be higher than you would expect just by adding up the number of
+ * bytes read through HdfsDataInputStream.
+ */
+ public DFSInputStream.ReadStatistics getReadStatistics() {
+ return getDFSInputStream().getReadStatistics();
+ }
+
+ public void clearReadStatistics() {
+ getDFSInputStream().clearReadStatistics();
+ }
+}
[02/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and
related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
deleted file mode 100644
index 4a016bd..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ /dev/null
@@ -1,1903 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status.SUCCESS;
-
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InterruptedIOException;
-import java.io.OutputStream;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.nio.channels.ClosedChannelException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
-import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
-import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
-import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
-import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
-import org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck;
-import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.Status;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException;
-import org.apache.hadoop.hdfs.util.ByteArrayManager;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.MultipleIOException;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.Time;
-import org.apache.htrace.NullScope;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceInfo;
-import org.apache.htrace.TraceScope;
-
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.cache.RemovalListener;
-import com.google.common.cache.RemovalNotification;
-
-/*********************************************************************
- *
- * The DataStreamer class is responsible for sending data packets to the
- * datanodes in the pipeline. It retrieves a new blockid and block locations
- * from the namenode, and starts streaming packets to the pipeline of
- * Datanodes. Every packet has a sequence number associated with
- * it. When all the packets for a block are sent out and acks for each
- * if them are received, the DataStreamer closes the current block.
- *
- * The DataStreamer thread picks up packets from the dataQueue, sends it to
- * the first datanode in the pipeline and moves it from the dataQueue to the
- * ackQueue. The ResponseProcessor receives acks from the datanodes. When an
- * successful ack for a packet is received from all datanodes, the
- * ResponseProcessor removes the corresponding packet from the ackQueue.
- *
- * In case of error, all outstanding packets are moved from ackQueue. A new
- * pipeline is setup by eliminating the bad datanode from the original
- * pipeline. The DataStreamer now starts sending packets from the dataQueue.
- *
- *********************************************************************/
-
-@InterfaceAudience.Private
-class DataStreamer extends Daemon {
- static final Log LOG = LogFactory.getLog(DataStreamer.class);
-
- /**
- * Create a socket for a write pipeline
- *
- * @param first the first datanode
- * @param length the pipeline length
- * @param client client
- * @return the socket connected to the first datanode
- */
- static Socket createSocketForPipeline(final DatanodeInfo first,
- final int length, final DFSClient client) throws IOException {
- final DfsClientConf conf = client.getConf();
- final String dnAddr = first.getXferAddr(conf.isConnectToDnViaHostname());
- if (LOG.isDebugEnabled()) {
- LOG.debug("Connecting to datanode " + dnAddr);
- }
- final InetSocketAddress isa = NetUtils.createSocketAddr(dnAddr);
- final Socket sock = client.socketFactory.createSocket();
- final int timeout = client.getDatanodeReadTimeout(length);
- NetUtils.connect(sock, isa, client.getRandomLocalInterfaceAddr(), conf.getSocketTimeout());
- sock.setSoTimeout(timeout);
- sock.setSendBufferSize(HdfsConstants.DEFAULT_DATA_SOCKET_SIZE);
- if (LOG.isDebugEnabled()) {
- LOG.debug("Send buf size " + sock.getSendBufferSize());
- }
- return sock;
- }
-
- /**
- * if this file is lazy persist
- *
- * @param stat the HdfsFileStatus of a file
- * @return if this file is lazy persist
- */
- static boolean isLazyPersist(HdfsFileStatus stat) {
- return stat.getStoragePolicy() == HdfsConstants.MEMORY_STORAGE_POLICY_ID;
- }
-
- /**
- * release a list of packets to ByteArrayManager
- *
- * @param packets packets to be release
- * @param bam ByteArrayManager
- */
- private static void releaseBuffer(List<DFSPacket> packets, ByteArrayManager bam) {
- for(DFSPacket p : packets) {
- p.releaseBuffer(bam);
- }
- packets.clear();
- }
-
- static class LastExceptionInStreamer {
- private IOException thrown;
-
- synchronized void set(Throwable t) {
- assert t != null;
- this.thrown = t instanceof IOException ?
- (IOException) t : new IOException(t);
- }
-
- synchronized void clear() {
- thrown = null;
- }
-
- /** Check if there already is an exception. */
- synchronized void check(boolean resetToNull) throws IOException {
- if (thrown != null) {
- if (LOG.isTraceEnabled()) {
- // wrap and print the exception to know when the check is called
- LOG.trace("Got Exception while checking", new Throwable(thrown));
- }
- final IOException e = thrown;
- if (resetToNull) {
- thrown = null;
- }
- throw e;
- }
- }
-
- synchronized void throwException4Close() throws IOException {
- check(false);
- throw new ClosedChannelException();
- }
- }
-
- static class ErrorState {
- private boolean error = false;
- private int badNodeIndex = -1;
- private int restartingNodeIndex = -1;
- private long restartingNodeDeadline = 0;
- private final long datanodeRestartTimeout;
-
- ErrorState(long datanodeRestartTimeout) {
- this.datanodeRestartTimeout = datanodeRestartTimeout;
- }
-
- synchronized void reset() {
- error = false;
- badNodeIndex = -1;
- restartingNodeIndex = -1;
- restartingNodeDeadline = 0;
- }
-
- synchronized boolean hasError() {
- return error;
- }
-
- synchronized boolean hasDatanodeError() {
- return error && isNodeMarked();
- }
-
- synchronized void setError(boolean err) {
- this.error = err;
- }
-
- synchronized void setBadNodeIndex(int index) {
- this.badNodeIndex = index;
- }
-
- synchronized int getBadNodeIndex() {
- return badNodeIndex;
- }
-
- synchronized int getRestartingNodeIndex() {
- return restartingNodeIndex;
- }
-
- synchronized void initRestartingNode(int i, String message) {
- restartingNodeIndex = i;
- restartingNodeDeadline = Time.monotonicNow() + datanodeRestartTimeout;
- // If the data streamer has already set the primary node
- // bad, clear it. It is likely that the write failed due to
- // the DN shutdown. Even if it was a real failure, the pipeline
- // recovery will take care of it.
- badNodeIndex = -1;
- LOG.info(message);
- }
-
- synchronized boolean isRestartingNode() {
- return restartingNodeIndex >= 0;
- }
-
- synchronized boolean isNodeMarked() {
- return badNodeIndex >= 0 || isRestartingNode();
- }
-
- /**
- * This method is used when no explicit error report was received, but
- * something failed. The first node is a suspect or unsure about the cause
- * so that it is marked as failed.
- */
- synchronized void markFirstNodeIfNotMarked() {
- // There should be no existing error and no ongoing restart.
- if (!isNodeMarked()) {
- badNodeIndex = 0;
- }
- }
-
- synchronized void adjustState4RestartingNode() {
- // Just took care of a node error while waiting for a node restart
- if (restartingNodeIndex >= 0) {
- // If the error came from a node further away than the restarting
- // node, the restart must have been complete.
- if (badNodeIndex > restartingNodeIndex) {
- restartingNodeIndex = -1;
- } else if (badNodeIndex < restartingNodeIndex) {
- // the node index has shifted.
- restartingNodeIndex--;
- } else {
- throw new IllegalStateException("badNodeIndex = " + badNodeIndex
- + " = restartingNodeIndex = " + restartingNodeIndex);
- }
- }
-
- if (!isRestartingNode()) {
- error = false;
- }
- badNodeIndex = -1;
- }
-
- synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) {
- if (restartingNodeIndex >= 0) {
- if (!error) {
- throw new IllegalStateException("error=false while checking" +
- " restarting node deadline");
- }
-
- // check badNodeIndex
- if (badNodeIndex == restartingNodeIndex) {
- // ignore, if came from the restarting node
- badNodeIndex = -1;
- }
- // not within the deadline
- if (Time.monotonicNow() >= restartingNodeDeadline) {
- // expired. declare the restarting node dead
- restartingNodeDeadline = 0;
- final int i = restartingNodeIndex;
- restartingNodeIndex = -1;
- LOG.warn("Datanode " + i + " did not restart within "
- + datanodeRestartTimeout + "ms: " + nodes[i]);
- // Mark the restarting node as failed. If there is any other failed
- // node during the last pipeline construction attempt, it will not be
- // overwritten/dropped. In this case, the restarting node will get
- // excluded in the following attempt, if it still does not come up.
- if (badNodeIndex == -1) {
- badNodeIndex = i;
- }
- }
- }
- }
- }
-
- private volatile boolean streamerClosed = false;
- private ExtendedBlock block; // its length is number of bytes acked
- private Token<BlockTokenIdentifier> accessToken;
- private DataOutputStream blockStream;
- private DataInputStream blockReplyStream;
- private ResponseProcessor response = null;
- private volatile DatanodeInfo[] nodes = null; // list of targets for current block
- private volatile StorageType[] storageTypes = null;
- private volatile String[] storageIDs = null;
- private final ErrorState errorState;
-
- private BlockConstructionStage stage; // block construction stage
- private long bytesSent = 0; // number of bytes that've been sent
- private final boolean isLazyPersistFile;
-
- /** Nodes have been used in the pipeline before and have failed. */
- private final List<DatanodeInfo> failed = new ArrayList<>();
- /** The last ack sequence number before pipeline failure. */
- private long lastAckedSeqnoBeforeFailure = -1;
- private int pipelineRecoveryCount = 0;
- /** Has the current block been hflushed? */
- private boolean isHflushed = false;
- /** Append on an existing block? */
- private final boolean isAppend;
-
- private long currentSeqno = 0;
- private long lastQueuedSeqno = -1;
- private long lastAckedSeqno = -1;
- private long bytesCurBlock = 0; // bytes written in current block
- private final LastExceptionInStreamer lastException = new LastExceptionInStreamer();
- private Socket s;
-
- private final DFSClient dfsClient;
- private final String src;
- /** Only for DataTransferProtocol.writeBlock(..) */
- private final DataChecksum checksum4WriteBlock;
- private final Progressable progress;
- private final HdfsFileStatus stat;
- // appending to existing partial block
- private volatile boolean appendChunk = false;
- // both dataQueue and ackQueue are protected by dataQueue lock
- private final LinkedList<DFSPacket> dataQueue = new LinkedList<>();
- private final LinkedList<DFSPacket> ackQueue = new LinkedList<>();
- private final AtomicReference<CachingStrategy> cachingStrategy;
- private final ByteArrayManager byteArrayManager;
- //persist blocks on namenode
- private final AtomicBoolean persistBlocks = new AtomicBoolean(false);
- private boolean failPacket = false;
- private final long dfsclientSlowLogThresholdMs;
- private long artificialSlowdown = 0;
- // List of congested data nodes. The stream will back off if the DataNodes
- // are congested
- private final List<DatanodeInfo> congestedNodes = new ArrayList<>();
- private static final int CONGESTION_BACKOFF_MEAN_TIME_IN_MS = 5000;
- private static final int CONGESTION_BACK_OFF_MAX_TIME_IN_MS =
- CONGESTION_BACKOFF_MEAN_TIME_IN_MS * 10;
- private int lastCongestionBackoffTime;
-
- private final LoadingCache<DatanodeInfo, DatanodeInfo> excludedNodes;
- private final String[] favoredNodes;
-
- private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src,
- Progressable progress, DataChecksum checksum,
- AtomicReference<CachingStrategy> cachingStrategy,
- ByteArrayManager byteArrayManage,
- boolean isAppend, String[] favoredNodes) {
- this.dfsClient = dfsClient;
- this.src = src;
- this.progress = progress;
- this.stat = stat;
- this.checksum4WriteBlock = checksum;
- this.cachingStrategy = cachingStrategy;
- this.byteArrayManager = byteArrayManage;
- this.isLazyPersistFile = isLazyPersist(stat);
- this.isAppend = isAppend;
- this.favoredNodes = favoredNodes;
-
- final DfsClientConf conf = dfsClient.getConf();
- this.dfsclientSlowLogThresholdMs = conf.getSlowIoWarningThresholdMs();
- this.excludedNodes = initExcludedNodes(conf.getExcludedNodesCacheExpiry());
- this.errorState = new ErrorState(conf.getDatanodeRestartTimeout());
- }
-
- /**
- * construction with tracing info
- */
- DataStreamer(HdfsFileStatus stat, ExtendedBlock block, DFSClient dfsClient,
- String src, Progressable progress, DataChecksum checksum,
- AtomicReference<CachingStrategy> cachingStrategy,
- ByteArrayManager byteArrayManage, String[] favoredNodes) {
- this(stat, dfsClient, src, progress, checksum, cachingStrategy,
- byteArrayManage, false, favoredNodes);
- this.block = block;
- stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
- }
-
- /**
- * Construct a data streamer for appending to the last partial block
- * @param lastBlock last block of the file to be appended
- * @param stat status of the file to be appended
- * @throws IOException if error occurs
- */
- DataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat, DFSClient dfsClient,
- String src, Progressable progress, DataChecksum checksum,
- AtomicReference<CachingStrategy> cachingStrategy,
- ByteArrayManager byteArrayManage) throws IOException {
- this(stat, dfsClient, src, progress, checksum, cachingStrategy,
- byteArrayManage, true, null);
- stage = BlockConstructionStage.PIPELINE_SETUP_APPEND;
- block = lastBlock.getBlock();
- bytesSent = block.getNumBytes();
- accessToken = lastBlock.getBlockToken();
- }
-
- /**
- * Set pipeline in construction
- *
- * @param lastBlock the last block of a file
- * @throws IOException
- */
- void setPipelineInConstruction(LocatedBlock lastBlock) throws IOException{
- // setup pipeline to append to the last block XXX retries??
- setPipeline(lastBlock);
- if (nodes.length < 1) {
- throw new IOException("Unable to retrieve blocks locations " +
- " for last block " + block +
- "of file " + src);
- }
- }
-
- private void setPipeline(LocatedBlock lb) {
- setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs());
- }
-
- private void setPipeline(DatanodeInfo[] nodes, StorageType[] storageTypes,
- String[] storageIDs) {
- this.nodes = nodes;
- this.storageTypes = storageTypes;
- this.storageIDs = storageIDs;
- }
-
- /**
- * Initialize for data streaming
- */
- private void initDataStreaming() {
- this.setName("DataStreamer for file " + src +
- " block " + block);
- response = new ResponseProcessor(nodes);
- response.start();
- stage = BlockConstructionStage.DATA_STREAMING;
- }
-
- private void endBlock() {
- if(LOG.isDebugEnabled()) {
- LOG.debug("Closing old block " + block);
- }
- this.setName("DataStreamer for file " + src);
- closeResponder();
- closeStream();
- setPipeline(null, null, null);
- stage = BlockConstructionStage.PIPELINE_SETUP_CREATE;
- }
-
- private boolean shouldStop() {
- return streamerClosed || errorState.hasError() || !dfsClient.clientRunning;
- }
-
- /*
- * streamer thread is the only thread that opens streams to datanode,
- * and closes them. Any error recovery is also done by this thread.
- */
- @Override
- public void run() {
- long lastPacket = Time.monotonicNow();
- TraceScope scope = NullScope.INSTANCE;
- while (!streamerClosed && dfsClient.clientRunning) {
- // if the Responder encountered an error, shutdown Responder
- if (errorState.hasError() && response != null) {
- try {
- response.close();
- response.join();
- response = null;
- } catch (InterruptedException e) {
- LOG.warn("Caught exception", e);
- }
- }
-
- DFSPacket one;
- try {
- // process datanode IO errors if any
- boolean doSleep = processDatanodeError();
-
- final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2;
- synchronized (dataQueue) {
- // wait for a packet to be sent.
- long now = Time.monotonicNow();
- while ((!shouldStop() && dataQueue.size() == 0 &&
- (stage != BlockConstructionStage.DATA_STREAMING ||
- stage == BlockConstructionStage.DATA_STREAMING &&
- now - lastPacket < halfSocketTimeout)) || doSleep ) {
- long timeout = halfSocketTimeout - (now-lastPacket);
- timeout = timeout <= 0 ? 1000 : timeout;
- timeout = (stage == BlockConstructionStage.DATA_STREAMING)?
- timeout : 1000;
- try {
- dataQueue.wait(timeout);
- } catch (InterruptedException e) {
- LOG.warn("Caught exception", e);
- }
- doSleep = false;
- now = Time.monotonicNow();
- }
- if (shouldStop()) {
- continue;
- }
- // get packet to be sent.
- if (dataQueue.isEmpty()) {
- one = createHeartbeatPacket();
- } else {
- try {
- backOffIfNecessary();
- } catch (InterruptedException e) {
- LOG.warn("Caught exception", e);
- }
- one = dataQueue.getFirst(); // regular data packet
- long parents[] = one.getTraceParents();
- if (parents.length > 0) {
- scope = Trace.startSpan("dataStreamer", new TraceInfo(0, parents[0]));
- // TODO: use setParents API once it's available from HTrace 3.2
- // scope = Trace.startSpan("dataStreamer", Sampler.ALWAYS);
- // scope.getSpan().setParents(parents);
- }
- }
- }
-
- // get new block from namenode.
- if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
- if(LOG.isDebugEnabled()) {
- LOG.debug("Allocating new block");
- }
- setPipeline(nextBlockOutputStream());
- initDataStreaming();
- } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) {
- if(LOG.isDebugEnabled()) {
- LOG.debug("Append to block " + block);
- }
- setupPipelineForAppendOrRecovery();
- if (streamerClosed) {
- continue;
- }
- initDataStreaming();
- }
-
- long lastByteOffsetInBlock = one.getLastByteOffsetBlock();
- if (lastByteOffsetInBlock > stat.getBlockSize()) {
- throw new IOException("BlockSize " + stat.getBlockSize() +
- " is smaller than data size. " +
- " Offset of packet in block " +
- lastByteOffsetInBlock +
- " Aborting file " + src);
- }
-
- if (one.isLastPacketInBlock()) {
- // wait for all data packets have been successfully acked
- synchronized (dataQueue) {
- while (!shouldStop() && ackQueue.size() != 0) {
- try {
- // wait for acks to arrive from datanodes
- dataQueue.wait(1000);
- } catch (InterruptedException e) {
- LOG.warn("Caught exception", e);
- }
- }
- }
- if (shouldStop()) {
- continue;
- }
- stage = BlockConstructionStage.PIPELINE_CLOSE;
- }
-
- // send the packet
- Span span = null;
- synchronized (dataQueue) {
- // move packet from dataQueue to ackQueue
- if (!one.isHeartbeatPacket()) {
- span = scope.detach();
- one.setTraceSpan(span);
- dataQueue.removeFirst();
- ackQueue.addLast(one);
- dataQueue.notifyAll();
- }
- }
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("DataStreamer block " + block +
- " sending packet " + one);
- }
-
- // write out data to remote datanode
- TraceScope writeScope = Trace.startSpan("writeTo", span);
- try {
- one.writeTo(blockStream);
- blockStream.flush();
- } catch (IOException e) {
- // HDFS-3398 treat primary DN is down since client is unable to
- // write to primary DN. If a failed or restarting node has already
- // been recorded by the responder, the following call will have no
- // effect. Pipeline recovery can handle only one node error at a
- // time. If the primary node fails again during the recovery, it
- // will be taken out then.
- errorState.markFirstNodeIfNotMarked();
- throw e;
- } finally {
- writeScope.close();
- }
- lastPacket = Time.monotonicNow();
-
- // update bytesSent
- long tmpBytesSent = one.getLastByteOffsetBlock();
- if (bytesSent < tmpBytesSent) {
- bytesSent = tmpBytesSent;
- }
-
- if (shouldStop()) {
- continue;
- }
-
- // Is this block full?
- if (one.isLastPacketInBlock()) {
- // wait for the close packet has been acked
- synchronized (dataQueue) {
- while (!shouldStop() && ackQueue.size() != 0) {
- dataQueue.wait(1000);// wait for acks to arrive from datanodes
- }
- }
- if (shouldStop()) {
- continue;
- }
-
- endBlock();
- }
- if (progress != null) { progress.progress(); }
-
- // This is used by unit test to trigger race conditions.
- if (artificialSlowdown != 0 && dfsClient.clientRunning) {
- Thread.sleep(artificialSlowdown);
- }
- } catch (Throwable e) {
- // Log warning if there was a real error.
- if (!errorState.isRestartingNode()) {
- // Since their messages are descriptive enough, do not always
- // log a verbose stack-trace WARN for quota exceptions.
- if (e instanceof QuotaExceededException) {
- LOG.debug("DataStreamer Quota Exception", e);
- } else {
- LOG.warn("DataStreamer Exception", e);
- }
- }
- lastException.set(e);
- assert !(e instanceof NullPointerException);
- errorState.setError(true);
- if (!errorState.isNodeMarked()) {
- // Not a datanode issue
- streamerClosed = true;
- }
- } finally {
- scope.close();
- }
- }
- closeInternal();
- }
-
- private void closeInternal() {
- closeResponder(); // close and join
- closeStream();
- streamerClosed = true;
- release();
- synchronized (dataQueue) {
- dataQueue.notifyAll();
- }
- }
-
- /**
- * release the DFSPackets in the two queues
- *
- */
- void release() {
- synchronized (dataQueue) {
- releaseBuffer(dataQueue, byteArrayManager);
- releaseBuffer(ackQueue, byteArrayManager);
- }
- }
-
- /**
- * wait for the ack of seqno
- *
- * @param seqno the sequence number to be acked
- * @throws IOException
- */
- void waitForAckedSeqno(long seqno) throws IOException {
- TraceScope scope = Trace.startSpan("waitForAckedSeqno", Sampler.NEVER);
- try {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Waiting for ack for: " + seqno);
- }
- long begin = Time.monotonicNow();
- try {
- synchronized (dataQueue) {
- while (!streamerClosed) {
- checkClosed();
- if (lastAckedSeqno >= seqno) {
- break;
- }
- try {
- dataQueue.wait(1000); // when we receive an ack, we notify on
- // dataQueue
- } catch (InterruptedException ie) {
- throw new InterruptedIOException(
- "Interrupted while waiting for data to be acknowledged by pipeline");
- }
- }
- }
- checkClosed();
- } catch (ClosedChannelException e) {
- }
- long duration = Time.monotonicNow() - begin;
- if (duration > dfsclientSlowLogThresholdMs) {
- LOG.warn("Slow waitForAckedSeqno took " + duration
- + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
- }
- } finally {
- scope.close();
- }
- }
-
- /**
- * wait for space of dataQueue and queue the packet
- *
- * @param packet the DFSPacket to be queued
- * @throws IOException
- */
- void waitAndQueuePacket(DFSPacket packet) throws IOException {
- synchronized (dataQueue) {
- try {
- // If queue is full, then wait till we have enough space
- boolean firstWait = true;
- try {
- while (!streamerClosed && dataQueue.size() + ackQueue.size() >
- dfsClient.getConf().getWriteMaxPackets()) {
- if (firstWait) {
- Span span = Trace.currentSpan();
- if (span != null) {
- span.addTimelineAnnotation("dataQueue.wait");
- }
- firstWait = false;
- }
- try {
- dataQueue.wait();
- } catch (InterruptedException e) {
- // If we get interrupted while waiting to queue data, we still need to get rid
- // of the current packet. This is because we have an invariant that if
- // currentPacket gets full, it will get queued before the next writeChunk.
- //
- // Rather than wait around for space in the queue, we should instead try to
- // return to the caller as soon as possible, even though we slightly overrun
- // the MAX_PACKETS length.
- Thread.currentThread().interrupt();
- break;
- }
- }
- } finally {
- Span span = Trace.currentSpan();
- if ((span != null) && (!firstWait)) {
- span.addTimelineAnnotation("end.wait");
- }
- }
- checkClosed();
- queuePacket(packet);
- } catch (ClosedChannelException e) {
- }
- }
- }
-
- /*
- * close the streamer, should be called only by an external thread
- * and only after all data to be sent has been flushed to datanode.
- *
- * Interrupt this data streamer if force is true
- *
- * @param force if this data stream is forced to be closed
- */
- void close(boolean force) {
- streamerClosed = true;
- synchronized (dataQueue) {
- dataQueue.notifyAll();
- }
- if (force) {
- this.interrupt();
- }
- }
-
-
- private void checkClosed() throws IOException {
- if (streamerClosed) {
- lastException.throwException4Close();
- }
- }
-
- private void closeResponder() {
- if (response != null) {
- try {
- response.close();
- response.join();
- } catch (InterruptedException e) {
- LOG.warn("Caught exception", e);
- } finally {
- response = null;
- }
- }
- }
-
- private void closeStream() {
- final MultipleIOException.Builder b = new MultipleIOException.Builder();
-
- if (blockStream != null) {
- try {
- blockStream.close();
- } catch (IOException e) {
- b.add(e);
- } finally {
- blockStream = null;
- }
- }
- if (blockReplyStream != null) {
- try {
- blockReplyStream.close();
- } catch (IOException e) {
- b.add(e);
- } finally {
- blockReplyStream = null;
- }
- }
- if (null != s) {
- try {
- s.close();
- } catch (IOException e) {
- b.add(e);
- } finally {
- s = null;
- }
- }
-
- final IOException ioe = b.build();
- if (ioe != null) {
- lastException.set(ioe);
- }
- }
-
- /**
- * Examine whether it is worth waiting for a node to restart.
- * @param index the node index
- */
- boolean shouldWaitForRestart(int index) {
- // Only one node in the pipeline.
- if (nodes.length == 1) {
- return true;
- }
-
- // Is it a local node?
- InetAddress addr = null;
- try {
- addr = InetAddress.getByName(nodes[index].getIpAddr());
- } catch (java.net.UnknownHostException e) {
- // we are passing an ip address. this should not happen.
- assert false;
- }
-
- if (addr != null && NetUtils.isLocalAddress(addr)) {
- return true;
- }
- return false;
- }
-
- //
- // Processes responses from the datanodes. A packet is removed
- // from the ackQueue when its response arrives.
- //
- private class ResponseProcessor extends Daemon {
-
- private volatile boolean responderClosed = false;
- private DatanodeInfo[] targets = null;
- private boolean isLastPacketInBlock = false;
-
- ResponseProcessor (DatanodeInfo[] targets) {
- this.targets = targets;
- }
-
- @Override
- public void run() {
-
- setName("ResponseProcessor for block " + block);
- PipelineAck ack = new PipelineAck();
-
- TraceScope scope = NullScope.INSTANCE;
- while (!responderClosed && dfsClient.clientRunning && !isLastPacketInBlock) {
- // process responses from datanodes.
- try {
- // read an ack from the pipeline
- long begin = Time.monotonicNow();
- ack.readFields(blockReplyStream);
- long duration = Time.monotonicNow() - begin;
- if (duration > dfsclientSlowLogThresholdMs
- && ack.getSeqno() != DFSPacket.HEART_BEAT_SEQNO) {
- LOG.warn("Slow ReadProcessor read fields took " + duration
- + "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
- + ack + ", targets: " + Arrays.asList(targets));
- } else if (LOG.isDebugEnabled()) {
- LOG.debug("DFSClient " + ack);
- }
-
- long seqno = ack.getSeqno();
- // processes response status from datanodes.
- ArrayList<DatanodeInfo> congestedNodesFromAck = new ArrayList<>();
- for (int i = ack.getNumOfReplies()-1; i >=0 && dfsClient.clientRunning; i--) {
- final Status reply = PipelineAck.getStatusFromHeader(ack
- .getHeaderFlag(i));
- if (PipelineAck.getECNFromHeader(ack.getHeaderFlag(i)) ==
- PipelineAck.ECN.CONGESTED) {
- congestedNodesFromAck.add(targets[i]);
- }
- // Restart will not be treated differently unless it is
- // the local node or the only one in the pipeline.
- if (PipelineAck.isRestartOOBStatus(reply) &&
- shouldWaitForRestart(i)) {
- final String message = "Datanode " + i + " is restarting: "
- + targets[i];
- errorState.initRestartingNode(i, message);
- throw new IOException(message);
- }
- // node error
- if (reply != SUCCESS) {
- errorState.setBadNodeIndex(i); // mark bad datanode
- throw new IOException("Bad response " + reply +
- " for " + block + " from datanode " + targets[i]);
- }
- }
-
- if (!congestedNodesFromAck.isEmpty()) {
- synchronized (congestedNodes) {
- congestedNodes.clear();
- congestedNodes.addAll(congestedNodesFromAck);
- }
- } else {
- synchronized (congestedNodes) {
- congestedNodes.clear();
- lastCongestionBackoffTime = 0;
- }
- }
-
- assert seqno != PipelineAck.UNKOWN_SEQNO :
- "Ack for unknown seqno should be a failed ack: " + ack;
- if (seqno == DFSPacket.HEART_BEAT_SEQNO) { // a heartbeat ack
- continue;
- }
-
- // a success ack for a data packet
- DFSPacket one;
- synchronized (dataQueue) {
- one = ackQueue.getFirst();
- }
- if (one.getSeqno() != seqno) {
- throw new IOException("ResponseProcessor: Expecting seqno " +
- " for block " + block +
- one.getSeqno() + " but received " + seqno);
- }
- isLastPacketInBlock = one.isLastPacketInBlock();
-
- // Fail the packet write for testing in order to force a
- // pipeline recovery.
- if (DFSClientFaultInjector.get().failPacket() &&
- isLastPacketInBlock) {
- failPacket = true;
- throw new IOException(
- "Failing the last packet for testing.");
- }
-
- // update bytesAcked
- block.setNumBytes(one.getLastByteOffsetBlock());
-
- synchronized (dataQueue) {
- scope = Trace.continueSpan(one.getTraceSpan());
- one.setTraceSpan(null);
- lastAckedSeqno = seqno;
- ackQueue.removeFirst();
- dataQueue.notifyAll();
-
- one.releaseBuffer(byteArrayManager);
- }
- } catch (Exception e) {
- if (!responderClosed) {
- lastException.set(e);
- errorState.setError(true);
- errorState.markFirstNodeIfNotMarked();
- synchronized (dataQueue) {
- dataQueue.notifyAll();
- }
- if (!errorState.isRestartingNode()) {
- LOG.warn("Exception for " + block, e);
- }
- responderClosed = true;
- }
- } finally {
- scope.close();
- }
- }
- }
-
- void close() {
- responderClosed = true;
- this.interrupt();
- }
- }
-
- /**
- * If this stream has encountered any errors, shutdown threads
- * and mark the stream as closed.
- *
- * @return true if it should sleep for a while after returning.
- */
- private boolean processDatanodeError() throws IOException {
- if (!errorState.hasDatanodeError()) {
- return false;
- }
- if (response != null) {
- LOG.info("Error Recovery for " + block +
- " waiting for responder to exit. ");
- return true;
- }
- closeStream();
-
- // move packets from ack queue to front of the data queue
- synchronized (dataQueue) {
- dataQueue.addAll(0, ackQueue);
- ackQueue.clear();
- }
-
- // Record the new pipeline failure recovery.
- if (lastAckedSeqnoBeforeFailure != lastAckedSeqno) {
- lastAckedSeqnoBeforeFailure = lastAckedSeqno;
- pipelineRecoveryCount = 1;
- } else {
- // If we had to recover the pipeline five times in a row for the
- // same packet, this client likely has corrupt data or corrupting
- // during transmission.
- if (++pipelineRecoveryCount > 5) {
- LOG.warn("Error recovering pipeline for writing " +
- block + ". Already retried 5 times for the same packet.");
- lastException.set(new IOException("Failing write. Tried pipeline " +
- "recovery 5 times without success."));
- streamerClosed = true;
- return false;
- }
- }
- boolean doSleep = setupPipelineForAppendOrRecovery();
-
- if (!streamerClosed && dfsClient.clientRunning) {
- if (stage == BlockConstructionStage.PIPELINE_CLOSE) {
-
- // If we had an error while closing the pipeline, we go through a fast-path
- // where the BlockReceiver does not run. Instead, the DataNode just finalizes
- // the block immediately during the 'connect ack' process. So, we want to pull
- // the end-of-block packet from the dataQueue, since we don't actually have
- // a true pipeline to send it over.
- //
- // We also need to set lastAckedSeqno to the end-of-block Packet's seqno, so that
- // a client waiting on close() will be aware that the flush finished.
- synchronized (dataQueue) {
- DFSPacket endOfBlockPacket = dataQueue.remove(); // remove the end of block packet
- Span span = endOfBlockPacket.getTraceSpan();
- if (span != null) {
- // Close any trace span associated with this Packet
- TraceScope scope = Trace.continueSpan(span);
- scope.close();
- }
- assert endOfBlockPacket.isLastPacketInBlock();
- assert lastAckedSeqno == endOfBlockPacket.getSeqno() - 1;
- lastAckedSeqno = endOfBlockPacket.getSeqno();
- dataQueue.notifyAll();
- }
- endBlock();
- } else {
- initDataStreaming();
- }
- }
-
- return doSleep;
- }
-
- void setHflush() {
- isHflushed = true;
- }
-
- private int findNewDatanode(final DatanodeInfo[] original
- ) throws IOException {
- if (nodes.length != original.length + 1) {
- throw new IOException(
- new StringBuilder()
- .append("Failed to replace a bad datanode on the existing pipeline ")
- .append("due to no more good datanodes being available to try. ")
- .append("(Nodes: current=").append(Arrays.asList(nodes))
- .append(", original=").append(Arrays.asList(original)).append("). ")
- .append("The current failed datanode replacement policy is ")
- .append(dfsClient.dtpReplaceDatanodeOnFailure).append(", and ")
- .append("a client may configure this via '")
- .append(BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY)
- .append("' in its configuration.")
- .toString());
- }
- for(int i = 0; i < nodes.length; i++) {
- int j = 0;
- for(; j < original.length && !nodes[i].equals(original[j]); j++);
- if (j == original.length) {
- return i;
- }
- }
- throw new IOException("Failed: new datanode not found: nodes="
- + Arrays.asList(nodes) + ", original=" + Arrays.asList(original));
- }
-
- private void addDatanode2ExistingPipeline() throws IOException {
- if (DataTransferProtocol.LOG.isDebugEnabled()) {
- DataTransferProtocol.LOG.debug("lastAckedSeqno = " + lastAckedSeqno);
- }
- /*
- * Is data transfer necessary? We have the following cases.
- *
- * Case 1: Failure in Pipeline Setup
- * - Append
- * + Transfer the stored replica, which may be a RBW or a finalized.
- * - Create
- * + If no data, then no transfer is required.
- * + If there are data written, transfer RBW. This case may happens
- * when there are streaming failure earlier in this pipeline.
- *
- * Case 2: Failure in Streaming
- * - Append/Create:
- * + transfer RBW
- *
- * Case 3: Failure in Close
- * - Append/Create:
- * + no transfer, let NameNode replicates the block.
- */
- if (!isAppend && lastAckedSeqno < 0
- && stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) {
- //no data have been written
- return;
- } else if (stage == BlockConstructionStage.PIPELINE_CLOSE
- || stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) {
- //pipeline is closing
- return;
- }
-
- //get a new datanode
- final DatanodeInfo[] original = nodes;
- final LocatedBlock lb = dfsClient.namenode.getAdditionalDatanode(
- src, stat.getFileId(), block, nodes, storageIDs,
- failed.toArray(new DatanodeInfo[failed.size()]),
- 1, dfsClient.clientName);
- setPipeline(lb);
-
- //find the new datanode
- final int d = findNewDatanode(original);
-
- //transfer replica
- final DatanodeInfo src = d == 0? nodes[1]: nodes[d - 1];
- final DatanodeInfo[] targets = {nodes[d]};
- final StorageType[] targetStorageTypes = {storageTypes[d]};
- transfer(src, targets, targetStorageTypes, lb.getBlockToken());
- }
-
- private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets,
- final StorageType[] targetStorageTypes,
- final Token<BlockTokenIdentifier> blockToken) throws IOException {
- //transfer replica to the new datanode
- Socket sock = null;
- DataOutputStream out = null;
- DataInputStream in = null;
- try {
- sock = createSocketForPipeline(src, 2, dfsClient);
- final long writeTimeout = dfsClient.getDatanodeWriteTimeout(2);
- final long readTimeout = dfsClient.getDatanodeReadTimeout(2);
-
- OutputStream unbufOut = NetUtils.getOutputStream(sock, writeTimeout);
- InputStream unbufIn = NetUtils.getInputStream(sock, readTimeout);
- IOStreamPair saslStreams = dfsClient.saslClient.socketSend(sock,
- unbufOut, unbufIn, dfsClient, blockToken, src);
- unbufOut = saslStreams.out;
- unbufIn = saslStreams.in;
- out = new DataOutputStream(new BufferedOutputStream(unbufOut,
- DFSUtil.getSmallBufferSize(dfsClient.getConfiguration())));
- in = new DataInputStream(unbufIn);
-
- //send the TRANSFER_BLOCK request
- new Sender(out).transferBlock(block, blockToken, dfsClient.clientName,
- targets, targetStorageTypes);
- out.flush();
-
- //ack
- BlockOpResponseProto response =
- BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
- if (SUCCESS != response.getStatus()) {
- throw new IOException("Failed to add a datanode");
- }
- } finally {
- IOUtils.closeStream(in);
- IOUtils.closeStream(out);
- IOUtils.closeSocket(sock);
- }
- }
-
- /**
- * Open a DataStreamer to a DataNode pipeline so that
- * it can be written to.
- * This happens when a file is appended or data streaming fails
- * It keeps on trying until a pipeline is setup
- */
- private boolean setupPipelineForAppendOrRecovery() throws IOException {
- // check number of datanodes
- if (nodes == null || nodes.length == 0) {
- String msg = "Could not get block locations. " + "Source file \""
- + src + "\" - Aborting...";
- LOG.warn(msg);
- lastException.set(new IOException(msg));
- streamerClosed = true;
- return false;
- }
-
- boolean success = false;
- long newGS = 0L;
- while (!success && !streamerClosed && dfsClient.clientRunning) {
- if (!handleRestartingDatanode()) {
- return false;
- }
-
- final boolean isRecovery = errorState.hasError();
- if (!handleBadDatanode()) {
- return false;
- }
-
- handleDatanodeReplacement();
-
- // get a new generation stamp and an access token
- final LocatedBlock lb = updateBlockForPipeline();
- newGS = lb.getBlock().getGenerationStamp();
- accessToken = lb.getBlockToken();
-
- // set up the pipeline again with the remaining nodes
- success = createBlockOutputStream(nodes, storageTypes, newGS, isRecovery);
-
- failPacket4Testing();
-
- errorState.checkRestartingNodeDeadline(nodes);
- } // while
-
- if (success) {
- block = updatePipeline(newGS);
- }
- return false; // do not sleep, continue processing
- }
-
- /**
- * Sleep if a node is restarting.
- * This process is repeated until the deadline or the node starts back up.
- * @return true if it should continue.
- */
- private boolean handleRestartingDatanode() {
- if (errorState.isRestartingNode()) {
- // 4 seconds or the configured deadline period, whichever is shorter.
- // This is the retry interval and recovery will be retried in this
- // interval until timeout or success.
- final long delay = Math.min(errorState.datanodeRestartTimeout, 4000L);
- try {
- Thread.sleep(delay);
- } catch (InterruptedException ie) {
- lastException.set(new IOException(
- "Interrupted while waiting for restarting "
- + nodes[errorState.getRestartingNodeIndex()]));
- streamerClosed = true;
- return false;
- }
- }
- return true;
- }
-
- /**
- * Remove bad node from list of nodes if badNodeIndex was set.
- * @return true if it should continue.
- */
- private boolean handleBadDatanode() {
- final int badNodeIndex = errorState.getBadNodeIndex();
- if (badNodeIndex >= 0) {
- if (nodes.length <= 1) {
- lastException.set(new IOException("All datanodes "
- + Arrays.toString(nodes) + " are bad. Aborting..."));
- streamerClosed = true;
- return false;
- }
-
- LOG.warn("Error Recovery for " + block + " in pipeline "
- + Arrays.toString(nodes) + ": datanode " + badNodeIndex
- + "("+ nodes[badNodeIndex] + ") is bad.");
- failed.add(nodes[badNodeIndex]);
-
- DatanodeInfo[] newnodes = new DatanodeInfo[nodes.length-1];
- arraycopy(nodes, newnodes, badNodeIndex);
-
- final StorageType[] newStorageTypes = new StorageType[newnodes.length];
- arraycopy(storageTypes, newStorageTypes, badNodeIndex);
-
- final String[] newStorageIDs = new String[newnodes.length];
- arraycopy(storageIDs, newStorageIDs, badNodeIndex);
-
- setPipeline(newnodes, newStorageTypes, newStorageIDs);
-
- errorState.adjustState4RestartingNode();
- lastException.clear();
- }
- return true;
- }
-
- /** Add a datanode if replace-datanode policy is satisfied. */
- private void handleDatanodeReplacement() throws IOException {
- if (dfsClient.dtpReplaceDatanodeOnFailure.satisfy(stat.getReplication(),
- nodes, isAppend, isHflushed)) {
- try {
- addDatanode2ExistingPipeline();
- } catch(IOException ioe) {
- if (!dfsClient.dtpReplaceDatanodeOnFailure.isBestEffort()) {
- throw ioe;
- }
- LOG.warn("Failed to replace datanode."
- + " Continue with the remaining datanodes since "
- + BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY
- + " is set to true.", ioe);
- }
- }
- }
-
- private void failPacket4Testing() {
- if (failPacket) { // for testing
- failPacket = false;
- try {
- // Give DNs time to send in bad reports. In real situations,
- // good reports should follow bad ones, if client committed
- // with those nodes.
- Thread.sleep(2000);
- } catch (InterruptedException ie) {}
- }
- }
-
- LocatedBlock updateBlockForPipeline() throws IOException {
- return dfsClient.namenode.updateBlockForPipeline(
- block, dfsClient.clientName);
- }
-
- /** update pipeline at the namenode */
- ExtendedBlock updatePipeline(long newGS) throws IOException {
- final ExtendedBlock newBlock = new ExtendedBlock(
- block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS);
- dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock,
- nodes, storageIDs);
- return newBlock;
- }
-
- /**
- * Open a DataStreamer to a DataNode so that it can be written to.
- * This happens when a file is created and each time a new block is allocated.
- * Must get block ID and the IDs of the destinations from the namenode.
- * Returns the list of target datanodes.
- */
- private LocatedBlock nextBlockOutputStream() throws IOException {
- LocatedBlock lb = null;
- DatanodeInfo[] nodes = null;
- StorageType[] storageTypes = null;
- int count = dfsClient.getConf().getNumBlockWriteRetry();
- boolean success = false;
- ExtendedBlock oldBlock = block;
- do {
- errorState.reset();
- lastException.clear();
- success = false;
-
- DatanodeInfo[] excluded =
- excludedNodes.getAllPresent(excludedNodes.asMap().keySet())
- .keySet()
- .toArray(new DatanodeInfo[0]);
- block = oldBlock;
- lb = locateFollowingBlock(excluded.length > 0 ? excluded : null);
- block = lb.getBlock();
- block.setNumBytes(0);
- bytesSent = 0;
- accessToken = lb.getBlockToken();
- nodes = lb.getLocations();
- storageTypes = lb.getStorageTypes();
-
- //
- // Connect to first DataNode in the list.
- //
- success = createBlockOutputStream(nodes, storageTypes, 0L, false);
-
- if (!success) {
- LOG.info("Abandoning " + block);
- dfsClient.namenode.abandonBlock(block, stat.getFileId(), src,
- dfsClient.clientName);
- block = null;
- final DatanodeInfo badNode = nodes[errorState.getBadNodeIndex()];
- LOG.info("Excluding datanode " + badNode);
- excludedNodes.put(badNode, badNode);
- }
- } while (!success && --count >= 0);
-
- if (!success) {
- throw new IOException("Unable to create new block.");
- }
- return lb;
- }
-
- // connects to the first datanode in the pipeline
- // Returns true if success, otherwise return failure.
- //
- private boolean createBlockOutputStream(DatanodeInfo[] nodes,
- StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) {
- if (nodes.length == 0) {
- LOG.info("nodes are empty for write pipeline of " + block);
- return false;
- }
- Status pipelineStatus = SUCCESS;
- String firstBadLink = "";
- boolean checkRestart = false;
- if (LOG.isDebugEnabled()) {
- LOG.debug("pipeline = " + Arrays.asList(nodes));
- }
-
- // persist blocks on namenode on next flush
- persistBlocks.set(true);
-
- int refetchEncryptionKey = 1;
- while (true) {
- boolean result = false;
- DataOutputStream out = null;
- try {
- assert null == s : "Previous socket unclosed";
- assert null == blockReplyStream : "Previous blockReplyStream unclosed";
- s = createSocketForPipeline(nodes[0], nodes.length, dfsClient);
- long writeTimeout = dfsClient.getDatanodeWriteTimeout(nodes.length);
- long readTimeout = dfsClient.getDatanodeReadTimeout(nodes.length);
-
- OutputStream unbufOut = NetUtils.getOutputStream(s, writeTimeout);
- InputStream unbufIn = NetUtils.getInputStream(s, readTimeout);
- IOStreamPair saslStreams = dfsClient.saslClient.socketSend(s,
- unbufOut, unbufIn, dfsClient, accessToken, nodes[0]);
- unbufOut = saslStreams.out;
- unbufIn = saslStreams.in;
- out = new DataOutputStream(new BufferedOutputStream(unbufOut,
- DFSUtil.getSmallBufferSize(dfsClient.getConfiguration())));
- blockReplyStream = new DataInputStream(unbufIn);
-
- //
- // Xmit header info to datanode
- //
-
- BlockConstructionStage bcs = recoveryFlag? stage.getRecoveryStage(): stage;
-
- // We cannot change the block length in 'block' as it counts the number
- // of bytes ack'ed.
- ExtendedBlock blockCopy = new ExtendedBlock(block);
- blockCopy.setNumBytes(stat.getBlockSize());
-
- boolean[] targetPinnings = getPinnings(nodes, true);
- // send the request
- new Sender(out).writeBlock(blockCopy, nodeStorageTypes[0], accessToken,
- dfsClient.clientName, nodes, nodeStorageTypes, null, bcs,
- nodes.length, block.getNumBytes(), bytesSent, newGS,
- checksum4WriteBlock, cachingStrategy.get(), isLazyPersistFile,
- (targetPinnings == null ? false : targetPinnings[0]), targetPinnings);
-
- // receive ack for connect
- BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
- PBHelperClient.vintPrefixed(blockReplyStream));
- pipelineStatus = resp.getStatus();
- firstBadLink = resp.getFirstBadLink();
-
- // Got an restart OOB ack.
- // If a node is already restarting, this status is not likely from
- // the same node. If it is from a different node, it is not
- // from the local datanode. Thus it is safe to treat this as a
- // regular node error.
- if (PipelineAck.isRestartOOBStatus(pipelineStatus) &&
- !errorState.isRestartingNode()) {
- checkRestart = true;
- throw new IOException("A datanode is restarting.");
- }
-
- String logInfo = "ack with firstBadLink as " + firstBadLink;
- DataTransferProtoUtil.checkBlockOpStatus(resp, logInfo);
-
- assert null == blockStream : "Previous blockStream unclosed";
- blockStream = out;
- result = true; // success
- errorState.reset();
- } catch (IOException ie) {
- if (!errorState.isRestartingNode()) {
- LOG.info("Exception in createBlockOutputStream", ie);
- }
- if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
- LOG.info("Will fetch a new encryption key and retry, "
- + "encryption key was invalid when connecting to "
- + nodes[0] + " : " + ie);
- // The encryption key used is invalid.
- refetchEncryptionKey--;
- dfsClient.clearDataEncryptionKey();
- // Don't close the socket/exclude this node just yet. Try again with
- // a new encryption key.
- continue;
- }
-
- // find the datanode that matches
- if (firstBadLink.length() != 0) {
- for (int i = 0; i < nodes.length; i++) {
- // NB: Unconditionally using the xfer addr w/o hostname
- if (firstBadLink.equals(nodes[i].getXferAddr())) {
- errorState.setBadNodeIndex(i);
- break;
- }
- }
- } else {
- assert checkRestart == false;
- errorState.setBadNodeIndex(0);
- }
-
- final int i = errorState.getBadNodeIndex();
- // Check whether there is a restart worth waiting for.
- if (checkRestart && shouldWaitForRestart(i)) {
- errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]);
- }
- errorState.setError(true);
- lastException.set(ie);
- result = false; // error
- } finally {
- if (!result) {
- IOUtils.closeSocket(s);
- s = null;
- IOUtils.closeStream(out);
- out = null;
- IOUtils.closeStream(blockReplyStream);
- blockReplyStream = null;
- }
- }
- return result;
- }
- }
-
- private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) {
- if (favoredNodes == null) {
- return null;
- } else {
- boolean[] pinnings = new boolean[nodes.length];
- HashSet<String> favoredSet =
- new HashSet<String>(Arrays.asList(favoredNodes));
- for (int i = 0; i < nodes.length; i++) {
- pinnings[i] = favoredSet.remove(nodes[i].getXferAddrWithHostname());
- if (LOG.isDebugEnabled()) {
- LOG.debug(nodes[i].getXferAddrWithHostname() +
- " was chosen by name node (favored=" + pinnings[i] + ").");
- }
- }
- if (shouldLog && !favoredSet.isEmpty()) {
- // There is one or more favored nodes that were not allocated.
- LOG.warn("These favored nodes were specified but not chosen: "
- + favoredSet + " Specified favored nodes: "
- + Arrays.toString(favoredNodes));
-
- }
- return pinnings;
- }
- }
-
- protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
- throws IOException {
- final DfsClientConf conf = dfsClient.getConf();
- int retries = conf.getNumBlockWriteLocateFollowingRetry();
- long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
- while (true) {
- long localstart = Time.monotonicNow();
- while (true) {
- try {
- return dfsClient.namenode.addBlock(src, dfsClient.clientName,
- block, excludedNodes, stat.getFileId(), favoredNodes);
- } catch (RemoteException e) {
- IOException ue =
- e.unwrapRemoteException(FileNotFoundException.class,
- AccessControlException.class,
- NSQuotaExceededException.class,
- DSQuotaExceededException.class,
- QuotaByStorageTypeExceededException.class,
- UnresolvedPathException.class);
- if (ue != e) {
- throw ue; // no need to retry these exceptions
- }
-
-
- if (NotReplicatedYetException.class.getName().
- equals(e.getClassName())) {
- if (retries == 0) {
- throw e;
- } else {
- --retries;
- LOG.info("Exception while adding a block", e);
- long elapsed = Time.monotonicNow() - localstart;
- if (elapsed > 5000) {
- LOG.info("Waiting for replication for "
- + (elapsed / 1000) + " seconds");
- }
- try {
- LOG.warn("NotReplicatedYetException sleeping " + src
- + " retries left " + retries);
- Thread.sleep(sleeptime);
- sleeptime *= 2;
- } catch (InterruptedException ie) {
- LOG.warn("Caught exception", ie);
- }
- }
- } else {
- throw e;
- }
-
- }
- }
- }
- }
-
- /**
- * This function sleeps for a certain amount of time when the writing
- * pipeline is congested. The function calculates the time based on a
- * decorrelated filter.
- *
- * @see
- * <a href="http://www.awsarchitectureblog.com/2015/03/backoff.html">
- * http://www.awsarchitectureblog.com/2015/03/backoff.html</a>
- */
- private void backOffIfNecessary() throws InterruptedException {
- int t = 0;
- synchronized (congestedNodes) {
- if (!congestedNodes.isEmpty()) {
- StringBuilder sb = new StringBuilder("DataNode");
- for (DatanodeInfo i : congestedNodes) {
- sb.append(' ').append(i);
- }
- int range = Math.abs(lastCongestionBackoffTime * 3 -
- CONGESTION_BACKOFF_MEAN_TIME_IN_MS);
- int base = Math.min(lastCongestionBackoffTime * 3,
- CONGESTION_BACKOFF_MEAN_TIME_IN_MS);
- t = Math.min(CONGESTION_BACK_OFF_MAX_TIME_IN_MS,
- (int)(base + Math.random() * range));
- lastCongestionBackoffTime = t;
- sb.append(" are congested. Backing off for ").append(t).append(" ms");
- LOG.info(sb.toString());
- congestedNodes.clear();
- }
- }
- if (t != 0) {
- Thread.sleep(t);
- }
- }
-
- /**
- * get the block this streamer is writing to
- *
- * @return the block this streamer is writing to
- */
- ExtendedBlock getBlock() {
- return block;
- }
-
- /**
- * return the target datanodes in the pipeline
- *
- * @return the target datanodes in the pipeline
- */
- DatanodeInfo[] getNodes() {
- return nodes;
- }
-
- /**
- * return the token of the block
- *
- * @return the token of the block
- */
- Token<BlockTokenIdentifier> getBlockToken() {
- return accessToken;
- }
-
- /**
- * Put a packet to the data queue
- *
- * @param packet the packet to be put into the data queued
- */
- void queuePacket(DFSPacket packet) {
- synchronized (dataQueue) {
- if (packet == null) return;
- packet.addTraceParent(Trace.currentSpan());
- dataQueue.addLast(packet);
- lastQueuedSeqno = packet.getSeqno();
- if (LOG.isDebugEnabled()) {
- LOG.debug("Queued packet " + packet.getSeqno());
- }
- dataQueue.notifyAll();
- }
- }
-
- /**
- * For heartbeat packets, create buffer directly by new byte[]
- * since heartbeats should not be blocked.
- */
- private DFSPacket createHeartbeatPacket() throws InterruptedIOException {
- final byte[] buf = new byte[PacketHeader.PKT_MAX_HEADER_LEN];
- return new DFSPacket(buf, 0, 0, DFSPacket.HEART_BEAT_SEQNO, 0, false);
- }
-
- private static LoadingCache<DatanodeInfo, DatanodeInfo> initExcludedNodes(
- long excludedNodesCacheExpiry) {
- return CacheBuilder.newBuilder()
- .expireAfterWrite(excludedNodesCacheExpiry, TimeUnit.MILLISECONDS)
- .removalListener(new RemovalListener<DatanodeInfo, DatanodeInfo>() {
- @Override
- public void onRemoval(
- RemovalNotification<DatanodeInfo, DatanodeInfo> notification) {
- LOG.info("Removing node " + notification.getKey()
- + " from the excluded nodes list");
- }
- }).build(new CacheLoader<DatanodeInfo, DatanodeInfo>() {
- @Override
- public DatanodeInfo load(DatanodeInfo key) throws Exception {
- return key;
- }
- });
- }
-
- private static <T> void arraycopy(T[] srcs, T[] dsts, int skipIndex) {
- System.arraycopy(srcs, 0, dsts, 0, skipIndex);
- System.arraycopy(srcs, skipIndex+1, dsts, skipIndex, dsts.length-skipIndex);
- }
-
- /**
- * check if to persist blocks on namenode
- *
- * @return if to persist blocks on namenode
- */
- AtomicBoolean getPersistBlocks(){
- return persistBlocks;
- }
-
- /**
- * check if to append a chunk
- *
- * @param appendChunk if to append a chunk
- */
- void setAppendChunk(boolean appendChunk){
- this.appendChunk = appendChunk;
- }
-
- /**
- * get if to append a chunk
- *
- * @return if to append a chunk
- */
- boolean getAppendChunk(){
- return appendChunk;
- }
-
- /**
- * @return the last exception
- */
- LastExceptionInStreamer getLastException(){
- return lastException;
- }
-
- /**
- * set socket to null
- */
- void setSocketToNull() {
- this.s = null;
- }
-
- /**
- * return current sequence number and then increase it by 1
- *
- * @return current sequence number before increasing
- */
- long getAndIncCurrentSeqno() {
- long old = this.currentSeqno;
- this.currentSeqno++;
- return old;
- }
-
- /**
- * get last queued sequence number
- *
- * @return last queued sequence number
- */
- long getLastQueuedSeqno() {
- return lastQueuedSeqno;
- }
-
- /**
- * get the number of bytes of current block
- *
- * @return the number of bytes of current block
- */
- long getBytesCurBlock() {
- return bytesCurBlock;
- }
-
- /**
- * set the bytes of current block that have been written
- *
- * @param bytesCurBlock bytes of current block that have been written
- */
- void setBytesCurBlock(long bytesCurBlock) {
- this.bytesCurBlock = bytesCurBlock;
- }
-
- /**
- * increase bytes of current block by len.
- *
- * @param len how many bytes to increase to current block
- */
- void incBytesCurBlock(long len) {
- this.bytesCurBlock += len;
- }
-
- /**
- * set artificial slow down for unit test
- *
- * @param period artificial slow down
- */
- void setArtificialSlowdown(long period) {
- this.artificialSlowdown = period;
- }
-
- /**
- * if this streamer is to terminate
- *
- * @return if this streamer is to terminate
- */
- boolean streamerClosed(){
- return streamerClosed;
- }
-
- void closeSocket() throws IOException {
- if (s != null) {
- s.close();
- }
- }
-
- @Override
- public String toString() {
- return (block == null? null: block.getLocalBlock())
- + "@" + Arrays.toString(getNodes());
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
index ef9f27a..b6bf6cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/HdfsConfiguration.java
@@ -31,9 +31,7 @@ public class HdfsConfiguration extends Configuration {
addDeprecatedKeys();
// adds the default resources
- Configuration.addDefaultResource("hdfs-default.xml");
- Configuration.addDefaultResource("hdfs-site.xml");
-
+ HdfsConfigurationLoader.init();
}
public HdfsConfiguration() {
@@ -52,9 +50,10 @@ public class HdfsConfiguration extends Configuration {
* This method is here so that when invoked, HdfsConfiguration is class-loaded if
* it hasn't already been previously loaded. Upon loading the class, the static
* initializer block above will be executed to add the deprecated keys and to add
- * the default resources. It is safe for this method to be called multiple times
- * as the static initializer block will only get invoked once.
- *
+ * the default resources via {@link HdfsConfigurationLoader#init()}. It is
+ * safe for this method to be called multiple times as the static initializer
+ * block will only get invoked once.
+ *
* This replaces the previously, dangerous practice of other classes calling
* Configuration.addDefaultResource("hdfs-default.xml") directly without loading
* HdfsConfiguration class first, thereby skipping the key deprecation
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
deleted file mode 100644
index f03e179..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/RemotePeerFactory.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import java.io.IOException;
-import java.net.InetSocketAddress;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.security.token.Token;
-
-@InterfaceAudience.Private
-public interface RemotePeerFactory {
- /**
- * @param addr The address to connect to.
- * @param blockToken Token used during optional SASL negotiation
- * @param datanodeId ID of destination DataNode
- * @return A new Peer connected to the address.
- *
- * @throws IOException If there was an error connecting or creating
- * the remote socket, encrypted stream, etc.
- */
- Peer newConnectedPeer(InetSocketAddress addr,
- Token<BlockTokenIdentifier> blockToken, DatanodeID datanodeId)
- throws IOException;
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java
deleted file mode 100644
index ec17bb8..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCipherSuiteException.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Thrown when an unknown cipher suite is encountered.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Stable
-public class UnknownCipherSuiteException extends IOException {
- public UnknownCipherSuiteException(String msg) {
- super(msg);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java
deleted file mode 100644
index 0aac8c8..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/UnknownCryptoProtocolVersionException.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class UnknownCryptoProtocolVersionException extends IOException {
- private static final long serialVersionUID = 8957192l;
-
- public UnknownCryptoProtocolVersionException() {
- super();
- }
-
- public UnknownCryptoProtocolVersionException(String msg) {
- super(msg);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
deleted file mode 100644
index 2655c40..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/XAttrHelper.java
+++ /dev/null
@@ -1,174 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.XAttr;
-import org.apache.hadoop.fs.XAttr.NameSpace;
-import org.apache.hadoop.util.StringUtils;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-
-@InterfaceAudience.Private
-public class XAttrHelper {
-
- /**
- * Build <code>XAttr</code> from xattr name with prefix.
- */
- public static XAttr buildXAttr(String name) {
- return buildXAttr(name, null);
- }
-
- /**
- * Build <code>XAttr</code> from name with prefix and value.
- * Name can not be null. Value can be null. The name and prefix
- * are validated.
- * Both name and namespace are case sensitive.
- */
- public static XAttr buildXAttr(String name, byte[] value) {
- Preconditions.checkNotNull(name, "XAttr name cannot be null.");
-
- final int prefixIndex = name.indexOf(".");
- if (prefixIndex < 3) {// Prefix length is at least 3.
- throw new HadoopIllegalArgumentException("An XAttr name must be " +
- "prefixed with user/trusted/security/system/raw, followed by a '.'");
- } else if (prefixIndex == name.length() - 1) {
- throw new HadoopIllegalArgumentException("XAttr name cannot be empty.");
- }
-
- NameSpace ns;
- final String prefix = name.substring(0, prefixIndex);
- if (StringUtils.equalsIgnoreCase(prefix, NameSpace.USER.toString())) {
- ns = NameSpace.USER;
- } else if (
- StringUtils.equalsIgnoreCase(prefix, NameSpace.TRUSTED.toString())) {
- ns = NameSpace.TRUSTED;
- } else if (
- StringUtils.equalsIgnoreCase(prefix, NameSpace.SYSTEM.toString())) {
- ns = NameSpace.SYSTEM;
- } else if (
- StringUtils.equalsIgnoreCase(prefix, NameSpace.SECURITY.toString())) {
- ns = NameSpace.SECURITY;
- } else if (
- StringUtils.equalsIgnoreCase(prefix, NameSpace.RAW.toString())) {
- ns = NameSpace.RAW;
- } else {
- throw new HadoopIllegalArgumentException("An XAttr name must be " +
- "prefixed with user/trusted/security/system/raw, followed by a '.'");
- }
- XAttr xAttr = (new XAttr.Builder()).setNameSpace(ns).setName(name.
- substring(prefixIndex + 1)).setValue(value).build();
-
- return xAttr;
- }
-
- /**
- * Build xattr name with prefix as <code>XAttr</code> list.
- */
- public static List<XAttr> buildXAttrAsList(String name) {
- XAttr xAttr = buildXAttr(name);
- List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
- xAttrs.add(xAttr);
-
- return xAttrs;
- }
-
- /**
- * Get value of first xattr from <code>XAttr</code> list
- */
- public static byte[] getFirstXAttrValue(List<XAttr> xAttrs) {
- byte[] value = null;
- XAttr xAttr = getFirstXAttr(xAttrs);
- if (xAttr != null) {
- value = xAttr.getValue();
- if (value == null) {
- value = new byte[0]; // xattr exists, but no value.
- }
- }
- return value;
- }
-
- /**
- * Get first xattr from <code>XAttr</code> list
- */
- public static XAttr getFirstXAttr(List<XAttr> xAttrs) {
- if (xAttrs != null && !xAttrs.isEmpty()) {
- return xAttrs.get(0);
- }
-
- return null;
- }
-
- /**
- * Build xattr map from <code>XAttr</code> list, the key is
- * xattr name with prefix, and value is xattr value.
- */
- public static Map<String, byte[]> buildXAttrMap(List<XAttr> xAttrs) {
- if (xAttrs == null) {
- return null;
- }
- Map<String, byte[]> xAttrMap = Maps.newHashMap();
- for (XAttr xAttr : xAttrs) {
- String name = getPrefixedName(xAttr);
- byte[] value = xAttr.getValue();
- if (value == null) {
- value = new byte[0];
- }
- xAttrMap.put(name, value);
- }
-
- return xAttrMap;
- }
-
- /**
- * Get name with prefix from <code>XAttr</code>
- */
- public static String getPrefixedName(XAttr xAttr) {
- if (xAttr == null) {
- return null;
- }
-
- return getPrefixedName(xAttr.getNameSpace(), xAttr.getName());
- }
-
- public static String getPrefixedName(XAttr.NameSpace ns, String name) {
- return StringUtils.toLowerCase(ns.toString()) + "." + name;
- }
-
- /**
- * Build <code>XAttr</code> list from xattr name list.
- */
- public static List<XAttr> buildXAttrs(List<String> names) {
- if (names == null || names.isEmpty()) {
- throw new HadoopIllegalArgumentException("XAttr names can not be " +
- "null or empty.");
- }
-
- List<XAttr> xAttrs = Lists.newArrayListWithCapacity(names.size());
- for (String name : names) {
- xAttrs.add(buildXAttr(name, null));
- }
- return xAttrs;
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
deleted file mode 100644
index e8ac686..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataInputStream.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs.client;
-
-import java.io.InputStream;
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.crypto.CryptoInputStream;
-import org.apache.hadoop.hdfs.DFSInputStream;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-
-import com.google.common.base.Preconditions;
-
-/**
- * The Hdfs implementation of {@link FSDataInputStream}.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class HdfsDataInputStream extends FSDataInputStream {
- public HdfsDataInputStream(DFSInputStream in) throws IOException {
- super(in);
- }
-
- public HdfsDataInputStream(CryptoInputStream in) throws IOException {
- super(in);
- Preconditions.checkArgument(in.getWrappedStream() instanceof DFSInputStream,
- "CryptoInputStream should wrap a DFSInputStream");
- }
-
- private DFSInputStream getDFSInputStream() {
- if (in instanceof CryptoInputStream) {
- return (DFSInputStream) ((CryptoInputStream) in).getWrappedStream();
- }
- return (DFSInputStream) in;
- }
-
- /**
- * Get a reference to the wrapped output stream. We always want to return the
- * actual underlying InputStream, even when we're using a CryptoStream. e.g.
- * in the delegated methods below.
- *
- * @return the underlying output stream
- */
- public InputStream getWrappedStream() {
- return in;
- }
-
- /**
- * Get the datanode from which the stream is currently reading.
- */
- public DatanodeInfo getCurrentDatanode() {
- return getDFSInputStream().getCurrentDatanode();
- }
-
- /**
- * Get the block containing the target position.
- */
- public ExtendedBlock getCurrentBlock() {
- return getDFSInputStream().getCurrentBlock();
- }
-
- /**
- * Get the collection of blocks that has already been located.
- */
- public List<LocatedBlock> getAllBlocks() throws IOException {
- return getDFSInputStream().getAllBlocks();
- }
-
- /**
- * Get the visible length of the file. It will include the length of the last
- * block even if that is in UnderConstruction state.
- *
- * @return The visible length of the file.
- */
- public long getVisibleLength() throws IOException {
- return getDFSInputStream().getFileLength();
- }
-
- /**
- * Get statistics about the reads which this DFSInputStream has done.
- * Note that because HdfsDataInputStream is buffered, these stats may
- * be higher than you would expect just by adding up the number of
- * bytes read through HdfsDataInputStream.
- */
- public DFSInputStream.ReadStatistics getReadStatistics() {
- return getDFSInputStream().getReadStatistics();
- }
-
- public void clearReadStatistics() {
- getDFSInputStream().clearReadStatistics();
- }
-}
[03/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and
related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
deleted file mode 100755
index 5466936..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ /dev/null
@@ -1,923 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InterruptedIOException;
-import java.nio.channels.ClosedChannelException;
-import java.util.EnumSet;
-import java.util.concurrent.atomic.AtomicReference;
-
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.crypto.CryptoProtocolVersion;
-import org.apache.hadoop.fs.CanSetDropBehind;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FSOutputSummer;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.Syncable;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
-import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
-import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
-import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
-import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.namenode.RetryStartFileException;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.hdfs.util.ByteArrayManager;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.DataChecksum.Type;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.Time;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-
-/****************************************************************
- * DFSOutputStream creates files from a stream of bytes.
- *
- * The client application writes data that is cached internally by
- * this stream. Data is broken up into packets, each packet is
- * typically 64K in size. A packet comprises of chunks. Each chunk
- * is typically 512 bytes and has an associated checksum with it.
- *
- * When a client application fills up the currentPacket, it is
- * enqueued into the dataQueue of DataStreamer. DataStreamer is a
- * thread that picks up packets from the dataQueue and sends it to
- * the first datanode in the pipeline.
- *
- ****************************************************************/
-@InterfaceAudience.Private
-public class DFSOutputStream extends FSOutputSummer
- implements Syncable, CanSetDropBehind {
- static final Logger LOG = LoggerFactory.getLogger(DFSOutputStream.class);
- /**
- * Number of times to retry creating a file when there are transient
- * errors (typically related to encryption zones and KeyProvider operations).
- */
- @VisibleForTesting
- static final int CREATE_RETRY_COUNT = 10;
- @VisibleForTesting
- static CryptoProtocolVersion[] SUPPORTED_CRYPTO_VERSIONS =
- CryptoProtocolVersion.supported();
-
- protected final DFSClient dfsClient;
- protected final ByteArrayManager byteArrayManager;
- // closed is accessed by different threads under different locks.
- protected volatile boolean closed = false;
-
- protected final String src;
- protected final long fileId;
- protected final long blockSize;
- protected final int bytesPerChecksum;
-
- protected DFSPacket currentPacket = null;
- private DataStreamer streamer;
- protected int packetSize = 0; // write packet size, not including the header.
- protected int chunksPerPacket = 0;
- protected long lastFlushOffset = 0; // offset when flush was invoked
- private long initialFileSize = 0; // at time of file open
- private final short blockReplication; // replication factor of file
- protected boolean shouldSyncBlock = false; // force blocks to disk upon close
- protected final AtomicReference<CachingStrategy> cachingStrategy;
- private FileEncryptionInfo fileEncryptionInfo;
-
- /** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
- protected DFSPacket createPacket(int packetSize, int chunksPerPkt, long offsetInBlock,
- long seqno, boolean lastPacketInBlock) throws InterruptedIOException {
- final byte[] buf;
- final int bufferSize = PacketHeader.PKT_MAX_HEADER_LEN + packetSize;
-
- try {
- buf = byteArrayManager.newByteArray(bufferSize);
- } catch (InterruptedException ie) {
- final InterruptedIOException iioe = new InterruptedIOException(
- "seqno=" + seqno);
- iioe.initCause(ie);
- throw iioe;
- }
-
- return new DFSPacket(buf, chunksPerPkt, offsetInBlock, seqno,
- getChecksumSize(), lastPacketInBlock);
- }
-
- @Override
- protected void checkClosed() throws IOException {
- if (isClosed()) {
- getStreamer().getLastException().throwException4Close();
- }
- }
-
- //
- // returns the list of targets, if any, that is being currently used.
- //
- @VisibleForTesting
- public synchronized DatanodeInfo[] getPipeline() {
- if (getStreamer().streamerClosed()) {
- return null;
- }
- DatanodeInfo[] currentNodes = getStreamer().getNodes();
- if (currentNodes == null) {
- return null;
- }
- DatanodeInfo[] value = new DatanodeInfo[currentNodes.length];
- for (int i = 0; i < currentNodes.length; i++) {
- value[i] = currentNodes[i];
- }
- return value;
- }
-
- /**
- * @return the object for computing checksum.
- * The type is NULL if checksum is not computed.
- */
- private static DataChecksum getChecksum4Compute(DataChecksum checksum,
- HdfsFileStatus stat) {
- if (DataStreamer.isLazyPersist(stat) && stat.getReplication() == 1) {
- // do not compute checksum for writing to single replica to memory
- return DataChecksum.newDataChecksum(Type.NULL,
- checksum.getBytesPerChecksum());
- }
- return checksum;
- }
-
- private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress,
- HdfsFileStatus stat, DataChecksum checksum) throws IOException {
- super(getChecksum4Compute(checksum, stat));
- this.dfsClient = dfsClient;
- this.src = src;
- this.fileId = stat.getFileId();
- this.blockSize = stat.getBlockSize();
- this.blockReplication = stat.getReplication();
- this.fileEncryptionInfo = stat.getFileEncryptionInfo();
- this.cachingStrategy = new AtomicReference<CachingStrategy>(
- dfsClient.getDefaultWriteCachingStrategy());
- if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug(
- "Set non-null progress callback on DFSOutputStream " + src);
- }
-
- this.bytesPerChecksum = checksum.getBytesPerChecksum();
- if (bytesPerChecksum <= 0) {
- throw new HadoopIllegalArgumentException(
- "Invalid value: bytesPerChecksum = " + bytesPerChecksum + " <= 0");
- }
- if (blockSize % bytesPerChecksum != 0) {
- throw new HadoopIllegalArgumentException("Invalid values: "
- + HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
- + ") must divide block size (=" + blockSize + ").");
- }
- this.byteArrayManager = dfsClient.getClientContext().getByteArrayManager();
- }
-
- /** Construct a new output stream for creating a file. */
- protected DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
- EnumSet<CreateFlag> flag, Progressable progress,
- DataChecksum checksum, String[] favoredNodes) throws IOException {
- this(dfsClient, src, progress, stat, checksum);
- this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
-
- computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
-
- streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
- cachingStrategy, byteArrayManager, favoredNodes);
- }
-
- static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
- FsPermission masked, EnumSet<CreateFlag> flag, boolean createParent,
- short replication, long blockSize, Progressable progress, int buffersize,
- DataChecksum checksum, String[] favoredNodes) throws IOException {
- TraceScope scope =
- dfsClient.getPathTraceScope("newStreamForCreate", src);
- try {
- HdfsFileStatus stat = null;
-
- // Retry the create if we get a RetryStartFileException up to a maximum
- // number of times
- boolean shouldRetry = true;
- int retryCount = CREATE_RETRY_COUNT;
- while (shouldRetry) {
- shouldRetry = false;
- try {
- stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
- new EnumSetWritable<CreateFlag>(flag), createParent, replication,
- blockSize, SUPPORTED_CRYPTO_VERSIONS);
- break;
- } catch (RemoteException re) {
- IOException e = re.unwrapRemoteException(
- AccessControlException.class,
- DSQuotaExceededException.class,
- QuotaByStorageTypeExceededException.class,
- FileAlreadyExistsException.class,
- FileNotFoundException.class,
- ParentNotDirectoryException.class,
- NSQuotaExceededException.class,
- RetryStartFileException.class,
- SafeModeException.class,
- UnresolvedPathException.class,
- SnapshotAccessControlException.class,
- UnknownCryptoProtocolVersionException.class);
- if (e instanceof RetryStartFileException) {
- if (retryCount > 0) {
- shouldRetry = true;
- retryCount--;
- } else {
- throw new IOException("Too many retries because of encryption" +
- " zone operations", e);
- }
- } else {
- throw e;
- }
- }
- }
- Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!");
- final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
- flag, progress, checksum, favoredNodes);
- out.start();
- return out;
- } finally {
- scope.close();
- }
- }
-
- /** Construct a new output stream for append. */
- private DFSOutputStream(DFSClient dfsClient, String src,
- EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
- HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
- throws IOException {
- this(dfsClient, src, progress, stat, checksum);
- initialFileSize = stat.getLen(); // length of file when opened
- this.shouldSyncBlock = flags.contains(CreateFlag.SYNC_BLOCK);
-
- boolean toNewBlock = flags.contains(CreateFlag.NEW_BLOCK);
-
- this.fileEncryptionInfo = stat.getFileEncryptionInfo();
-
- // The last partial block of the file has to be filled.
- if (!toNewBlock && lastBlock != null) {
- // indicate that we are appending to an existing block
- streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress, checksum,
- cachingStrategy, byteArrayManager);
- getStreamer().setBytesCurBlock(lastBlock.getBlockSize());
- adjustPacketChunkSize(stat);
- getStreamer().setPipelineInConstruction(lastBlock);
- } else {
- computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
- bytesPerChecksum);
- streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
- dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
- favoredNodes);
- }
- }
-
- private void adjustPacketChunkSize(HdfsFileStatus stat) throws IOException{
-
- long usedInLastBlock = stat.getLen() % blockSize;
- int freeInLastBlock = (int)(blockSize - usedInLastBlock);
-
- // calculate the amount of free space in the pre-existing
- // last crc chunk
- int usedInCksum = (int)(stat.getLen() % bytesPerChecksum);
- int freeInCksum = bytesPerChecksum - usedInCksum;
-
- // if there is space in the last block, then we have to
- // append to that block
- if (freeInLastBlock == blockSize) {
- throw new IOException("The last block for file " +
- src + " is full.");
- }
-
- if (usedInCksum > 0 && freeInCksum > 0) {
- // if there is space in the last partial chunk, then
- // setup in such a way that the next packet will have only
- // one chunk that fills up the partial chunk.
- //
- computePacketChunkSize(0, freeInCksum);
- setChecksumBufSize(freeInCksum);
- getStreamer().setAppendChunk(true);
- } else {
- // if the remaining space in the block is smaller than
- // that expected size of of a packet, then create
- // smaller size packet.
- //
- computePacketChunkSize(
- Math.min(dfsClient.getConf().getWritePacketSize(), freeInLastBlock),
- bytesPerChecksum);
- }
- }
-
- static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
- EnumSet<CreateFlag> flags, int bufferSize, Progressable progress,
- LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum,
- String[] favoredNodes) throws IOException {
- TraceScope scope =
- dfsClient.getPathTraceScope("newStreamForAppend", src);
- try {
- final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
- progress, lastBlock, stat, checksum, favoredNodes);
- out.start();
- return out;
- } finally {
- scope.close();
- }
- }
-
- protected void computePacketChunkSize(int psize, int csize) {
- final int bodySize = psize - PacketHeader.PKT_MAX_HEADER_LEN;
- final int chunkSize = csize + getChecksumSize();
- chunksPerPacket = Math.max(bodySize/chunkSize, 1);
- packetSize = chunkSize*chunksPerPacket;
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("computePacketChunkSize: src=" + src +
- ", chunkSize=" + chunkSize +
- ", chunksPerPacket=" + chunksPerPacket +
- ", packetSize=" + packetSize);
- }
- }
-
- protected TraceScope createWriteTraceScope() {
- return dfsClient.getPathTraceScope("DFSOutputStream#write", src);
- }
-
- // @see FSOutputSummer#writeChunk()
- @Override
- protected synchronized void writeChunk(byte[] b, int offset, int len,
- byte[] checksum, int ckoff, int cklen) throws IOException {
- dfsClient.checkOpen();
- checkClosed();
-
- if (len > bytesPerChecksum) {
- throw new IOException("writeChunk() buffer size is " + len +
- " is larger than supported bytesPerChecksum " +
- bytesPerChecksum);
- }
- if (cklen != 0 && cklen != getChecksumSize()) {
- throw new IOException("writeChunk() checksum size is supposed to be " +
- getChecksumSize() + " but found to be " + cklen);
- }
-
- if (currentPacket == null) {
- currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer()
- .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false);
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno=" +
- currentPacket.getSeqno() +
- ", src=" + src +
- ", packetSize=" + packetSize +
- ", chunksPerPacket=" + chunksPerPacket +
- ", bytesCurBlock=" + getStreamer().getBytesCurBlock());
- }
- }
-
- currentPacket.writeChecksum(checksum, ckoff, cklen);
- currentPacket.writeData(b, offset, len);
- currentPacket.incNumChunks();
- getStreamer().incBytesCurBlock(len);
-
- // If packet is full, enqueue it for transmission
- //
- if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
- getStreamer().getBytesCurBlock() == blockSize) {
- enqueueCurrentPacketFull();
- }
- }
-
- void enqueueCurrentPacket() throws IOException {
- getStreamer().waitAndQueuePacket(currentPacket);
- currentPacket = null;
- }
-
- void enqueueCurrentPacketFull() throws IOException {
- LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
- + " appendChunk={}, {}", currentPacket, src, getStreamer()
- .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
- getStreamer());
- enqueueCurrentPacket();
- adjustChunkBoundary();
- endBlock();
- }
-
- /** create an empty packet to mark the end of the block. */
- void setCurrentPacketToEmpty() throws InterruptedIOException {
- currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
- getStreamer().getAndIncCurrentSeqno(), true);
- currentPacket.setSyncBlock(shouldSyncBlock);
- }
-
- /**
- * If the reopened file did not end at chunk boundary and the above
- * write filled up its partial chunk. Tell the summer to generate full
- * crc chunks from now on.
- */
- protected void adjustChunkBoundary() {
- if (getStreamer().getAppendChunk() &&
- getStreamer().getBytesCurBlock() % bytesPerChecksum == 0) {
- getStreamer().setAppendChunk(false);
- resetChecksumBufSize();
- }
-
- if (!getStreamer().getAppendChunk()) {
- int psize = Math.min((int)(blockSize- getStreamer().getBytesCurBlock()),
- dfsClient.getConf().getWritePacketSize());
- computePacketChunkSize(psize, bytesPerChecksum);
- }
- }
-
- /**
- * if encountering a block boundary, send an empty packet to
- * indicate the end of block and reset bytesCurBlock.
- *
- * @throws IOException
- */
- protected void endBlock() throws IOException {
- if (getStreamer().getBytesCurBlock() == blockSize) {
- setCurrentPacketToEmpty();
- enqueueCurrentPacket();
- getStreamer().setBytesCurBlock(0);
- lastFlushOffset = 0;
- }
- }
-
- @Deprecated
- public void sync() throws IOException {
- hflush();
- }
-
- /**
- * Flushes out to all replicas of the block. The data is in the buffers
- * of the DNs but not necessarily in the DN's OS buffers.
- *
- * It is a synchronous operation. When it returns,
- * it guarantees that flushed data become visible to new readers.
- * It is not guaranteed that data has been flushed to
- * persistent store on the datanode.
- * Block allocations are persisted on namenode.
- */
- @Override
- public void hflush() throws IOException {
- TraceScope scope =
- dfsClient.getPathTraceScope("hflush", src);
- try {
- flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
- } finally {
- scope.close();
- }
- }
-
- @Override
- public void hsync() throws IOException {
- TraceScope scope =
- dfsClient.getPathTraceScope("hsync", src);
- try {
- flushOrSync(true, EnumSet.noneOf(SyncFlag.class));
- } finally {
- scope.close();
- }
- }
-
- /**
- * The expected semantics is all data have flushed out to all replicas
- * and all replicas have done posix fsync equivalent - ie the OS has
- * flushed it to the disk device (but the disk may have it in its cache).
- *
- * Note that only the current block is flushed to the disk device.
- * To guarantee durable sync across block boundaries the stream should
- * be created with {@link CreateFlag#SYNC_BLOCK}.
- *
- * @param syncFlags
- * Indicate the semantic of the sync. Currently used to specify
- * whether or not to update the block length in NameNode.
- */
- public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
- TraceScope scope =
- dfsClient.getPathTraceScope("hsync", src);
- try {
- flushOrSync(true, syncFlags);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Flush/Sync buffered data to DataNodes.
- *
- * @param isSync
- * Whether or not to require all replicas to flush data to the disk
- * device
- * @param syncFlags
- * Indicate extra detailed semantic of the flush/sync. Currently
- * mainly used to specify whether or not to update the file length in
- * the NameNode
- * @throws IOException
- */
- private void flushOrSync(boolean isSync, EnumSet<SyncFlag> syncFlags)
- throws IOException {
- dfsClient.checkOpen();
- checkClosed();
- try {
- long toWaitFor;
- long lastBlockLength = -1L;
- boolean updateLength = syncFlags.contains(SyncFlag.UPDATE_LENGTH);
- boolean endBlock = syncFlags.contains(SyncFlag.END_BLOCK);
- synchronized (this) {
- // flush checksum buffer, but keep checksum buffer intact if we do not
- // need to end the current block
- int numKept = flushBuffer(!endBlock, true);
- // bytesCurBlock potentially incremented if there was buffered data
-
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("DFSClient flush(): "
- + " bytesCurBlock=" + getStreamer().getBytesCurBlock()
- + " lastFlushOffset=" + lastFlushOffset
- + " createNewBlock=" + endBlock);
- }
- // Flush only if we haven't already flushed till this offset.
- if (lastFlushOffset != getStreamer().getBytesCurBlock()) {
- assert getStreamer().getBytesCurBlock() > lastFlushOffset;
- // record the valid offset of this flush
- lastFlushOffset = getStreamer().getBytesCurBlock();
- if (isSync && currentPacket == null && !endBlock) {
- // Nothing to send right now,
- // but sync was requested.
- // Send an empty packet if we do not end the block right now
- currentPacket = createPacket(packetSize, chunksPerPacket,
- getStreamer().getBytesCurBlock(), getStreamer()
- .getAndIncCurrentSeqno(), false);
- }
- } else {
- if (isSync && getStreamer().getBytesCurBlock() > 0 && !endBlock) {
- // Nothing to send right now,
- // and the block was partially written,
- // and sync was requested.
- // So send an empty sync packet if we do not end the block right
- // now
- currentPacket = createPacket(packetSize, chunksPerPacket,
- getStreamer().getBytesCurBlock(), getStreamer()
- .getAndIncCurrentSeqno(), false);
- } else if (currentPacket != null) {
- // just discard the current packet since it is already been sent.
- currentPacket.releaseBuffer(byteArrayManager);
- currentPacket = null;
- }
- }
- if (currentPacket != null) {
- currentPacket.setSyncBlock(isSync);
- enqueueCurrentPacket();
- }
- if (endBlock && getStreamer().getBytesCurBlock() > 0) {
- // Need to end the current block, thus send an empty packet to
- // indicate this is the end of the block and reset bytesCurBlock
- currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
- getStreamer().getAndIncCurrentSeqno(), true);
- currentPacket.setSyncBlock(shouldSyncBlock || isSync);
- enqueueCurrentPacket();
- getStreamer().setBytesCurBlock(0);
- lastFlushOffset = 0;
- } else {
- // Restore state of stream. Record the last flush offset
- // of the last full chunk that was flushed.
- getStreamer().setBytesCurBlock(
- getStreamer().getBytesCurBlock() - numKept);
- }
-
- toWaitFor = getStreamer().getLastQueuedSeqno();
- } // end synchronized
-
- getStreamer().waitForAckedSeqno(toWaitFor);
-
- // update the block length first time irrespective of flag
- if (updateLength || getStreamer().getPersistBlocks().get()) {
- synchronized (this) {
- if (!getStreamer().streamerClosed()
- && getStreamer().getBlock() != null) {
- lastBlockLength = getStreamer().getBlock().getNumBytes();
- }
- }
- }
- // If 1) any new blocks were allocated since the last flush, or 2) to
- // update length in NN is required, then persist block locations on
- // namenode.
- if (getStreamer().getPersistBlocks().getAndSet(false) || updateLength) {
- try {
- dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
- lastBlockLength);
- } catch (IOException ioe) {
- DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
- // If we got an error here, it might be because some other thread called
- // close before our hflush completed. In that case, we should throw an
- // exception that the stream is closed.
- checkClosed();
- // If we aren't closed but failed to sync, we should expose that to the
- // caller.
- throw ioe;
- }
- }
-
- synchronized(this) {
- if (!getStreamer().streamerClosed()) {
- getStreamer().setHflush();
- }
- }
- } catch (InterruptedIOException interrupt) {
- // This kind of error doesn't mean that the stream itself is broken - just the
- // flushing thread got interrupted. So, we shouldn't close down the writer,
- // but instead just propagate the error
- throw interrupt;
- } catch (IOException e) {
- DFSClient.LOG.warn("Error while syncing", e);
- synchronized (this) {
- if (!isClosed()) {
- getStreamer().getLastException().set(e);
- closeThreads(true);
- }
- }
- throw e;
- }
- }
-
- /**
- * @deprecated use {@link HdfsDataOutputStream#getCurrentBlockReplication()}.
- */
- @Deprecated
- public synchronized int getNumCurrentReplicas() throws IOException {
- return getCurrentBlockReplication();
- }
-
- /**
- * Note that this is not a public API;
- * use {@link HdfsDataOutputStream#getCurrentBlockReplication()} instead.
- *
- * @return the number of valid replicas of the current block
- */
- public synchronized int getCurrentBlockReplication() throws IOException {
- dfsClient.checkOpen();
- checkClosed();
- if (getStreamer().streamerClosed()) {
- return blockReplication; // no pipeline, return repl factor of file
- }
- DatanodeInfo[] currentNodes = getStreamer().getNodes();
- if (currentNodes == null) {
- return blockReplication; // no pipeline, return repl factor of file
- }
- return currentNodes.length;
- }
-
- /**
- * Waits till all existing data is flushed and confirmations
- * received from datanodes.
- */
- protected void flushInternal() throws IOException {
- long toWaitFor;
- synchronized (this) {
- dfsClient.checkOpen();
- checkClosed();
- //
- // If there is data in the current buffer, send it across
- //
- getStreamer().queuePacket(currentPacket);
- currentPacket = null;
- toWaitFor = getStreamer().getLastQueuedSeqno();
- }
-
- getStreamer().waitForAckedSeqno(toWaitFor);
- }
-
- protected synchronized void start() {
- getStreamer().start();
- }
-
- /**
- * Aborts this output stream and releases any system
- * resources associated with this stream.
- */
- synchronized void abort() throws IOException {
- if (isClosed()) {
- return;
- }
- getStreamer().getLastException().set(new IOException("Lease timeout of "
- + (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
- closeThreads(true);
- dfsClient.endFileLease(fileId);
- }
-
- boolean isClosed() {
- return closed || getStreamer().streamerClosed();
- }
-
- void setClosed() {
- closed = true;
- getStreamer().release();
- }
-
- // shutdown datastreamer and responseprocessor threads.
- // interrupt datastreamer if force is true
- protected void closeThreads(boolean force) throws IOException {
- try {
- getStreamer().close(force);
- getStreamer().join();
- getStreamer().closeSocket();
- } catch (InterruptedException e) {
- throw new IOException("Failed to shutdown streamer");
- } finally {
- getStreamer().setSocketToNull();
- setClosed();
- }
- }
-
- /**
- * Closes this output stream and releases any system
- * resources associated with this stream.
- */
- @Override
- public synchronized void close() throws IOException {
- TraceScope scope =
- dfsClient.getPathTraceScope("DFSOutputStream#close", src);
- try {
- closeImpl();
- } finally {
- scope.close();
- }
- }
-
- protected synchronized void closeImpl() throws IOException {
- if (isClosed()) {
- getStreamer().getLastException().check(true);
- return;
- }
-
- try {
- flushBuffer(); // flush from all upper layers
-
- if (currentPacket != null) {
- enqueueCurrentPacket();
- }
-
- if (getStreamer().getBytesCurBlock() != 0) {
- setCurrentPacketToEmpty();
- }
-
- flushInternal(); // flush all data to Datanodes
- // get last block before destroying the streamer
- ExtendedBlock lastBlock = getStreamer().getBlock();
- closeThreads(false);
- TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
- try {
- completeFile(lastBlock);
- } finally {
- scope.close();
- }
- dfsClient.endFileLease(fileId);
- } catch (ClosedChannelException e) {
- } finally {
- setClosed();
- }
- }
-
- // should be called holding (this) lock since setTestFilename() may
- // be called during unit tests
- protected void completeFile(ExtendedBlock last) throws IOException {
- long localstart = Time.monotonicNow();
- final DfsClientConf conf = dfsClient.getConf();
- long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
- boolean fileComplete = false;
- int retries = conf.getNumBlockWriteLocateFollowingRetry();
- while (!fileComplete) {
- fileComplete =
- dfsClient.namenode.complete(src, dfsClient.clientName, last, fileId);
- if (!fileComplete) {
- final int hdfsTimeout = conf.getHdfsTimeout();
- if (!dfsClient.clientRunning
- || (hdfsTimeout > 0
- && localstart + hdfsTimeout < Time.monotonicNow())) {
- String msg = "Unable to close file because dfsclient " +
- " was unable to contact the HDFS servers." +
- " clientRunning " + dfsClient.clientRunning +
- " hdfsTimeout " + hdfsTimeout;
- DFSClient.LOG.info(msg);
- throw new IOException(msg);
- }
- try {
- if (retries == 0) {
- throw new IOException("Unable to close file because the last block"
- + " does not have enough number of replicas.");
- }
- retries--;
- Thread.sleep(sleeptime);
- sleeptime *= 2;
- if (Time.monotonicNow() - localstart > 5000) {
- DFSClient.LOG.info("Could not complete " + src + " retrying...");
- }
- } catch (InterruptedException ie) {
- DFSClient.LOG.warn("Caught exception ", ie);
- }
- }
- }
- }
-
- @VisibleForTesting
- public void setArtificialSlowdown(long period) {
- getStreamer().setArtificialSlowdown(period);
- }
-
- @VisibleForTesting
- public synchronized void setChunksPerPacket(int value) {
- chunksPerPacket = Math.min(chunksPerPacket, value);
- packetSize = (bytesPerChecksum + getChecksumSize()) * chunksPerPacket;
- }
-
- /**
- * Returns the size of a file as it was when this stream was opened
- */
- public long getInitialLen() {
- return initialFileSize;
- }
-
- /**
- * @return the FileEncryptionInfo for this stream, or null if not encrypted.
- */
- public FileEncryptionInfo getFileEncryptionInfo() {
- return fileEncryptionInfo;
- }
-
- /**
- * Returns the access token currently used by streamer, for testing only
- */
- synchronized Token<BlockTokenIdentifier> getBlockToken() {
- return getStreamer().getBlockToken();
- }
-
- @Override
- public void setDropBehind(Boolean dropBehind) throws IOException {
- CachingStrategy prevStrategy, nextStrategy;
- // CachingStrategy is immutable. So build a new CachingStrategy with the
- // modifications we want, and compare-and-swap it in.
- do {
- prevStrategy = this.cachingStrategy.get();
- nextStrategy = new CachingStrategy.Builder(prevStrategy).
- setDropBehind(dropBehind).build();
- } while (!this.cachingStrategy.compareAndSet(prevStrategy, nextStrategy));
- }
-
- @VisibleForTesting
- ExtendedBlock getBlock() {
- return getStreamer().getBlock();
- }
-
- @VisibleForTesting
- public long getFileId() {
- return fileId;
- }
-
- /**
- * Return the source of stream.
- */
- String getSrc() {
- return src;
- }
-
- /**
- * Returns the data streamer object.
- */
- protected DataStreamer getStreamer() {
- return streamer;
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
deleted file mode 100755
index 22055c3..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
+++ /dev/null
@@ -1,345 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.BufferOverflowException;
-import java.nio.channels.ClosedChannelException;
-import java.util.Arrays;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
-import org.apache.hadoop.hdfs.util.ByteArrayManager;
-import org.apache.htrace.Span;
-
-/****************************************************************
- * DFSPacket is used by DataStreamer and DFSOutputStream.
- * DFSOutputStream generates packets and then ask DatStreamer
- * to send them to datanodes.
- ****************************************************************/
-
-@InterfaceAudience.Private
-class DFSPacket {
- public static final long HEART_BEAT_SEQNO = -1L;
- private static long[] EMPTY = new long[0];
- private final long seqno; // sequence number of buffer in block
- private final long offsetInBlock; // offset in block
- private boolean syncBlock; // this packet forces the current block to disk
- private int numChunks; // number of chunks currently in packet
- private final int maxChunks; // max chunks in packet
- private byte[] buf;
- private final boolean lastPacketInBlock; // is this the last packet in block?
-
- /**
- * buf is pointed into like follows:
- * (C is checksum data, D is payload data)
- *
- * [_________CCCCCCCCC________________DDDDDDDDDDDDDDDD___]
- * ^ ^ ^ ^
- * | checksumPos dataStart dataPos
- * checksumStart
- *
- * Right before sending, we move the checksum data to immediately precede
- * the actual data, and then insert the header into the buffer immediately
- * preceding the checksum data, so we make sure to keep enough space in
- * front of the checksum data to support the largest conceivable header.
- */
- private int checksumStart;
- private int checksumPos;
- private final int dataStart;
- private int dataPos;
- private long[] traceParents = EMPTY;
- private int traceParentsUsed;
- private Span span;
-
- /**
- * Create a new packet.
- *
- * @param buf the buffer storing data and checksums
- * @param chunksPerPkt maximum number of chunks per packet.
- * @param offsetInBlock offset in bytes into the HDFS block.
- * @param seqno the sequence number of this packet
- * @param checksumSize the size of checksum
- * @param lastPacketInBlock if this is the last packet
- */
- DFSPacket(byte[] buf, int chunksPerPkt, long offsetInBlock, long seqno,
- int checksumSize, boolean lastPacketInBlock) {
- this.lastPacketInBlock = lastPacketInBlock;
- this.numChunks = 0;
- this.offsetInBlock = offsetInBlock;
- this.seqno = seqno;
-
- this.buf = buf;
-
- checksumStart = PacketHeader.PKT_MAX_HEADER_LEN;
- checksumPos = checksumStart;
- dataStart = checksumStart + (chunksPerPkt * checksumSize);
- dataPos = dataStart;
- maxChunks = chunksPerPkt;
- }
-
- /**
- * Write data to this packet.
- *
- * @param inarray input array of data
- * @param off the offset of data to write
- * @param len the length of data to write
- * @throws ClosedChannelException
- */
- synchronized void writeData(byte[] inarray, int off, int len)
- throws ClosedChannelException {
- checkBuffer();
- if (dataPos + len > buf.length) {
- throw new BufferOverflowException();
- }
- System.arraycopy(inarray, off, buf, dataPos, len);
- dataPos += len;
- }
-
- /**
- * Write checksums to this packet
- *
- * @param inarray input array of checksums
- * @param off the offset of checksums to write
- * @param len the length of checksums to write
- * @throws ClosedChannelException
- */
- synchronized void writeChecksum(byte[] inarray, int off, int len)
- throws ClosedChannelException {
- checkBuffer();
- if (len == 0) {
- return;
- }
- if (checksumPos + len > dataStart) {
- throw new BufferOverflowException();
- }
- System.arraycopy(inarray, off, buf, checksumPos, len);
- checksumPos += len;
- }
-
- /**
- * Write the full packet, including the header, to the given output stream.
- *
- * @param stm
- * @throws IOException
- */
- synchronized void writeTo(DataOutputStream stm) throws IOException {
- checkBuffer();
-
- final int dataLen = dataPos - dataStart;
- final int checksumLen = checksumPos - checksumStart;
- final int pktLen = HdfsConstants.BYTES_IN_INTEGER + dataLen + checksumLen;
-
- PacketHeader header = new PacketHeader(
- pktLen, offsetInBlock, seqno, lastPacketInBlock, dataLen, syncBlock);
-
- if (checksumPos != dataStart) {
- // Move the checksum to cover the gap. This can happen for the last
- // packet or during an hflush/hsync call.
- System.arraycopy(buf, checksumStart, buf,
- dataStart - checksumLen , checksumLen);
- checksumPos = dataStart;
- checksumStart = checksumPos - checksumLen;
- }
-
- final int headerStart = checksumStart - header.getSerializedSize();
- assert checksumStart + 1 >= header.getSerializedSize();
- assert headerStart >= 0;
- assert headerStart + header.getSerializedSize() == checksumStart;
-
- // Copy the header data into the buffer immediately preceding the checksum
- // data.
- System.arraycopy(header.getBytes(), 0, buf, headerStart,
- header.getSerializedSize());
-
- // corrupt the data for testing.
- if (DFSClientFaultInjector.get().corruptPacket()) {
- buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
- }
-
- // Write the now contiguous full packet to the output stream.
- stm.write(buf, headerStart, header.getSerializedSize() + checksumLen + dataLen);
-
- // undo corruption.
- if (DFSClientFaultInjector.get().uncorruptPacket()) {
- buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
- }
- }
-
- private synchronized void checkBuffer() throws ClosedChannelException {
- if (buf == null) {
- throw new ClosedChannelException();
- }
- }
-
- /**
- * Release the buffer in this packet to ByteArrayManager.
- *
- * @param bam
- */
- synchronized void releaseBuffer(ByteArrayManager bam) {
- bam.release(buf);
- buf = null;
- }
-
- /**
- * get the packet's last byte's offset in the block
- *
- * @return the packet's last byte's offset in the block
- */
- synchronized long getLastByteOffsetBlock() {
- return offsetInBlock + dataPos - dataStart;
- }
-
- /**
- * Check if this packet is a heart beat packet
- *
- * @return true if the sequence number is HEART_BEAT_SEQNO
- */
- boolean isHeartbeatPacket() {
- return seqno == HEART_BEAT_SEQNO;
- }
-
- /**
- * check if this packet is the last packet in block
- *
- * @return true if the packet is the last packet
- */
- boolean isLastPacketInBlock(){
- return lastPacketInBlock;
- }
-
- /**
- * get sequence number of this packet
- *
- * @return the sequence number of this packet
- */
- long getSeqno(){
- return seqno;
- }
-
- /**
- * get the number of chunks this packet contains
- *
- * @return the number of chunks in this packet
- */
- synchronized int getNumChunks(){
- return numChunks;
- }
-
- /**
- * increase the number of chunks by one
- */
- synchronized void incNumChunks(){
- numChunks++;
- }
-
- /**
- * get the maximum number of packets
- *
- * @return the maximum number of packets
- */
- int getMaxChunks(){
- return maxChunks;
- }
-
- /**
- * set if to sync block
- *
- * @param syncBlock if to sync block
- */
- synchronized void setSyncBlock(boolean syncBlock){
- this.syncBlock = syncBlock;
- }
-
- @Override
- public String toString() {
- return "packet seqno: " + this.seqno +
- " offsetInBlock: " + this.offsetInBlock +
- " lastPacketInBlock: " + this.lastPacketInBlock +
- " lastByteOffsetInBlock: " + this.getLastByteOffsetBlock();
- }
-
- /**
- * Add a trace parent span for this packet.<p/>
- *
- * Trace parent spans for a packet are the trace spans responsible for
- * adding data to that packet. We store them as an array of longs for
- * efficiency.<p/>
- *
- * Protected by the DFSOutputStream dataQueue lock.
- */
- public void addTraceParent(Span span) {
- if (span == null) {
- return;
- }
- addTraceParent(span.getSpanId());
- }
-
- public void addTraceParent(long id) {
- if (traceParentsUsed == traceParents.length) {
- int newLength = (traceParents.length == 0) ? 8 :
- traceParents.length * 2;
- traceParents = Arrays.copyOf(traceParents, newLength);
- }
- traceParents[traceParentsUsed] = id;
- traceParentsUsed++;
- }
-
- /**
- * Get the trace parent spans for this packet.<p/>
- *
- * Will always be non-null.<p/>
- *
- * Protected by the DFSOutputStream dataQueue lock.
- */
- public long[] getTraceParents() {
- // Remove duplicates from the array.
- int len = traceParentsUsed;
- Arrays.sort(traceParents, 0, len);
- int i = 0, j = 0;
- long prevVal = 0; // 0 is not a valid span id
- while (true) {
- if (i == len) {
- break;
- }
- long val = traceParents[i];
- if (val != prevVal) {
- traceParents[j] = val;
- j++;
- prevVal = val;
- }
- i++;
- }
- if (j < traceParents.length) {
- traceParents = Arrays.copyOf(traceParents, j);
- traceParentsUsed = traceParents.length;
- }
- return traceParents;
- }
-
- public void setTraceSpan(Span span) {
- this.span = span;
- }
-
- public Span getTraceSpan() {
- return span;
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
index fee7138..16695fa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java
@@ -1477,27 +1477,4 @@ public class DFSUtil {
return cryptoProvider;
}
- public static int getIoFileBufferSize(Configuration conf) {
- return conf.getInt(
- CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
- CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
- }
-
- public static int getSmallBufferSize(Configuration conf) {
- return Math.min(getIoFileBufferSize(conf) / 2, 512);
- }
-
- /**
- * Probe for HDFS Encryption being enabled; this uses the value of
- * the option {@link DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI},
- * returning true if that property contains a non-empty, non-whitespace
- * string.
- * @param conf configuration to probe
- * @return true if encryption is considered enabled.
- */
- public static boolean isHDFSEncryptionEnabled(Configuration conf) {
- return !conf.getTrimmed(
- DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "").isEmpty();
- }
-
}
[10/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and
related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
new file mode 100644
index 0000000..12496e2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
@@ -0,0 +1,60 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Used for injecting faults in DFSClient and DFSOutputStream tests.
+ * Calls into this are a no-op in production code.
+ */
+@VisibleForTesting
+@InterfaceAudience.Private
+public class DFSClientFaultInjector {
+ private static DFSClientFaultInjector instance = new DFSClientFaultInjector();
+ public static AtomicLong exceptionNum = new AtomicLong(0);
+
+ public static DFSClientFaultInjector get() {
+ return instance;
+ }
+ public static void set(DFSClientFaultInjector instance) {
+ DFSClientFaultInjector.instance = instance;
+ }
+
+ public boolean corruptPacket() {
+ return false;
+ }
+
+ public boolean uncorruptPacket() {
+ return false;
+ }
+
+ public boolean failPacket() {
+ return false;
+ }
+
+ public void startFetchFromDatanode() {}
+
+ public void fetchFromDatanodeException() {}
+
+ public void readFromDatanodeDelay() {}
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
new file mode 100644
index 0000000..2a228e8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
@@ -0,0 +1,58 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * The client-side metrics for hedged read feature.
+ * This class has a number of metrics variables that are publicly accessible,
+ * we can grab them from client side, like HBase.
+ */
+@InterfaceAudience.Private
+public class DFSHedgedReadMetrics {
+ public final AtomicLong hedgedReadOps = new AtomicLong();
+ public final AtomicLong hedgedReadOpsWin = new AtomicLong();
+ public final AtomicLong hedgedReadOpsInCurThread = new AtomicLong();
+
+ public void incHedgedReadOps() {
+ hedgedReadOps.incrementAndGet();
+ }
+
+ public void incHedgedReadOpsInCurThread() {
+ hedgedReadOpsInCurThread.incrementAndGet();
+ }
+
+ public void incHedgedReadWins() {
+ hedgedReadOpsWin.incrementAndGet();
+ }
+
+ public long getHedgedReadOps() {
+ return hedgedReadOps.longValue();
+ }
+
+ public long getHedgedReadOpsInCurThread() {
+ return hedgedReadOpsInCurThread.longValue();
+ }
+
+ public long getHedgedReadWins() {
+ return hedgedReadOpsWin.longValue();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
new file mode 100644
index 0000000..11a1d29
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
@@ -0,0 +1,239 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import com.google.common.collect.Iterators;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.inotify.EventBatch;
+import org.apache.hadoop.hdfs.inotify.EventBatchList;
+import org.apache.hadoop.hdfs.inotify.MissingEventsException;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.util.Time;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Stream for reading inotify events. DFSInotifyEventInputStreams should not
+ * be shared among multiple threads.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Unstable
+public class DFSInotifyEventInputStream {
+ public static final Logger LOG = LoggerFactory.getLogger(
+ DFSInotifyEventInputStream.class);
+
+ /**
+ * The trace sampler to use when making RPCs to the NameNode.
+ */
+ private final Sampler<?> traceSampler;
+
+ private final ClientProtocol namenode;
+ private Iterator<EventBatch> it;
+ private long lastReadTxid;
+ /**
+ * The most recent txid the NameNode told us it has sync'ed -- helps us
+ * determine how far behind we are in the edit stream.
+ */
+ private long syncTxid;
+ /**
+ * Used to generate wait times in {@link DFSInotifyEventInputStream#take()}.
+ */
+ private Random rng = new Random();
+
+ private static final int INITIAL_WAIT_MS = 10;
+
+ DFSInotifyEventInputStream(Sampler<?> traceSampler, ClientProtocol namenode)
+ throws IOException {
+ // Only consider new transaction IDs.
+ this(traceSampler, namenode, namenode.getCurrentEditLogTxid());
+ }
+
+ DFSInotifyEventInputStream(Sampler traceSampler, ClientProtocol namenode,
+ long lastReadTxid) throws IOException {
+ this.traceSampler = traceSampler;
+ this.namenode = namenode;
+ this.it = Iterators.emptyIterator();
+ this.lastReadTxid = lastReadTxid;
+ }
+
+ /**
+ * Returns the next batch of events in the stream or null if no new
+ * batches are currently available.
+ *
+ * @throws IOException because of network error or edit log
+ * corruption. Also possible if JournalNodes are unresponsive in the
+ * QJM setting (even one unresponsive JournalNode is enough in rare cases),
+ * so catching this exception and retrying at least a few times is
+ * recommended.
+ * @throws MissingEventsException if we cannot return the next batch in the
+ * stream because the data for the events (and possibly some subsequent
+ * events) has been deleted (generally because this stream is a very large
+ * number of transactions behind the current state of the NameNode). It is
+ * safe to continue reading from the stream after this exception is thrown
+ * The next available batch of events will be returned.
+ */
+ public EventBatch poll() throws IOException, MissingEventsException {
+ TraceScope scope =
+ Trace.startSpan("inotifyPoll", traceSampler);
+ try {
+ // need to keep retrying until the NN sends us the latest committed txid
+ if (lastReadTxid == -1) {
+ LOG.debug("poll(): lastReadTxid is -1, reading current txid from NN");
+ lastReadTxid = namenode.getCurrentEditLogTxid();
+ return null;
+ }
+ if (!it.hasNext()) {
+ EventBatchList el = namenode.getEditsFromTxid(lastReadTxid + 1);
+ if (el.getLastTxid() != -1) {
+ // we only want to set syncTxid when we were actually able to read some
+ // edits on the NN -- otherwise it will seem like edits are being
+ // generated faster than we can read them when the problem is really
+ // that we are temporarily unable to read edits
+ syncTxid = el.getSyncTxid();
+ it = el.getBatches().iterator();
+ long formerLastReadTxid = lastReadTxid;
+ lastReadTxid = el.getLastTxid();
+ if (el.getFirstTxid() != formerLastReadTxid + 1) {
+ throw new MissingEventsException(formerLastReadTxid + 1,
+ el.getFirstTxid());
+ }
+ } else {
+ LOG.debug("poll(): read no edits from the NN when requesting edits " +
+ "after txid {}", lastReadTxid);
+ return null;
+ }
+ }
+
+ if (it.hasNext()) { // can be empty if el.getLastTxid != -1 but none of the
+ // newly seen edit log ops actually got converted to events
+ return it.next();
+ } else {
+ return null;
+ }
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Return a estimate of how many transaction IDs behind the NameNode's
+ * current state this stream is. Clients should periodically call this method
+ * and check if its result is steadily increasing, which indicates that they
+ * are falling behind (i.e. transaction are being generated faster than the
+ * client is reading them). If a client falls too far behind events may be
+ * deleted before the client can read them.
+ * <p/>
+ * A return value of -1 indicates that an estimate could not be produced, and
+ * should be ignored. The value returned by this method is really only useful
+ * when compared to previous or subsequent returned values.
+ */
+ public long getTxidsBehindEstimate() {
+ if (syncTxid == 0) {
+ return -1;
+ } else {
+ assert syncTxid >= lastReadTxid;
+ // this gives the difference between the last txid we have fetched to the
+ // client and syncTxid at the time we last fetched events from the
+ // NameNode
+ return syncTxid - lastReadTxid;
+ }
+ }
+
+ /**
+ * Returns the next event batch in the stream, waiting up to the specified
+ * amount of time for a new batch. Returns null if one is not available at the
+ * end of the specified amount of time. The time before the method returns may
+ * exceed the specified amount of time by up to the time required for an RPC
+ * to the NameNode.
+ *
+ * @param time number of units of the given TimeUnit to wait
+ * @param tu the desired TimeUnit
+ * @throws IOException see {@link DFSInotifyEventInputStream#poll()}
+ * @throws MissingEventsException
+ * see {@link DFSInotifyEventInputStream#poll()}
+ * @throws InterruptedException if the calling thread is interrupted
+ */
+ public EventBatch poll(long time, TimeUnit tu) throws IOException,
+ InterruptedException, MissingEventsException {
+ TraceScope scope = Trace.startSpan("inotifyPollWithTimeout", traceSampler);
+ EventBatch next = null;
+ try {
+ long initialTime = Time.monotonicNow();
+ long totalWait = TimeUnit.MILLISECONDS.convert(time, tu);
+ long nextWait = INITIAL_WAIT_MS;
+ while ((next = poll()) == null) {
+ long timeLeft = totalWait - (Time.monotonicNow() - initialTime);
+ if (timeLeft <= 0) {
+ LOG.debug("timed poll(): timed out");
+ break;
+ } else if (timeLeft < nextWait * 2) {
+ nextWait = timeLeft;
+ } else {
+ nextWait *= 2;
+ }
+ LOG.debug("timed poll(): poll() returned null, sleeping for {} ms",
+ nextWait);
+ Thread.sleep(nextWait);
+ }
+ } finally {
+ scope.close();
+ }
+ return next;
+ }
+
+ /**
+ * Returns the next batch of events in the stream, waiting indefinitely if
+ * a new batch is not immediately available.
+ *
+ * @throws IOException see {@link DFSInotifyEventInputStream#poll()}
+ * @throws MissingEventsException see
+ * {@link DFSInotifyEventInputStream#poll()}
+ * @throws InterruptedException if the calling thread is interrupted
+ */
+ public EventBatch take() throws IOException, InterruptedException,
+ MissingEventsException {
+ TraceScope scope = Trace.startSpan("inotifyTake", traceSampler);
+ EventBatch next = null;
+ try {
+ int nextWaitMin = INITIAL_WAIT_MS;
+ while ((next = poll()) == null) {
+ // sleep for a random period between nextWaitMin and nextWaitMin * 2
+ // to avoid stampedes at the NN if there are multiple clients
+ int sleepTime = nextWaitMin + rng.nextInt(nextWaitMin);
+ LOG.debug("take(): poll() returned null, sleeping for {} ms", sleepTime);
+ Thread.sleep(sleepTime);
+ // the maximum sleep is 2 minutes
+ nextWaitMin = Math.min(60000, nextWaitMin * 2);
+ }
+ } finally {
+ scope.close();
+ }
+
+ return next;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
new file mode 100644
index 0000000..139a27c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -0,0 +1,1915 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.AbstractMap;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.ByteBufferReadable;
+import org.apache.hadoop.fs.ByteBufferUtil;
+import org.apache.hadoop.fs.CanSetDropBehind;
+import org.apache.hadoop.fs.CanSetReadahead;
+import org.apache.hadoop.fs.CanUnbuffer;
+import org.apache.hadoop.fs.ChecksumException;
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
+import org.apache.hadoop.fs.ReadOption;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
+import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
+import org.apache.hadoop.io.ByteBufferPool;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.IdentityHashStore;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+import com.google.common.annotations.VisibleForTesting;
+
+/****************************************************************
+ * DFSInputStream provides bytes from a named file. It handles
+ * negotiation of the namenode and various datanodes as necessary.
+ ****************************************************************/
+@InterfaceAudience.Private
+public class DFSInputStream extends FSInputStream
+implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
+ HasEnhancedByteBufferAccess, CanUnbuffer {
+ @VisibleForTesting
+ public static boolean tcpReadsDisabledForTesting = false;
+ private long hedgedReadOpsLoopNumForTesting = 0;
+ protected final DFSClient dfsClient;
+ protected AtomicBoolean closed = new AtomicBoolean(false);
+ protected final String src;
+ protected final boolean verifyChecksum;
+
+ // state by stateful read only:
+ // (protected by lock on this)
+ /////
+ private DatanodeInfo currentNode = null;
+ protected LocatedBlock currentLocatedBlock = null;
+ protected long pos = 0;
+ protected long blockEnd = -1;
+ private BlockReader blockReader = null;
+ ////
+
+ // state shared by stateful and positional read:
+ // (protected by lock on infoLock)
+ ////
+ protected LocatedBlocks locatedBlocks = null;
+ private long lastBlockBeingWrittenLength = 0;
+ private FileEncryptionInfo fileEncryptionInfo = null;
+ protected CachingStrategy cachingStrategy;
+ ////
+
+ protected final ReadStatistics readStatistics = new ReadStatistics();
+ // lock for state shared between read and pread
+ // Note: Never acquire a lock on <this> with this lock held to avoid deadlocks
+ // (it's OK to acquire this lock when the lock on <this> is held)
+ protected final Object infoLock = new Object();
+
+ /**
+ * Track the ByteBuffers that we have handed out to readers.
+ *
+ * The value type can be either ByteBufferPool or ClientMmap, depending on
+ * whether we this is a memory-mapped buffer or not.
+ */
+ private IdentityHashStore<ByteBuffer, Object> extendedReadBuffers;
+
+ private synchronized IdentityHashStore<ByteBuffer, Object>
+ getExtendedReadBuffers() {
+ if (extendedReadBuffers == null) {
+ extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0);
+ }
+ return extendedReadBuffers;
+ }
+
+ public static class ReadStatistics {
+ public ReadStatistics() {
+ clear();
+ }
+
+ public ReadStatistics(ReadStatistics rhs) {
+ this.totalBytesRead = rhs.getTotalBytesRead();
+ this.totalLocalBytesRead = rhs.getTotalLocalBytesRead();
+ this.totalShortCircuitBytesRead = rhs.getTotalShortCircuitBytesRead();
+ this.totalZeroCopyBytesRead = rhs.getTotalZeroCopyBytesRead();
+ }
+
+ /**
+ * @return The total bytes read. This will always be at least as
+ * high as the other numbers, since it includes all of them.
+ */
+ public long getTotalBytesRead() {
+ return totalBytesRead;
+ }
+
+ /**
+ * @return The total local bytes read. This will always be at least
+ * as high as totalShortCircuitBytesRead, since all short-circuit
+ * reads are also local.
+ */
+ public long getTotalLocalBytesRead() {
+ return totalLocalBytesRead;
+ }
+
+ /**
+ * @return The total short-circuit local bytes read.
+ */
+ public long getTotalShortCircuitBytesRead() {
+ return totalShortCircuitBytesRead;
+ }
+
+ /**
+ * @return The total number of zero-copy bytes read.
+ */
+ public long getTotalZeroCopyBytesRead() {
+ return totalZeroCopyBytesRead;
+ }
+
+ /**
+ * @return The total number of bytes read which were not local.
+ */
+ public long getRemoteBytesRead() {
+ return totalBytesRead - totalLocalBytesRead;
+ }
+
+ void addRemoteBytes(long amt) {
+ this.totalBytesRead += amt;
+ }
+
+ void addLocalBytes(long amt) {
+ this.totalBytesRead += amt;
+ this.totalLocalBytesRead += amt;
+ }
+
+ void addShortCircuitBytes(long amt) {
+ this.totalBytesRead += amt;
+ this.totalLocalBytesRead += amt;
+ this.totalShortCircuitBytesRead += amt;
+ }
+
+ void addZeroCopyBytes(long amt) {
+ this.totalBytesRead += amt;
+ this.totalLocalBytesRead += amt;
+ this.totalShortCircuitBytesRead += amt;
+ this.totalZeroCopyBytesRead += amt;
+ }
+
+ void clear() {
+ this.totalBytesRead = 0;
+ this.totalLocalBytesRead = 0;
+ this.totalShortCircuitBytesRead = 0;
+ this.totalZeroCopyBytesRead = 0;
+ }
+
+ private long totalBytesRead;
+
+ private long totalLocalBytesRead;
+
+ private long totalShortCircuitBytesRead;
+
+ private long totalZeroCopyBytesRead;
+ }
+
+ /**
+ * This variable tracks the number of failures since the start of the
+ * most recent user-facing operation. That is to say, it should be reset
+ * whenever the user makes a call on this stream, and if at any point
+ * during the retry logic, the failure count exceeds a threshold,
+ * the errors will be thrown back to the operation.
+ *
+ * Specifically this counts the number of times the client has gone
+ * back to the namenode to get a new list of block locations, and is
+ * capped at maxBlockAcquireFailures
+ */
+ protected int failures = 0;
+
+ /* XXX Use of CocurrentHashMap is temp fix. Need to fix
+ * parallel accesses to DFSInputStream (through ptreads) properly */
+ private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
+ new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
+
+ private byte[] oneByteBuf; // used for 'int read()'
+
+ void addToDeadNodes(DatanodeInfo dnInfo) {
+ deadNodes.put(dnInfo, dnInfo);
+ }
+
+ DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
+ LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException {
+ this.dfsClient = dfsClient;
+ this.verifyChecksum = verifyChecksum;
+ this.src = src;
+ synchronized (infoLock) {
+ this.cachingStrategy = dfsClient.getDefaultReadCachingStrategy();
+ }
+ this.locatedBlocks = locatedBlocks;
+ openInfo(false);
+ }
+
+ /**
+ * Grab the open-file info from namenode
+ * @param refreshLocatedBlocks whether to re-fetch locatedblocks
+ */
+ void openInfo(boolean refreshLocatedBlocks) throws IOException,
+ UnresolvedLinkException {
+ final DfsClientConf conf = dfsClient.getConf();
+ synchronized(infoLock) {
+ lastBlockBeingWrittenLength =
+ fetchLocatedBlocksAndGetLastBlockLength(refreshLocatedBlocks);
+ int retriesForLastBlockLength = conf.getRetryTimesForGetLastBlockLength();
+ while (retriesForLastBlockLength > 0) {
+ // Getting last block length as -1 is a special case. When cluster
+ // restarts, DNs may not report immediately. At this time partial block
+ // locations will not be available with NN for getting the length. Lets
+ // retry for 3 times to get the length.
+ if (lastBlockBeingWrittenLength == -1) {
+ DFSClient.LOG.warn("Last block locations not available. "
+ + "Datanodes might not have reported blocks completely."
+ + " Will retry for " + retriesForLastBlockLength + " times");
+ waitFor(conf.getRetryIntervalForGetLastBlockLength());
+ lastBlockBeingWrittenLength =
+ fetchLocatedBlocksAndGetLastBlockLength(true);
+ } else {
+ break;
+ }
+ retriesForLastBlockLength--;
+ }
+ if (retriesForLastBlockLength == 0) {
+ throw new IOException("Could not obtain the last block locations.");
+ }
+ }
+ }
+
+ private void waitFor(int waitTime) throws IOException {
+ try {
+ Thread.sleep(waitTime);
+ } catch (InterruptedException e) {
+ throw new IOException(
+ "Interrupted while getting the last block length.");
+ }
+ }
+
+ private long fetchLocatedBlocksAndGetLastBlockLength(boolean refresh)
+ throws IOException {
+ LocatedBlocks newInfo = locatedBlocks;
+ if (locatedBlocks == null || refresh) {
+ newInfo = dfsClient.getLocatedBlocks(src, 0);
+ }
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("newInfo = " + newInfo);
+ }
+ if (newInfo == null) {
+ throw new IOException("Cannot open filename " + src);
+ }
+
+ if (locatedBlocks != null) {
+ Iterator<LocatedBlock> oldIter = locatedBlocks.getLocatedBlocks().iterator();
+ Iterator<LocatedBlock> newIter = newInfo.getLocatedBlocks().iterator();
+ while (oldIter.hasNext() && newIter.hasNext()) {
+ if (! oldIter.next().getBlock().equals(newIter.next().getBlock())) {
+ throw new IOException("Blocklist for " + src + " has changed!");
+ }
+ }
+ }
+ locatedBlocks = newInfo;
+ long lastBlockBeingWrittenLength = 0;
+ if (!locatedBlocks.isLastBlockComplete()) {
+ final LocatedBlock last = locatedBlocks.getLastLocatedBlock();
+ if (last != null) {
+ if (last.getLocations().length == 0) {
+ if (last.getBlockSize() == 0) {
+ // if the length is zero, then no data has been written to
+ // datanode. So no need to wait for the locations.
+ return 0;
+ }
+ return -1;
+ }
+ final long len = readBlockLength(last);
+ last.getBlock().setNumBytes(len);
+ lastBlockBeingWrittenLength = len;
+ }
+ }
+
+ fileEncryptionInfo = locatedBlocks.getFileEncryptionInfo();
+
+ return lastBlockBeingWrittenLength;
+ }
+
+ /** Read the block length from one of the datanodes. */
+ private long readBlockLength(LocatedBlock locatedblock) throws IOException {
+ assert locatedblock != null : "LocatedBlock cannot be null";
+ int replicaNotFoundCount = locatedblock.getLocations().length;
+
+ final DfsClientConf conf = dfsClient.getConf();
+ for(DatanodeInfo datanode : locatedblock.getLocations()) {
+ ClientDatanodeProtocol cdp = null;
+
+ try {
+ cdp = DFSUtilClient.createClientDatanodeProtocolProxy(datanode,
+ dfsClient.getConfiguration(), conf.getSocketTimeout(),
+ conf.isConnectToDnViaHostname(), locatedblock);
+
+ final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
+
+ if (n >= 0) {
+ return n;
+ }
+ }
+ catch(IOException ioe) {
+ if (ioe instanceof RemoteException &&
+ (((RemoteException) ioe).unwrapRemoteException() instanceof
+ ReplicaNotFoundException)) {
+ // special case : replica might not be on the DN, treat as 0 length
+ replicaNotFoundCount--;
+ }
+
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode "
+ + datanode + " for block " + locatedblock.getBlock(), ioe);
+ }
+ } finally {
+ if (cdp != null) {
+ RPC.stopProxy(cdp);
+ }
+ }
+ }
+
+ // Namenode told us about these locations, but none know about the replica
+ // means that we hit the race between pipeline creation start and end.
+ // we require all 3 because some other exception could have happened
+ // on a DN that has it. we want to report that error
+ if (replicaNotFoundCount == 0) {
+ return 0;
+ }
+
+ throw new IOException("Cannot obtain block length for " + locatedblock);
+ }
+
+ public long getFileLength() {
+ synchronized(infoLock) {
+ return locatedBlocks == null? 0:
+ locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
+ }
+ }
+
+ // Short circuit local reads are forbidden for files that are
+ // under construction. See HDFS-2757.
+ boolean shortCircuitForbidden() {
+ synchronized(infoLock) {
+ return locatedBlocks.isUnderConstruction();
+ }
+ }
+
+ /**
+ * Returns the datanode from which the stream is currently reading.
+ */
+ public synchronized DatanodeInfo getCurrentDatanode() {
+ return currentNode;
+ }
+
+ /**
+ * Returns the block containing the target position.
+ */
+ synchronized public ExtendedBlock getCurrentBlock() {
+ if (currentLocatedBlock == null){
+ return null;
+ }
+ return currentLocatedBlock.getBlock();
+ }
+
+ /**
+ * Return collection of blocks that has already been located.
+ */
+ public List<LocatedBlock> getAllBlocks() throws IOException {
+ return getBlockRange(0, getFileLength());
+ }
+
+ /**
+ * Get block at the specified position.
+ * Fetch it from the namenode if not cached.
+ *
+ * @param offset block corresponding to this offset in file is returned
+ * @return located block
+ * @throws IOException
+ */
+ protected LocatedBlock getBlockAt(long offset) throws IOException {
+ synchronized(infoLock) {
+ assert (locatedBlocks != null) : "locatedBlocks is null";
+
+ final LocatedBlock blk;
+
+ //check offset
+ if (offset < 0 || offset >= getFileLength()) {
+ throw new IOException("offset < 0 || offset >= getFileLength(), offset="
+ + offset
+ + ", locatedBlocks=" + locatedBlocks);
+ }
+ else if (offset >= locatedBlocks.getFileLength()) {
+ // offset to the portion of the last block,
+ // which is not known to the name-node yet;
+ // getting the last block
+ blk = locatedBlocks.getLastLocatedBlock();
+ }
+ else {
+ // search cached blocks first
+ int targetBlockIdx = locatedBlocks.findBlock(offset);
+ if (targetBlockIdx < 0) { // block is not cached
+ targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
+ // fetch more blocks
+ final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
+ assert (newBlocks != null) : "Could not find target position " + offset;
+ locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
+ }
+ blk = locatedBlocks.get(targetBlockIdx);
+ }
+ return blk;
+ }
+ }
+
+ /** Fetch a block from namenode and cache it */
+ protected void fetchBlockAt(long offset) throws IOException {
+ synchronized(infoLock) {
+ int targetBlockIdx = locatedBlocks.findBlock(offset);
+ if (targetBlockIdx < 0) { // block is not cached
+ targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
+ }
+ // fetch blocks
+ final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
+ if (newBlocks == null) {
+ throw new IOException("Could not find target position " + offset);
+ }
+ locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
+ }
+ }
+
+ /**
+ * Get blocks in the specified range.
+ * Fetch them from the namenode if not cached. This function
+ * will not get a read request beyond the EOF.
+ * @param offset starting offset in file
+ * @param length length of data
+ * @return consequent segment of located blocks
+ * @throws IOException
+ */
+ private List<LocatedBlock> getBlockRange(long offset,
+ long length) throws IOException {
+ // getFileLength(): returns total file length
+ // locatedBlocks.getFileLength(): returns length of completed blocks
+ if (offset >= getFileLength()) {
+ throw new IOException("Offset: " + offset +
+ " exceeds file length: " + getFileLength());
+ }
+ synchronized(infoLock) {
+ final List<LocatedBlock> blocks;
+ final long lengthOfCompleteBlk = locatedBlocks.getFileLength();
+ final boolean readOffsetWithinCompleteBlk = offset < lengthOfCompleteBlk;
+ final boolean readLengthPastCompleteBlk = offset + length > lengthOfCompleteBlk;
+
+ if (readOffsetWithinCompleteBlk) {
+ //get the blocks of finalized (completed) block range
+ blocks = getFinalizedBlockRange(offset,
+ Math.min(length, lengthOfCompleteBlk - offset));
+ } else {
+ blocks = new ArrayList<LocatedBlock>(1);
+ }
+
+ // get the blocks from incomplete block range
+ if (readLengthPastCompleteBlk) {
+ blocks.add(locatedBlocks.getLastLocatedBlock());
+ }
+
+ return blocks;
+ }
+ }
+
+ /**
+ * Get blocks in the specified range.
+ * Includes only the complete blocks.
+ * Fetch them from the namenode if not cached.
+ */
+ private List<LocatedBlock> getFinalizedBlockRange(
+ long offset, long length) throws IOException {
+ synchronized(infoLock) {
+ assert (locatedBlocks != null) : "locatedBlocks is null";
+ List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
+ // search cached blocks first
+ int blockIdx = locatedBlocks.findBlock(offset);
+ if (blockIdx < 0) { // block is not cached
+ blockIdx = LocatedBlocks.getInsertIndex(blockIdx);
+ }
+ long remaining = length;
+ long curOff = offset;
+ while(remaining > 0) {
+ LocatedBlock blk = null;
+ if(blockIdx < locatedBlocks.locatedBlockCount())
+ blk = locatedBlocks.get(blockIdx);
+ if (blk == null || curOff < blk.getStartOffset()) {
+ LocatedBlocks newBlocks;
+ newBlocks = dfsClient.getLocatedBlocks(src, curOff, remaining);
+ locatedBlocks.insertRange(blockIdx, newBlocks.getLocatedBlocks());
+ continue;
+ }
+ assert curOff >= blk.getStartOffset() : "Block not found";
+ blockRange.add(blk);
+ long bytesRead = blk.getStartOffset() + blk.getBlockSize() - curOff;
+ remaining -= bytesRead;
+ curOff += bytesRead;
+ blockIdx++;
+ }
+ return blockRange;
+ }
+ }
+
+ /**
+ * Open a DataInputStream to a DataNode so that it can be read from.
+ * We get block ID and the IDs of the destinations at startup, from the namenode.
+ */
+ private synchronized DatanodeInfo blockSeekTo(long target) throws IOException {
+ if (target >= getFileLength()) {
+ throw new IOException("Attempted to read past end of file");
+ }
+
+ // Will be getting a new BlockReader.
+ closeCurrentBlockReaders();
+
+ //
+ // Connect to best DataNode for desired Block, with potential offset
+ //
+ DatanodeInfo chosenNode = null;
+ int refetchToken = 1; // only need to get a new access token once
+ int refetchEncryptionKey = 1; // only need to get a new encryption key once
+
+ boolean connectFailedOnce = false;
+
+ while (true) {
+ //
+ // Compute desired block
+ //
+ LocatedBlock targetBlock = getBlockAt(target);
+
+ // update current position
+ this.pos = target;
+ this.blockEnd = targetBlock.getStartOffset() +
+ targetBlock.getBlockSize() - 1;
+ this.currentLocatedBlock = targetBlock;
+
+ long offsetIntoBlock = target - targetBlock.getStartOffset();
+
+ DNAddrPair retval = chooseDataNode(targetBlock, null);
+ chosenNode = retval.info;
+ InetSocketAddress targetAddr = retval.addr;
+ StorageType storageType = retval.storageType;
+
+ try {
+ blockReader = getBlockReader(targetBlock, offsetIntoBlock,
+ targetBlock.getBlockSize() - offsetIntoBlock, targetAddr,
+ storageType, chosenNode);
+ if(connectFailedOnce) {
+ DFSClient.LOG.info("Successfully connected to " + targetAddr +
+ " for " + targetBlock.getBlock());
+ }
+ return chosenNode;
+ } catch (IOException ex) {
+ if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
+ DFSClient.LOG.info("Will fetch a new encryption key and retry, "
+ + "encryption key was invalid when connecting to " + targetAddr
+ + " : " + ex);
+ // The encryption key used is invalid.
+ refetchEncryptionKey--;
+ dfsClient.clearDataEncryptionKey();
+ } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) {
+ refetchToken--;
+ fetchBlockAt(target);
+ } else {
+ connectFailedOnce = true;
+ DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
+ + ", add to deadNodes and continue. " + ex, ex);
+ // Put chosen node into dead list, continue
+ addToDeadNodes(chosenNode);
+ }
+ }
+ }
+ }
+
+ protected BlockReader getBlockReader(LocatedBlock targetBlock,
+ long offsetInBlock, long length, InetSocketAddress targetAddr,
+ StorageType storageType, DatanodeInfo datanode) throws IOException {
+ ExtendedBlock blk = targetBlock.getBlock();
+ Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
+ CachingStrategy curCachingStrategy;
+ boolean shortCircuitForbidden;
+ synchronized (infoLock) {
+ curCachingStrategy = cachingStrategy;
+ shortCircuitForbidden = shortCircuitForbidden();
+ }
+ return new BlockReaderFactory(dfsClient.getConf()).
+ setInetSocketAddress(targetAddr).
+ setRemotePeerFactory(dfsClient).
+ setDatanodeInfo(datanode).
+ setStorageType(storageType).
+ setFileName(src).
+ setBlock(blk).
+ setBlockToken(accessToken).
+ setStartOffset(offsetInBlock).
+ setVerifyChecksum(verifyChecksum).
+ setClientName(dfsClient.clientName).
+ setLength(length).
+ setCachingStrategy(curCachingStrategy).
+ setAllowShortCircuitLocalReads(!shortCircuitForbidden).
+ setClientCacheContext(dfsClient.getClientContext()).
+ setUserGroupInformation(dfsClient.ugi).
+ setConfiguration(dfsClient.getConfiguration()).
+ build();
+ }
+
+ /**
+ * Close it down!
+ */
+ @Override
+ public synchronized void close() throws IOException {
+ if (!closed.compareAndSet(false, true)) {
+ DFSClient.LOG.debug("DFSInputStream has been closed already");
+ return;
+ }
+ dfsClient.checkOpen();
+
+ if ((extendedReadBuffers != null) && (!extendedReadBuffers.isEmpty())) {
+ final StringBuilder builder = new StringBuilder();
+ extendedReadBuffers.visitAll(new IdentityHashStore.Visitor<ByteBuffer, Object>() {
+ private String prefix = "";
+ @Override
+ public void accept(ByteBuffer k, Object v) {
+ builder.append(prefix).append(k);
+ prefix = ", ";
+ }
+ });
+ DFSClient.LOG.warn("closing file " + src + ", but there are still " +
+ "unreleased ByteBuffers allocated by read(). " +
+ "Please release " + builder.toString() + ".");
+ }
+ closeCurrentBlockReaders();
+ super.close();
+ }
+
+ @Override
+ public synchronized int read() throws IOException {
+ if (oneByteBuf == null) {
+ oneByteBuf = new byte[1];
+ }
+ int ret = read( oneByteBuf, 0, 1 );
+ return ( ret <= 0 ) ? -1 : (oneByteBuf[0] & 0xff);
+ }
+
+ /**
+ * Wraps different possible read implementations so that readBuffer can be
+ * strategy-agnostic.
+ */
+ interface ReaderStrategy {
+ public int doRead(BlockReader blockReader, int off, int len)
+ throws ChecksumException, IOException;
+
+ /**
+ * Copy data from the src ByteBuffer into the read buffer.
+ * @param src The src buffer where the data is copied from
+ * @param offset Useful only when the ReadStrategy is based on a byte array.
+ * Indicate the offset of the byte array for copy.
+ * @param length Useful only when the ReadStrategy is based on a byte array.
+ * Indicate the length of the data to copy.
+ */
+ public int copyFrom(ByteBuffer src, int offset, int length);
+ }
+
+ protected void updateReadStatistics(ReadStatistics readStatistics,
+ int nRead, BlockReader blockReader) {
+ if (nRead <= 0) return;
+ synchronized(infoLock) {
+ if (blockReader.isShortCircuit()) {
+ readStatistics.addShortCircuitBytes(nRead);
+ } else if (blockReader.isLocal()) {
+ readStatistics.addLocalBytes(nRead);
+ } else {
+ readStatistics.addRemoteBytes(nRead);
+ }
+ }
+ }
+
+ /**
+ * Used to read bytes into a byte[]
+ */
+ private class ByteArrayStrategy implements ReaderStrategy {
+ final byte[] buf;
+
+ public ByteArrayStrategy(byte[] buf) {
+ this.buf = buf;
+ }
+
+ @Override
+ public int doRead(BlockReader blockReader, int off, int len)
+ throws ChecksumException, IOException {
+ int nRead = blockReader.read(buf, off, len);
+ updateReadStatistics(readStatistics, nRead, blockReader);
+ return nRead;
+ }
+
+ @Override
+ public int copyFrom(ByteBuffer src, int offset, int length) {
+ ByteBuffer writeSlice = src.duplicate();
+ writeSlice.get(buf, offset, length);
+ return length;
+ }
+ }
+
+ /**
+ * Used to read bytes into a user-supplied ByteBuffer
+ */
+ protected class ByteBufferStrategy implements ReaderStrategy {
+ final ByteBuffer buf;
+ ByteBufferStrategy(ByteBuffer buf) {
+ this.buf = buf;
+ }
+
+ @Override
+ public int doRead(BlockReader blockReader, int off, int len)
+ throws ChecksumException, IOException {
+ int oldpos = buf.position();
+ int oldlimit = buf.limit();
+ boolean success = false;
+ try {
+ int ret = blockReader.read(buf);
+ success = true;
+ updateReadStatistics(readStatistics, ret, blockReader);
+ if (ret == 0) {
+ DFSClient.LOG.warn("zero");
+ }
+ return ret;
+ } finally {
+ if (!success) {
+ // Reset to original state so that retries work correctly.
+ buf.position(oldpos);
+ buf.limit(oldlimit);
+ }
+ }
+ }
+
+ @Override
+ public int copyFrom(ByteBuffer src, int offset, int length) {
+ ByteBuffer writeSlice = src.duplicate();
+ int remaining = Math.min(buf.remaining(), writeSlice.remaining());
+ writeSlice.limit(writeSlice.position() + remaining);
+ buf.put(writeSlice);
+ return remaining;
+ }
+ }
+
+ /* This is a used by regular read() and handles ChecksumExceptions.
+ * name readBuffer() is chosen to imply similarity to readBuffer() in
+ * ChecksumFileSystem
+ */
+ private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
+ Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+ throws IOException {
+ IOException ioe;
+
+ /* we retry current node only once. So this is set to true only here.
+ * Intention is to handle one common case of an error that is not a
+ * failure on datanode or client : when DataNode closes the connection
+ * since client is idle. If there are other cases of "non-errors" then
+ * then a datanode might be retried by setting this to true again.
+ */
+ boolean retryCurrentNode = true;
+
+ while (true) {
+ // retry as many times as seekToNewSource allows.
+ try {
+ return reader.doRead(blockReader, off, len);
+ } catch ( ChecksumException ce ) {
+ DFSClient.LOG.warn("Found Checksum error for "
+ + getCurrentBlock() + " from " + currentNode
+ + " at " + ce.getPos());
+ ioe = ce;
+ retryCurrentNode = false;
+ // we want to remember which block replicas we have tried
+ addIntoCorruptedBlockMap(getCurrentBlock(), currentNode,
+ corruptedBlockMap);
+ } catch ( IOException e ) {
+ if (!retryCurrentNode) {
+ DFSClient.LOG.warn("Exception while reading from "
+ + getCurrentBlock() + " of " + src + " from "
+ + currentNode, e);
+ }
+ ioe = e;
+ }
+ boolean sourceFound = false;
+ if (retryCurrentNode) {
+ /* possibly retry the same node so that transient errors don't
+ * result in application level failures (e.g. Datanode could have
+ * closed the connection because the client is idle for too long).
+ */
+ sourceFound = seekToBlockSource(pos);
+ } else {
+ addToDeadNodes(currentNode);
+ sourceFound = seekToNewSource(pos);
+ }
+ if (!sourceFound) {
+ throw ioe;
+ }
+ retryCurrentNode = false;
+ }
+ }
+
+ protected synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException {
+ dfsClient.checkOpen();
+ if (closed.get()) {
+ throw new IOException("Stream closed");
+ }
+ Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
+ = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
+ failures = 0;
+ if (pos < getFileLength()) {
+ int retries = 2;
+ while (retries > 0) {
+ try {
+ // currentNode can be left as null if previous read had a checksum
+ // error on the same block. See HDFS-3067
+ if (pos > blockEnd || currentNode == null) {
+ currentNode = blockSeekTo(pos);
+ }
+ int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
+ synchronized(infoLock) {
+ if (locatedBlocks.isLastBlockComplete()) {
+ realLen = (int) Math.min(realLen,
+ locatedBlocks.getFileLength() - pos);
+ }
+ }
+ int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
+
+ if (result >= 0) {
+ pos += result;
+ } else {
+ // got a EOS from reader though we expect more data on it.
+ throw new IOException("Unexpected EOS from the reader");
+ }
+ if (dfsClient.stats != null) {
+ dfsClient.stats.incrementBytesRead(result);
+ }
+ return result;
+ } catch (ChecksumException ce) {
+ throw ce;
+ } catch (IOException e) {
+ if (retries == 1) {
+ DFSClient.LOG.warn("DFS Read", e);
+ }
+ blockEnd = -1;
+ if (currentNode != null) { addToDeadNodes(currentNode); }
+ if (--retries == 0) {
+ throw e;
+ }
+ } finally {
+ // Check if need to report block replicas corruption either read
+ // was successful or ChecksumException occured.
+ reportCheckSumFailure(corruptedBlockMap,
+ currentLocatedBlock.getLocations().length);
+ }
+ }
+ }
+ return -1;
+ }
+
+ /**
+ * Read the entire buffer.
+ */
+ @Override
+ public synchronized int read(final byte buf[], int off, int len) throws IOException {
+ ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
+ TraceScope scope =
+ dfsClient.getPathTraceScope("DFSInputStream#byteArrayRead", src);
+ try {
+ return readWithStrategy(byteArrayReader, off, len);
+ } finally {
+ scope.close();
+ }
+ }
+
+ @Override
+ public synchronized int read(final ByteBuffer buf) throws IOException {
+ ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
+ TraceScope scope =
+ dfsClient.getPathTraceScope("DFSInputStream#byteBufferRead", src);
+ try {
+ return readWithStrategy(byteBufferReader, 0, buf.remaining());
+ } finally {
+ scope.close();
+ }
+ }
+
+
+ /**
+ * Add corrupted block replica into map.
+ */
+ protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
+ Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
+ Set<DatanodeInfo> dnSet = null;
+ if((corruptedBlockMap.containsKey(blk))) {
+ dnSet = corruptedBlockMap.get(blk);
+ }else {
+ dnSet = new HashSet<DatanodeInfo>();
+ }
+ if (!dnSet.contains(node)) {
+ dnSet.add(node);
+ corruptedBlockMap.put(blk, dnSet);
+ }
+ }
+
+ private DNAddrPair chooseDataNode(LocatedBlock block,
+ Collection<DatanodeInfo> ignoredNodes) throws IOException {
+ while (true) {
+ DNAddrPair result = getBestNodeDNAddrPair(block, ignoredNodes);
+ if (result != null) {
+ return result;
+ } else {
+ String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
+ deadNodes, ignoredNodes);
+ String blockInfo = block.getBlock() + " file=" + src;
+ if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
+ String description = "Could not obtain block: " + blockInfo;
+ DFSClient.LOG.warn(description + errMsg
+ + ". Throwing a BlockMissingException");
+ throw new BlockMissingException(src, description,
+ block.getStartOffset());
+ }
+
+ DatanodeInfo[] nodes = block.getLocations();
+ if (nodes == null || nodes.length == 0) {
+ DFSClient.LOG.info("No node available for " + blockInfo);
+ }
+ DFSClient.LOG.info("Could not obtain " + block.getBlock()
+ + " from any node: " + errMsg
+ + ". Will get new block locations from namenode and retry...");
+ try {
+ // Introducing a random factor to the wait time before another retry.
+ // The wait time is dependent on # of failures and a random factor.
+ // At the first time of getting a BlockMissingException, the wait time
+ // is a random number between 0..3000 ms. If the first retry
+ // still fails, we will wait 3000 ms grace period before the 2nd retry.
+ // Also at the second retry, the waiting window is expanded to 6000 ms
+ // alleviating the request rate from the server. Similarly the 3rd retry
+ // will wait 6000ms grace period before retry and the waiting window is
+ // expanded to 9000ms.
+ final int timeWindow = dfsClient.getConf().getTimeWindow();
+ double waitTime = timeWindow * failures + // grace period for the last round of attempt
+ // expanding time window for each failure
+ timeWindow * (failures + 1) *
+ ThreadLocalRandom.current().nextDouble();
+ DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
+ Thread.sleep((long)waitTime);
+ } catch (InterruptedException iex) {
+ }
+ deadNodes.clear(); //2nd option is to remove only nodes[blockId]
+ openInfo(true);
+ block = refreshLocatedBlock(block);
+ failures++;
+ }
+ }
+ }
+
+ /**
+ * Get the best node from which to stream the data.
+ * @param block LocatedBlock, containing nodes in priority order.
+ * @param ignoredNodes Do not choose nodes in this array (may be null)
+ * @return The DNAddrPair of the best node. Null if no node can be chosen.
+ */
+ protected DNAddrPair getBestNodeDNAddrPair(LocatedBlock block,
+ Collection<DatanodeInfo> ignoredNodes) {
+ DatanodeInfo[] nodes = block.getLocations();
+ StorageType[] storageTypes = block.getStorageTypes();
+ DatanodeInfo chosenNode = null;
+ StorageType storageType = null;
+ if (nodes != null) {
+ for (int i = 0; i < nodes.length; i++) {
+ if (!deadNodes.containsKey(nodes[i])
+ && (ignoredNodes == null || !ignoredNodes.contains(nodes[i]))) {
+ chosenNode = nodes[i];
+ // Storage types are ordered to correspond with nodes, so use the same
+ // index to get storage type.
+ if (storageTypes != null && i < storageTypes.length) {
+ storageType = storageTypes[i];
+ }
+ break;
+ }
+ }
+ }
+ if (chosenNode == null) {
+ DFSClient.LOG.warn("No live nodes contain block " + block.getBlock() +
+ " after checking nodes = " + Arrays.toString(nodes) +
+ ", ignoredNodes = " + ignoredNodes);
+ return null;
+ }
+ final String dnAddr =
+ chosenNode.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
+ }
+ InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
+ return new DNAddrPair(chosenNode, targetAddr, storageType);
+ }
+
+ private static String getBestNodeDNAddrPairErrorString(
+ DatanodeInfo nodes[], AbstractMap<DatanodeInfo,
+ DatanodeInfo> deadNodes, Collection<DatanodeInfo> ignoredNodes) {
+ StringBuilder errMsgr = new StringBuilder(
+ " No live nodes contain current block ");
+ errMsgr.append("Block locations:");
+ for (DatanodeInfo datanode : nodes) {
+ errMsgr.append(" ");
+ errMsgr.append(datanode.toString());
+ }
+ errMsgr.append(" Dead nodes: ");
+ for (DatanodeInfo datanode : deadNodes.keySet()) {
+ errMsgr.append(" ");
+ errMsgr.append(datanode.toString());
+ }
+ if (ignoredNodes != null) {
+ errMsgr.append(" Ignored nodes: ");
+ for (DatanodeInfo datanode : ignoredNodes) {
+ errMsgr.append(" ");
+ errMsgr.append(datanode.toString());
+ }
+ }
+ return errMsgr.toString();
+ }
+
+ protected void fetchBlockByteRange(LocatedBlock block, long start, long end,
+ byte[] buf, int offset,
+ Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+ throws IOException {
+ block = refreshLocatedBlock(block);
+ while (true) {
+ DNAddrPair addressPair = chooseDataNode(block, null);
+ try {
+ actualGetFromOneDataNode(addressPair, block, start, end,
+ buf, offset, corruptedBlockMap);
+ return;
+ } catch (IOException e) {
+ // Ignore. Already processed inside the function.
+ // Loop through to try the next node.
+ }
+ }
+ }
+
+ private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
+ final LocatedBlock block, final long start, final long end,
+ final ByteBuffer bb,
+ final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
+ final int hedgedReadId) {
+ final Span parentSpan = Trace.currentSpan();
+ return new Callable<ByteBuffer>() {
+ @Override
+ public ByteBuffer call() throws Exception {
+ byte[] buf = bb.array();
+ int offset = bb.position();
+ TraceScope scope =
+ Trace.startSpan("hedgedRead" + hedgedReadId, parentSpan);
+ try {
+ actualGetFromOneDataNode(datanode, block, start, end, buf,
+ offset, corruptedBlockMap);
+ return bb;
+ } finally {
+ scope.close();
+ }
+ }
+ };
+ }
+
+ /**
+ * Used when reading contiguous blocks
+ */
+ private void actualGetFromOneDataNode(final DNAddrPair datanode,
+ LocatedBlock block, final long start, final long end, byte[] buf,
+ int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+ throws IOException {
+ final int length = (int) (end - start + 1);
+ actualGetFromOneDataNode(datanode, block, start, end, buf,
+ new int[]{offset}, new int[]{length}, corruptedBlockMap);
+ }
+
+ /**
+ * Read data from one DataNode.
+ * @param datanode the datanode from which to read data
+ * @param block the located block containing the requested data
+ * @param startInBlk the startInBlk offset of the block
+ * @param endInBlk the endInBlk offset of the block
+ * @param buf the given byte array into which the data is read
+ * @param offsets the data may be read into multiple segments of the buf
+ * (when reading a striped block). this array indicates the
+ * offset of each buf segment.
+ * @param lengths the length of each buf segment
+ * @param corruptedBlockMap map recording list of datanodes with corrupted
+ * block replica
+ */
+ void actualGetFromOneDataNode(final DNAddrPair datanode,
+ LocatedBlock block, final long startInBlk, final long endInBlk,
+ byte[] buf, int[] offsets, int[] lengths,
+ Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+ throws IOException {
+ DFSClientFaultInjector.get().startFetchFromDatanode();
+ int refetchToken = 1; // only need to get a new access token once
+ int refetchEncryptionKey = 1; // only need to get a new encryption key once
+ final int len = (int) (endInBlk - startInBlk + 1);
+ checkReadPortions(offsets, lengths, len);
+
+ while (true) {
+ // cached block locations may have been updated by chooseDataNode()
+ // or fetchBlockAt(). Always get the latest list of locations at the
+ // start of the loop.
+ block = refreshLocatedBlock(block);
+ BlockReader reader = null;
+ try {
+ DFSClientFaultInjector.get().fetchFromDatanodeException();
+ reader = getBlockReader(block, startInBlk, len, datanode.addr,
+ datanode.storageType, datanode.info);
+ for (int i = 0; i < offsets.length; i++) {
+ int nread = reader.readAll(buf, offsets[i], lengths[i]);
+ updateReadStatistics(readStatistics, nread, reader);
+ if (nread != lengths[i]) {
+ throw new IOException("truncated return from reader.read(): " +
+ "excpected " + lengths[i] + ", got " + nread);
+ }
+ }
+ DFSClientFaultInjector.get().readFromDatanodeDelay();
+ return;
+ } catch (ChecksumException e) {
+ String msg = "fetchBlockByteRange(). Got a checksum exception for "
+ + src + " at " + block.getBlock() + ":" + e.getPos() + " from "
+ + datanode.info;
+ DFSClient.LOG.warn(msg);
+ // we want to remember what we have tried
+ addIntoCorruptedBlockMap(block.getBlock(), datanode.info,
+ corruptedBlockMap);
+ addToDeadNodes(datanode.info);
+ throw new IOException(msg);
+ } catch (IOException e) {
+ if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
+ DFSClient.LOG.info("Will fetch a new encryption key and retry, "
+ + "encryption key was invalid when connecting to " + datanode.addr
+ + " : " + e);
+ // The encryption key used is invalid.
+ refetchEncryptionKey--;
+ dfsClient.clearDataEncryptionKey();
+ } else if (refetchToken > 0 && tokenRefetchNeeded(e, datanode.addr)) {
+ refetchToken--;
+ try {
+ fetchBlockAt(block.getStartOffset());
+ } catch (IOException fbae) {
+ // ignore IOE, since we can retry it later in a loop
+ }
+ } else {
+ String msg = "Failed to connect to " + datanode.addr + " for file "
+ + src + " for block " + block.getBlock() + ":" + e;
+ DFSClient.LOG.warn("Connection failure: " + msg, e);
+ addToDeadNodes(datanode.info);
+ throw new IOException(msg);
+ }
+ } finally {
+ if (reader != null) {
+ reader.close();
+ }
+ }
+ }
+ }
+
+ /**
+ * Refresh cached block locations.
+ * @param block The currently cached block locations
+ * @return Refreshed block locations
+ * @throws IOException
+ */
+ protected LocatedBlock refreshLocatedBlock(LocatedBlock block)
+ throws IOException {
+ return getBlockAt(block.getStartOffset());
+ }
+
+ /**
+ * This method verifies that the read portions are valid and do not overlap
+ * with each other.
+ */
+ private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) {
+ Preconditions.checkArgument(offsets.length == lengths.length && offsets.length > 0);
+ int sum = 0;
+ for (int i = 0; i < lengths.length; i++) {
+ if (i > 0) {
+ int gap = offsets[i] - offsets[i - 1];
+ // make sure read portions do not overlap with each other
+ Preconditions.checkArgument(gap >= lengths[i - 1]);
+ }
+ sum += lengths[i];
+ }
+ Preconditions.checkArgument(sum == totalLen);
+ }
+
+ /**
+ * Like {@link #fetchBlockByteRange}except we start up a second, parallel,
+ * 'hedged' read if the first read is taking longer than configured amount of
+ * time. We then wait on which ever read returns first.
+ */
+ private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
+ long end, byte[] buf, int offset,
+ Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+ throws IOException {
+ final DfsClientConf conf = dfsClient.getConf();
+ ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
+ CompletionService<ByteBuffer> hedgedService =
+ new ExecutorCompletionService<ByteBuffer>(
+ dfsClient.getHedgedReadsThreadPool());
+ ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
+ ByteBuffer bb = null;
+ int len = (int) (end - start + 1);
+ int hedgedReadId = 0;
+ block = refreshLocatedBlock(block);
+ while (true) {
+ // see HDFS-6591, this metric is used to verify/catch unnecessary loops
+ hedgedReadOpsLoopNumForTesting++;
+ DNAddrPair chosenNode = null;
+ // there is no request already executing.
+ if (futures.isEmpty()) {
+ // chooseDataNode is a commitment. If no node, we go to
+ // the NN to reget block locations. Only go here on first read.
+ chosenNode = chooseDataNode(block, ignored);
+ bb = ByteBuffer.wrap(buf, offset, len);
+ Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
+ chosenNode, block, start, end, bb,
+ corruptedBlockMap, hedgedReadId++);
+ Future<ByteBuffer> firstRequest = hedgedService
+ .submit(getFromDataNodeCallable);
+ futures.add(firstRequest);
+ try {
+ Future<ByteBuffer> future = hedgedService.poll(
+ conf.getHedgedReadThresholdMillis(), TimeUnit.MILLISECONDS);
+ if (future != null) {
+ future.get();
+ return;
+ }
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("Waited " + conf.getHedgedReadThresholdMillis()
+ + "ms to read from " + chosenNode.info
+ + "; spawning hedged read");
+ }
+ // Ignore this node on next go around.
+ ignored.add(chosenNode.info);
+ dfsClient.getHedgedReadMetrics().incHedgedReadOps();
+ continue; // no need to refresh block locations
+ } catch (InterruptedException e) {
+ // Ignore
+ } catch (ExecutionException e) {
+ // Ignore already logged in the call.
+ }
+ } else {
+ // We are starting up a 'hedged' read. We have a read already
+ // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
+ // If no nodes to do hedged reads against, pass.
+ try {
+ chosenNode = getBestNodeDNAddrPair(block, ignored);
+ if (chosenNode == null) {
+ chosenNode = chooseDataNode(block, ignored);
+ }
+ bb = ByteBuffer.allocate(len);
+ Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
+ chosenNode, block, start, end, bb,
+ corruptedBlockMap, hedgedReadId++);
+ Future<ByteBuffer> oneMoreRequest = hedgedService
+ .submit(getFromDataNodeCallable);
+ futures.add(oneMoreRequest);
+ } catch (IOException ioe) {
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("Failed getting node for hedged read: "
+ + ioe.getMessage());
+ }
+ }
+ // if not succeeded. Submit callables for each datanode in a loop, wait
+ // for a fixed interval and get the result from the fastest one.
+ try {
+ ByteBuffer result = getFirstToComplete(hedgedService, futures);
+ // cancel the rest.
+ cancelAll(futures);
+ if (result.array() != buf) { // compare the array pointers
+ dfsClient.getHedgedReadMetrics().incHedgedReadWins();
+ System.arraycopy(result.array(), result.position(), buf, offset,
+ len);
+ } else {
+ dfsClient.getHedgedReadMetrics().incHedgedReadOps();
+ }
+ return;
+ } catch (InterruptedException ie) {
+ // Ignore and retry
+ }
+ // We got here if exception. Ignore this node on next go around IFF
+ // we found a chosenNode to hedge read against.
+ if (chosenNode != null && chosenNode.info != null) {
+ ignored.add(chosenNode.info);
+ }
+ }
+ }
+ }
+
+ @VisibleForTesting
+ public long getHedgedReadOpsLoopNumForTesting() {
+ return hedgedReadOpsLoopNumForTesting;
+ }
+
+ private ByteBuffer getFirstToComplete(
+ CompletionService<ByteBuffer> hedgedService,
+ ArrayList<Future<ByteBuffer>> futures) throws InterruptedException {
+ if (futures.isEmpty()) {
+ throw new InterruptedException("let's retry");
+ }
+ Future<ByteBuffer> future = null;
+ try {
+ future = hedgedService.take();
+ ByteBuffer bb = future.get();
+ futures.remove(future);
+ return bb;
+ } catch (ExecutionException e) {
+ // already logged in the Callable
+ futures.remove(future);
+ } catch (CancellationException ce) {
+ // already logged in the Callable
+ futures.remove(future);
+ }
+
+ throw new InterruptedException("let's retry");
+ }
+
+ private void cancelAll(List<Future<ByteBuffer>> futures) {
+ for (Future<ByteBuffer> future : futures) {
+ // Unfortunately, hdfs reads do not take kindly to interruption.
+ // Threads return a variety of interrupted-type exceptions but
+ // also complaints about invalid pbs -- likely because read
+ // is interrupted before gets whole pb. Also verbose WARN
+ // logging. So, for now, do not interrupt running read.
+ future.cancel(false);
+ }
+ }
+
+ /**
+ * Should the block access token be refetched on an exception
+ *
+ * @param ex Exception received
+ * @param targetAddr Target datanode address from where exception was received
+ * @return true if block access token has expired or invalid and it should be
+ * refetched
+ */
+ protected static boolean tokenRefetchNeeded(IOException ex,
+ InetSocketAddress targetAddr) {
+ /*
+ * Get a new access token and retry. Retry is needed in 2 cases. 1)
+ * When both NN and DN re-started while DFSClient holding a cached
+ * access token. 2) In the case that NN fails to update its
+ * access key at pre-set interval (by a wide margin) and
+ * subsequently restarts. In this case, DN re-registers itself with
+ * NN and receives a new access key, but DN will delete the old
+ * access key from its memory since it's considered expired based on
+ * the estimated expiration date.
+ */
+ if (ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken) {
+ DFSClient.LOG.info("Access token was invalid when connecting to "
+ + targetAddr + " : " + ex);
+ return true;
+ }
+ return false;
+ }
+
+ /**
+ * Read bytes starting from the specified position.
+ *
+ * @param position start read from this position
+ * @param buffer read buffer
+ * @param offset offset into buffer
+ * @param length number of bytes to read
+ *
+ * @return actual number of bytes read
+ */
+ @Override
+ public int read(long position, byte[] buffer, int offset, int length)
+ throws IOException {
+ TraceScope scope =
+ dfsClient.getPathTraceScope("DFSInputStream#byteArrayPread", src);
+ try {
+ return pread(position, buffer, offset, length);
+ } finally {
+ scope.close();
+ }
+ }
+
+ private int pread(long position, byte[] buffer, int offset, int length)
+ throws IOException {
+ // sanity checks
+ dfsClient.checkOpen();
+ if (closed.get()) {
+ throw new IOException("Stream closed");
+ }
+ failures = 0;
+ long filelen = getFileLength();
+ if ((position < 0) || (position >= filelen)) {
+ return -1;
+ }
+ int realLen = length;
+ if ((position + length) > filelen) {
+ realLen = (int)(filelen - position);
+ }
+
+ // determine the block and byte range within the block
+ // corresponding to position and realLen
+ List<LocatedBlock> blockRange = getBlockRange(position, realLen);
+ int remaining = realLen;
+ Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
+ = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
+ for (LocatedBlock blk : blockRange) {
+ long targetStart = position - blk.getStartOffset();
+ long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
+ try {
+ if (dfsClient.isHedgedReadsEnabled()) {
+ hedgedFetchBlockByteRange(blk, targetStart,
+ targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
+ } else {
+ fetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1,
+ buffer, offset, corruptedBlockMap);
+ }
+ } finally {
+ // Check and report if any block replicas are corrupted.
+ // BlockMissingException may be caught if all block replicas are
+ // corrupted.
+ reportCheckSumFailure(corruptedBlockMap, blk.getLocations().length);
+ }
+
+ remaining -= bytesToRead;
+ position += bytesToRead;
+ offset += bytesToRead;
+ }
+ assert remaining == 0 : "Wrong number of bytes read.";
+ if (dfsClient.stats != null) {
+ dfsClient.stats.incrementBytesRead(realLen);
+ }
+ return realLen;
+ }
+
+ /**
+ * DFSInputStream reports checksum failure.
+ * Case I : client has tried multiple data nodes and at least one of the
+ * attempts has succeeded. We report the other failures as corrupted block to
+ * namenode.
+ * Case II: client has tried out all data nodes, but all failed. We
+ * only report if the total number of replica is 1. We do not
+ * report otherwise since this maybe due to the client is a handicapped client
+ * (who can not read).
+ * @param corruptedBlockMap map of corrupted blocks
+ * @param dataNodeCount number of data nodes who contains the block replicas
+ */
+ protected void reportCheckSumFailure(
+ Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
+ int dataNodeCount) {
+ if (corruptedBlockMap.isEmpty()) {
+ return;
+ }
+ Iterator<Entry<ExtendedBlock, Set<DatanodeInfo>>> it = corruptedBlockMap
+ .entrySet().iterator();
+ Entry<ExtendedBlock, Set<DatanodeInfo>> entry = it.next();
+ ExtendedBlock blk = entry.getKey();
+ Set<DatanodeInfo> dnSet = entry.getValue();
+ if (((dnSet.size() < dataNodeCount) && (dnSet.size() > 0))
+ || ((dataNodeCount == 1) && (dnSet.size() == dataNodeCount))) {
+ DatanodeInfo[] locs = new DatanodeInfo[dnSet.size()];
+ int i = 0;
+ for (DatanodeInfo dn:dnSet) {
+ locs[i++] = dn;
+ }
+ LocatedBlock [] lblocks = { new LocatedBlock(blk, locs) };
+ dfsClient.reportChecksumFailure(src, lblocks);
+ }
+ corruptedBlockMap.clear();
+ }
+
+ @Override
+ public long skip(long n) throws IOException {
+ if ( n > 0 ) {
+ long curPos = getPos();
+ long fileLen = getFileLength();
+ if( n+curPos > fileLen ) {
+ n = fileLen - curPos;
+ }
+ seek(curPos+n);
+ return n;
+ }
+ return n < 0 ? -1 : 0;
+ }
+
+ /**
+ * Seek to a new arbitrary location
+ */
+ @Override
+ public synchronized void seek(long targetPos) throws IOException {
+ if (targetPos > getFileLength()) {
+ throw new EOFException("Cannot seek after EOF");
+ }
+ if (targetPos < 0) {
+ throw new EOFException("Cannot seek to negative offset");
+ }
+ if (closed.get()) {
+ throw new IOException("Stream is closed!");
+ }
+ boolean done = false;
+ if (pos <= targetPos && targetPos <= blockEnd) {
+ //
+ // If this seek is to a positive position in the current
+ // block, and this piece of data might already be lying in
+ // the TCP buffer, then just eat up the intervening data.
+ //
+ int diff = (int)(targetPos - pos);
+ if (diff <= blockReader.available()) {
+ try {
+ pos += blockReader.skip(diff);
+ if (pos == targetPos) {
+ done = true;
+ } else {
+ // The range was already checked. If the block reader returns
+ // something unexpected instead of throwing an exception, it is
+ // most likely a bug.
+ String errMsg = "BlockReader failed to seek to " +
+ targetPos + ". Instead, it seeked to " + pos + ".";
+ DFSClient.LOG.warn(errMsg);
+ throw new IOException(errMsg);
+ }
+ } catch (IOException e) {//make following read to retry
+ if(DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("Exception while seek to " + targetPos
+ + " from " + getCurrentBlock() + " of " + src + " from "
+ + currentNode, e);
+ }
+ }
+ }
+ }
+ if (!done) {
+ pos = targetPos;
+ blockEnd = -1;
+ }
+ }
+
+ /**
+ * Same as {@link #seekToNewSource(long)} except that it does not exclude
+ * the current datanode and might connect to the same node.
+ */
+ private boolean seekToBlockSource(long targetPos)
+ throws IOException {
+ currentNode = blockSeekTo(targetPos);
+ return true;
+ }
+
+ /**
+ * Seek to given position on a node other than the current node. If
+ * a node other than the current node is found, then returns true.
+ * If another node could not be found, then returns false.
+ */
+ @Override
+ public synchronized boolean seekToNewSource(long targetPos) throws IOException {
+ if (currentNode == null) {
+ return seekToBlockSource(targetPos);
+ }
+ boolean markedDead = deadNodes.containsKey(currentNode);
+ addToDeadNodes(currentNode);
+ DatanodeInfo oldNode = currentNode;
+ DatanodeInfo newNode = blockSeekTo(targetPos);
+ if (!markedDead) {
+ /* remove it from deadNodes. blockSeekTo could have cleared
+ * deadNodes and added currentNode again. Thats ok. */
+ deadNodes.remove(oldNode);
+ }
+ if (!oldNode.getDatanodeUuid().equals(newNode.getDatanodeUuid())) {
+ currentNode = newNode;
+ return true;
+ } else {
+ return false;
+ }
+ }
+
+ /**
+ */
+ @Override
+ public synchronized long getPos() {
+ return pos;
+ }
+
+ /** Return the size of the remaining available bytes
+ * if the size is less than or equal to {@link Integer#MAX_VALUE},
+ * otherwise, return {@link Integer#MAX_VALUE}.
+ */
+ @Override
+ public synchronized int available() throws IOException {
+ if (closed.get()) {
+ throw new IOException("Stream closed");
+ }
+
+ final long remaining = getFileLength() - pos;
+ return remaining <= Integer.MAX_VALUE? (int)remaining: Integer.MAX_VALUE;
+ }
+
+ /**
+ * We definitely don't support marks
+ */
+ @Override
+ public boolean markSupported() {
+ return false;
+ }
+ @Override
+ public void mark(int readLimit) {
+ }
+ @Override
+ public void reset() throws IOException {
+ throw new IOException("Mark/reset not supported");
+ }
+
+ /** Utility class to encapsulate data node info and its address. */
+ static final class DNAddrPair {
+ final DatanodeInfo info;
+ final InetSocketAddress addr;
+ final StorageType storageType;
+
+ DNAddrPair(DatanodeInfo info, InetSocketAddress addr,
+ StorageType storageType) {
+ this.info = info;
+ this.addr = addr;
+ this.storageType = storageType;
+ }
+ }
+
+ /**
+ * Get statistics about the reads which this DFSInputStream has done.
+ */
+ public ReadStatistics getReadStatistics() {
+ synchronized(infoLock) {
+ return new ReadStatistics(readStatistics);
+ }
+ }
+
+ /**
+ * Clear statistics about the reads which this DFSInputStream has done.
+ */
+ public void clearReadStatistics() {
+ synchronized(infoLock) {
+ readStatistics.clear();
+ }
+ }
+
+ public FileEncryptionInfo getFileEncryptionInfo() {
+ synchronized(infoLock) {
+ return fileEncryptionInfo;
+ }
+ }
+
+ protected void closeCurrentBlockReaders() {
+ if (blockReader == null) return;
+ // Close the current block reader so that the new caching settings can
+ // take effect immediately.
+ try {
+ blockReader.close();
+ } catch (IOException e) {
+ DFSClient.LOG.error("error closing blockReader", e);
+ }
+ blockReader = null;
+ blockEnd = -1;
+ }
+
+ @Override
+ public synchronized void setReadahead(Long readahead)
+ throws IOException {
+ synchronized (infoLock) {
+ this.cachingStrategy =
+ new CachingStrategy.Builder(this.cachingStrategy).setReadahead(readahead).build();
+ }
+ closeCurrentBlockReaders();
+ }
+
+ @Override
+ public synchronized void setDropBehind(Boolean dropBehind)
+ throws IOException {
+ synchronized (infoLock) {
+ this.cachingStrategy =
+ new CachingStrategy.Builder(this.cachingStrategy).setDropBehind(dropBehind).build();
+ }
+ closeCurrentBlockReaders();
+ }
+
+ /**
+ * The immutable empty buffer we return when we reach EOF when doing a
+ * zero-copy read.
+ */
+ private static final ByteBuffer EMPTY_BUFFER =
+ ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
+
+ @Override
+ public synchronized ByteBuffer read(ByteBufferPool bufferPool,
+ int maxLength, EnumSet<ReadOption> opts)
+ throws IOException, UnsupportedOperationException {
+ if (maxLength == 0) {
+ return EMPTY_BUFFER;
+ } else if (maxLength < 0) {
+ throw new IllegalArgumentException("can't read a negative " +
+ "number of bytes.");
+ }
+ if ((blockReader == null) || (blockEnd == -1)) {
+ if (pos >= getFileLength()) {
+ return null;
+ }
+ /*
+ * If we don't have a blockReader, or the one we have has no more bytes
+ * left to read, we call seekToBlockSource to get a new blockReader and
+ * recalculate blockEnd. Note that we assume we're not at EOF here
+ * (we check this above).
+ */
+ if ((!seekToBlockSource(pos)) || (blockReader == null)) {
+ throw new IOException("failed to allocate new BlockReader " +
+ "at position " + pos);
+ }
+ }
+ ByteBuffer buffer = null;
+ if (dfsClient.getConf().getShortCircuitConf().isShortCircuitMmapEnabled()) {
+ buffer = tryReadZeroCopy(maxLength, opts);
+ }
+ if (buffer != null) {
+ return buffer;
+ }
+ buffer = ByteBufferUtil.fallbackRead(this, bufferPool, maxLength);
+ if (buffer != null) {
+ getExtendedReadBuffers().put(buffer, bufferPool);
+ }
+ return buffer;
+ }
+
+ private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
+ EnumSet<ReadOption> opts) throws IOException {
+ // Copy 'pos' and 'blockEnd' to local variables to make it easier for the
+ // JVM to optimize this function.
+ final long curPos = pos;
+ final long curEnd = blockEnd;
+ final long blockStartInFile = currentLocatedBlock.getStartOffset();
+ final long blockPos = curPos - blockStartInFile;
+
+ // Shorten this read if the end of the block is nearby.
+ long length63;
+ if ((curPos + maxLength) <= (curEnd + 1)) {
+ length63 = maxLength;
+ } else {
+ length63 = 1 + curEnd - curPos;
+ if (length63 <= 0) {
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
+ curPos + " of " + src + "; " + length63 + " bytes left in block. " +
+ "blockPos=" + blockPos + "; curPos=" + curPos +
+ "; curEnd=" + curEnd);
+ }
+ return null;
+ }
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("Reducing read length from " + maxLength +
+ " to " + length63 + " to avoid going more than one byte " +
+ "past the end of the block. blockPos=" + blockPos +
+ "; curPos=" + curPos + "; curEnd=" + curEnd);
+ }
+ }
+ // Make sure that don't go beyond 31-bit offsets in the MappedByteBuffer.
+ int length;
+ if (blockPos + length63 <= Integer.MAX_VALUE) {
+ length = (int)length63;
+ } else {
+ long length31 = Integer.MAX_VALUE - blockPos;
+ if (length31 <= 0) {
+ // Java ByteBuffers can't be longer than 2 GB, because they use
+ // 4-byte signed integers to represent capacity, etc.
+ // So we can't mmap the parts of the block higher than the 2 GB offset.
+ // FIXME: we could work around this with multiple memory maps.
+ // See HDFS-5101.
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
+ curPos + " of " + src + "; 31-bit MappedByteBuffer limit " +
+ "exceeded. blockPos=" + blockPos + ", curEnd=" + curEnd);
+ }
+ return null;
+ }
+ length = (int)length31;
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("Reducing read length from " + maxLength +
+ " to " + length + " to avoid 31-bit limit. " +
+ "blockPos=" + blockPos + "; curPos=" + curPos +
+ "; curEnd=" + curEnd);
+ }
+ }
+ final ClientMmap clientMmap = blockReader.getClientMmap(opts);
+ if (clientMmap == null) {
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
+ curPos + " of " + src + "; BlockReader#getClientMmap returned " +
+ "null.");
+ }
+ return null;
+ }
+ boolean success = false;
+ ByteBuffer buffer;
+ try {
+ seek(curPos + length);
+ buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
+ buffer.position((int)blockPos);
+ buffer.limit((int)(blockPos + length));
+ getExtendedReadBuffers().put(buffer, clientMmap);
+ synchronized (infoLock) {
+ readStatistics.addZeroCopyBytes(length);
+ }
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("readZeroCopy read " + length +
+ " bytes from offset " + curPos + " via the zero-copy read " +
+ "path. blockEnd = " + blockEnd);
+ }
+ success = true;
+ } finally {
+ if (!success) {
+ IOUtils.closeQuietly(clientMmap);
+ }
+ }
+ return buffer;
+ }
+
+ @Override
+ public synchronized void releaseBuffer(ByteBuffer buffer) {
+ if (buffer == EMPTY_BUFFER) return;
+ Object val = getExtendedReadBuffers().remove(buffer);
+ if (val == null) {
+ throw new IllegalArgumentException("tried to release a buffer " +
+ "that was not created by this stream, " + buffer);
+ }
+ if (val instanceof ClientMmap) {
+ IOUtils.closeQuietly((ClientMmap)val);
+ } else if (val instanceof ByteBufferPool) {
+ ((ByteBufferPool)val).putBuffer(buffer);
+ }
+ }
+
+ @Override
+ public synchronized void unbuffer() {
+ closeCurrentBlockReaders();
+ }
+}
[05/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and
related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
deleted file mode 100644
index 3b09953..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ /dev/null
@@ -1,3200 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_READAHEAD;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT;
-import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT;
-
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.lang.reflect.Proxy;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.Socket;
-import java.net.SocketAddress;
-import java.net.URI;
-import java.net.UnknownHostException;
-import java.security.GeneralSecurityException;
-import java.util.ArrayList;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import javax.net.SocketFactory;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.HadoopIllegalArgumentException;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.CipherSuite;
-import org.apache.hadoop.crypto.CryptoCodec;
-import org.apache.hadoop.crypto.CryptoInputStream;
-import org.apache.hadoop.crypto.CryptoOutputStream;
-import org.apache.hadoop.crypto.CryptoProtocolVersion;
-import org.apache.hadoop.crypto.key.KeyProvider;
-import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
-import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
-import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.BlockStorageLocation;
-import org.apache.hadoop.fs.CacheFlag;
-import org.apache.hadoop.fs.ContentSummary;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FileAlreadyExistsException;
-import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.FsServerDefaults;
-import org.apache.hadoop.fs.FsStatus;
-import org.apache.hadoop.fs.HdfsBlockLocation;
-import org.apache.hadoop.fs.InvalidPathException;
-import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
-import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
-import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
-import org.apache.hadoop.fs.Options;
-import org.apache.hadoop.fs.Options.ChecksumOpt;
-import org.apache.hadoop.fs.ParentNotDirectoryException;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.RemoteIterator;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.fs.VolumeId;
-import org.apache.hadoop.fs.XAttr;
-import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.fs.permission.AclEntry;
-import org.apache.hadoop.fs.permission.AclStatus;
-import org.apache.hadoop.fs.permission.FsAction;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
-import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
-import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
-import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
-import org.apache.hadoop.hdfs.net.Peer;
-import org.apache.hadoop.hdfs.protocol.AclException;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
-import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
-import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
-import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
-import org.apache.hadoop.hdfs.protocol.CachePoolIterator;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
-import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.DatanodeID;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.DirectoryListing;
-import org.apache.hadoop.hdfs.protocol.EncryptionZone;
-import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
-import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
-import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
-import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
-import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
-import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
-import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
-import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
-import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
-import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
-import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
-import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
-import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
-import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
-import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
-import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
-import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
-import org.apache.hadoop.io.DataOutputBuffer;
-import org.apache.hadoop.io.EnumSetWritable;
-import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.io.MD5Hash;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.ipc.RpcInvocationHandler;
-import org.apache.hadoop.net.DNS;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenRenewer;
-import org.apache.hadoop.tracing.SpanReceiverHost;
-import org.apache.hadoop.tracing.TraceUtils;
-import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.DataChecksum.Type;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.Time;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.SamplerBuilder;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.net.InetAddresses;
-
-/********************************************************
- * DFSClient can connect to a Hadoop Filesystem and
- * perform basic file tasks. It uses the ClientProtocol
- * to communicate with a NameNode daemon, and connects
- * directly to DataNodes to read/write block data.
- *
- * Hadoop DFS users should obtain an instance of
- * DistributedFileSystem, which uses DFSClient to handle
- * filesystem tasks.
- *
- ********************************************************/
-@InterfaceAudience.Private
-public class DFSClient implements java.io.Closeable, RemotePeerFactory,
- DataEncryptionKeyFactory {
- public static final Log LOG = LogFactory.getLog(DFSClient.class);
- public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
-
- private final Configuration conf;
- private final DfsClientConf dfsClientConf;
- final ClientProtocol namenode;
- /* The service used for delegation tokens */
- private Text dtService;
-
- final UserGroupInformation ugi;
- volatile boolean clientRunning = true;
- volatile long lastLeaseRenewal;
- private volatile FsServerDefaults serverDefaults;
- private volatile long serverDefaultsLastUpdate;
- final String clientName;
- final SocketFactory socketFactory;
- final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
- final FileSystem.Statistics stats;
- private final String authority;
- private final Random r = new Random();
- private SocketAddress[] localInterfaceAddrs;
- private DataEncryptionKey encryptionKey;
- final SaslDataTransferClient saslClient;
- private final CachingStrategy defaultReadCachingStrategy;
- private final CachingStrategy defaultWriteCachingStrategy;
- private final ClientContext clientContext;
-
- private static final DFSHedgedReadMetrics HEDGED_READ_METRIC =
- new DFSHedgedReadMetrics();
- private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
- private final Sampler<?> traceSampler;
- private final int smallBufferSize;
-
- public DfsClientConf getConf() {
- return dfsClientConf;
- }
-
- Configuration getConfiguration() {
- return conf;
- }
-
- /**
- * A map from file names to {@link DFSOutputStream} objects
- * that are currently being written by this client.
- * Note that a file can only be written by a single client.
- */
- private final Map<Long, DFSOutputStream> filesBeingWritten
- = new HashMap<Long, DFSOutputStream>();
-
- /**
- * Same as this(NameNode.getNNAddress(conf), conf);
- * @see #DFSClient(InetSocketAddress, Configuration)
- * @deprecated Deprecated at 0.21
- */
- @Deprecated
- public DFSClient(Configuration conf) throws IOException {
- this(DFSUtilClient.getNNAddress(conf), conf);
- }
-
- public DFSClient(InetSocketAddress address, Configuration conf) throws IOException {
- this(DFSUtilClient.getNNUri(address), conf);
- }
-
- /**
- * Same as this(nameNodeUri, conf, null);
- * @see #DFSClient(URI, Configuration, FileSystem.Statistics)
- */
- public DFSClient(URI nameNodeUri, Configuration conf
- ) throws IOException {
- this(nameNodeUri, conf, null);
- }
-
- /**
- * Same as this(nameNodeUri, null, conf, stats);
- * @see #DFSClient(URI, ClientProtocol, Configuration, FileSystem.Statistics)
- */
- public DFSClient(URI nameNodeUri, Configuration conf,
- FileSystem.Statistics stats)
- throws IOException {
- this(nameNodeUri, null, conf, stats);
- }
-
- /**
- * Create a new DFSClient connected to the given nameNodeUri or rpcNamenode.
- * If HA is enabled and a positive value is set for
- * {@link HdfsClientConfigKeys#DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY}
- * in the configuration, the DFSClient will use
- * {@link LossyRetryInvocationHandler} as its RetryInvocationHandler.
- * Otherwise one of nameNodeUri or rpcNamenode must be null.
- */
- @VisibleForTesting
- public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
- Configuration conf, FileSystem.Statistics stats)
- throws IOException {
- SpanReceiverHost.get(conf, HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX);
- traceSampler = new SamplerBuilder(TraceUtils.
- wrapHadoopConf(HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX, conf))
- .build();
- // Copy only the required DFSClient configuration
- this.dfsClientConf = new DfsClientConf(conf);
- this.conf = conf;
- this.stats = stats;
- this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
- this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
- this.smallBufferSize = DFSUtil.getSmallBufferSize(conf);
-
- this.ugi = UserGroupInformation.getCurrentUser();
-
- this.authority = nameNodeUri == null? "null": nameNodeUri.getAuthority();
- this.clientName = "DFSClient_" + dfsClientConf.getTaskId() + "_" +
- ThreadLocalRandom.current().nextInt() + "_" +
- Thread.currentThread().getId();
- int numResponseToDrop = conf.getInt(
- HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
- HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
- ProxyAndInfo<ClientProtocol> proxyInfo = null;
- AtomicBoolean nnFallbackToSimpleAuth = new AtomicBoolean(false);
- if (numResponseToDrop > 0) {
- // This case is used for testing.
- LOG.warn(HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
- + " is set to " + numResponseToDrop
- + ", this hacked client will proactively drop responses");
- proxyInfo = NameNodeProxiesClient.createProxyWithLossyRetryHandler(conf,
- nameNodeUri, ClientProtocol.class, numResponseToDrop,
- nnFallbackToSimpleAuth);
- }
-
- if (proxyInfo != null) {
- this.dtService = proxyInfo.getDelegationTokenService();
- this.namenode = proxyInfo.getProxy();
- } else if (rpcNamenode != null) {
- // This case is used for testing.
- Preconditions.checkArgument(nameNodeUri == null);
- this.namenode = rpcNamenode;
- dtService = null;
- } else {
- Preconditions.checkArgument(nameNodeUri != null,
- "null URI");
- proxyInfo = NameNodeProxiesClient.createProxyWithClientProtocol(conf,
- nameNodeUri, nnFallbackToSimpleAuth);
- this.dtService = proxyInfo.getDelegationTokenService();
- this.namenode = proxyInfo.getProxy();
- }
-
- String localInterfaces[] =
- conf.getTrimmedStrings(HdfsClientConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
- localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
- if (LOG.isDebugEnabled() && 0 != localInterfaces.length) {
- LOG.debug("Using local interfaces [" +
- Joiner.on(',').join(localInterfaces)+ "] with addresses [" +
- Joiner.on(',').join(localInterfaceAddrs) + "]");
- }
-
- Boolean readDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_READS) == null) ?
- null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_READS, false);
- Long readahead = (conf.get(DFS_CLIENT_CACHE_READAHEAD) == null) ?
- null : conf.getLong(DFS_CLIENT_CACHE_READAHEAD, 0);
- Boolean writeDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES) == null) ?
- null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES, false);
- this.defaultReadCachingStrategy =
- new CachingStrategy(readDropBehind, readahead);
- this.defaultWriteCachingStrategy =
- new CachingStrategy(writeDropBehind, readahead);
- this.clientContext = ClientContext.get(
- conf.get(DFS_CLIENT_CONTEXT, DFS_CLIENT_CONTEXT_DEFAULT),
- dfsClientConf);
-
- if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) {
- this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize());
- }
- this.saslClient = new SaslDataTransferClient(
- conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf),
- TrustedChannelResolver.getInstance(conf), nnFallbackToSimpleAuth);
- }
-
- /**
- * Return the socket addresses to use with each configured
- * local interface. Local interfaces may be specified by IP
- * address, IP address range using CIDR notation, interface
- * name (e.g. eth0) or sub-interface name (e.g. eth0:0).
- * The socket addresses consist of the IPs for the interfaces
- * and the ephemeral port (port 0). If an IP, IP range, or
- * interface name matches an interface with sub-interfaces
- * only the IP of the interface is used. Sub-interfaces can
- * be used by specifying them explicitly (by IP or name).
- *
- * @return SocketAddresses for the configured local interfaces,
- * or an empty array if none are configured
- * @throws UnknownHostException if a given interface name is invalid
- */
- private static SocketAddress[] getLocalInterfaceAddrs(
- String interfaceNames[]) throws UnknownHostException {
- List<SocketAddress> localAddrs = new ArrayList<SocketAddress>();
- for (String interfaceName : interfaceNames) {
- if (InetAddresses.isInetAddress(interfaceName)) {
- localAddrs.add(new InetSocketAddress(interfaceName, 0));
- } else if (NetUtils.isValidSubnet(interfaceName)) {
- for (InetAddress addr : NetUtils.getIPs(interfaceName, false)) {
- localAddrs.add(new InetSocketAddress(addr, 0));
- }
- } else {
- for (String ip : DNS.getIPs(interfaceName, false)) {
- localAddrs.add(new InetSocketAddress(ip, 0));
- }
- }
- }
- return localAddrs.toArray(new SocketAddress[localAddrs.size()]);
- }
-
- /**
- * Select one of the configured local interfaces at random. We use a random
- * interface because other policies like round-robin are less effective
- * given that we cache connections to datanodes.
- *
- * @return one of the local interface addresses at random, or null if no
- * local interfaces are configured
- */
- SocketAddress getRandomLocalInterfaceAddr() {
- if (localInterfaceAddrs.length == 0) {
- return null;
- }
- final int idx = r.nextInt(localInterfaceAddrs.length);
- final SocketAddress addr = localInterfaceAddrs[idx];
- if (LOG.isDebugEnabled()) {
- LOG.debug("Using local interface " + addr);
- }
- return addr;
- }
-
- /**
- * Return the timeout that clients should use when writing to datanodes.
- * @param numNodes the number of nodes in the pipeline.
- */
- int getDatanodeWriteTimeout(int numNodes) {
- final int t = dfsClientConf.getDatanodeSocketWriteTimeout();
- return t > 0? t + HdfsConstants.WRITE_TIMEOUT_EXTENSION*numNodes: 0;
- }
-
- int getDatanodeReadTimeout(int numNodes) {
- final int t = dfsClientConf.getSocketTimeout();
- return t > 0? HdfsConstants.READ_TIMEOUT_EXTENSION*numNodes + t: 0;
- }
-
- @VisibleForTesting
- public String getClientName() {
- return clientName;
- }
-
- void checkOpen() throws IOException {
- if (!clientRunning) {
- IOException result = new IOException("Filesystem closed");
- throw result;
- }
- }
-
- /** Return the lease renewer instance. The renewer thread won't start
- * until the first output stream is created. The same instance will
- * be returned until all output streams are closed.
- */
- public LeaseRenewer getLeaseRenewer() throws IOException {
- return LeaseRenewer.getInstance(authority, ugi, this);
- }
-
- /** Get a lease and start automatic renewal */
- private void beginFileLease(final long inodeId, final DFSOutputStream out)
- throws IOException {
- getLeaseRenewer().put(inodeId, out, this);
- }
-
- /** Stop renewal of lease for the file. */
- void endFileLease(final long inodeId) throws IOException {
- getLeaseRenewer().closeFile(inodeId, this);
- }
-
-
- /** Put a file. Only called from LeaseRenewer, where proper locking is
- * enforced to consistently update its local dfsclients array and
- * client's filesBeingWritten map.
- */
- public void putFileBeingWritten(final long inodeId, final DFSOutputStream out) {
- synchronized(filesBeingWritten) {
- filesBeingWritten.put(inodeId, out);
- // update the last lease renewal time only when there was no
- // writes. once there is one write stream open, the lease renewer
- // thread keeps it updated well with in anyone's expiration time.
- if (lastLeaseRenewal == 0) {
- updateLastLeaseRenewal();
- }
- }
- }
-
- /** Remove a file. Only called from LeaseRenewer. */
- public void removeFileBeingWritten(final long inodeId) {
- synchronized(filesBeingWritten) {
- filesBeingWritten.remove(inodeId);
- if (filesBeingWritten.isEmpty()) {
- lastLeaseRenewal = 0;
- }
- }
- }
-
- /** Is file-being-written map empty? */
- public boolean isFilesBeingWrittenEmpty() {
- synchronized(filesBeingWritten) {
- return filesBeingWritten.isEmpty();
- }
- }
-
- /** @return true if the client is running */
- public boolean isClientRunning() {
- return clientRunning;
- }
-
- long getLastLeaseRenewal() {
- return lastLeaseRenewal;
- }
-
- void updateLastLeaseRenewal() {
- synchronized(filesBeingWritten) {
- if (filesBeingWritten.isEmpty()) {
- return;
- }
- lastLeaseRenewal = Time.monotonicNow();
- }
- }
-
- /**
- * Renew leases.
- * @return true if lease was renewed. May return false if this
- * client has been closed or has no files open.
- **/
- public boolean renewLease() throws IOException {
- if (clientRunning && !isFilesBeingWrittenEmpty()) {
- try {
- namenode.renewLease(clientName);
- updateLastLeaseRenewal();
- return true;
- } catch (IOException e) {
- // Abort if the lease has already expired.
- final long elapsed = Time.monotonicNow() - getLastLeaseRenewal();
- if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
- LOG.warn("Failed to renew lease for " + clientName + " for "
- + (elapsed/1000) + " seconds (>= hard-limit ="
- + (HdfsConstants.LEASE_HARDLIMIT_PERIOD / 1000) + " seconds.) "
- + "Closing all files being written ...", e);
- closeAllFilesBeingWritten(true);
- } else {
- // Let the lease renewer handle it and retry.
- throw e;
- }
- }
- }
- return false;
- }
-
- /**
- * Close connections the Namenode.
- */
- void closeConnectionToNamenode() {
- RPC.stopProxy(namenode);
- }
-
- /** Close/abort all files being written. */
- public void closeAllFilesBeingWritten(final boolean abort) {
- for(;;) {
- final long inodeId;
- final DFSOutputStream out;
- synchronized(filesBeingWritten) {
- if (filesBeingWritten.isEmpty()) {
- return;
- }
- inodeId = filesBeingWritten.keySet().iterator().next();
- out = filesBeingWritten.remove(inodeId);
- }
- if (out != null) {
- try {
- if (abort) {
- out.abort();
- } else {
- out.close();
- }
- } catch(IOException ie) {
- LOG.error("Failed to " + (abort ? "abort" : "close") + " file: "
- + out.getSrc() + " with inode: " + inodeId, ie);
- }
- }
- }
- }
-
- /**
- * Close the file system, abandoning all of the leases and files being
- * created and close connections to the namenode.
- */
- @Override
- public synchronized void close() throws IOException {
- if(clientRunning) {
- closeAllFilesBeingWritten(false);
- clientRunning = false;
- getLeaseRenewer().closeClient(this);
- // close connections to the namenode
- closeConnectionToNamenode();
- }
- }
-
- /**
- * Close all open streams, abandoning all of the leases and files being
- * created.
- * @param abort whether streams should be gracefully closed
- */
- public void closeOutputStreams(boolean abort) {
- if (clientRunning) {
- closeAllFilesBeingWritten(abort);
- }
- }
-
- /**
- * @see ClientProtocol#getPreferredBlockSize(String)
- */
- public long getBlockSize(String f) throws IOException {
- TraceScope scope = getPathTraceScope("getBlockSize", f);
- try {
- return namenode.getPreferredBlockSize(f);
- } catch (IOException ie) {
- LOG.warn("Problem getting block size", ie);
- throw ie;
- } finally {
- scope.close();
- }
- }
-
- /**
- * Get server default values for a number of configuration params.
- * @see ClientProtocol#getServerDefaults()
- */
- public FsServerDefaults getServerDefaults() throws IOException {
- long now = Time.monotonicNow();
- if ((serverDefaults == null) ||
- (now - serverDefaultsLastUpdate > SERVER_DEFAULTS_VALIDITY_PERIOD)) {
- serverDefaults = namenode.getServerDefaults();
- serverDefaultsLastUpdate = now;
- }
- assert serverDefaults != null;
- return serverDefaults;
- }
-
- /**
- * Get a canonical token service name for this client's tokens. Null should
- * be returned if the client is not using tokens.
- * @return the token service for the client
- */
- @InterfaceAudience.LimitedPrivate( { "HDFS" })
- public String getCanonicalServiceName() {
- return (dtService != null) ? dtService.toString() : null;
- }
-
- /**
- * @see ClientProtocol#getDelegationToken(Text)
- */
- public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
- throws IOException {
- assert dtService != null;
- TraceScope scope = Trace.startSpan("getDelegationToken", traceSampler);
- try {
- Token<DelegationTokenIdentifier> token =
- namenode.getDelegationToken(renewer);
- if (token != null) {
- token.setService(this.dtService);
- LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(token));
- } else {
- LOG.info("Cannot get delegation token from " + renewer);
- }
- return token;
- } finally {
- scope.close();
- }
- }
-
- /**
- * Renew a delegation token
- * @param token the token to renew
- * @return the new expiration time
- * @throws InvalidToken
- * @throws IOException
- * @deprecated Use Token.renew instead.
- */
- @Deprecated
- public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
- throws InvalidToken, IOException {
- LOG.info("Renewing " + DelegationTokenIdentifier.stringifyToken(token));
- try {
- return token.renew(conf);
- } catch (InterruptedException ie) {
- throw new RuntimeException("caught interrupted", ie);
- } catch (RemoteException re) {
- throw re.unwrapRemoteException(InvalidToken.class,
- AccessControlException.class);
- }
- }
-
- /**
- * Cancel a delegation token
- * @param token the token to cancel
- * @throws InvalidToken
- * @throws IOException
- * @deprecated Use Token.cancel instead.
- */
- @Deprecated
- public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
- throws InvalidToken, IOException {
- LOG.info("Cancelling " + DelegationTokenIdentifier.stringifyToken(token));
- try {
- token.cancel(conf);
- } catch (InterruptedException ie) {
- throw new RuntimeException("caught interrupted", ie);
- } catch (RemoteException re) {
- throw re.unwrapRemoteException(InvalidToken.class,
- AccessControlException.class);
- }
- }
-
- @InterfaceAudience.Private
- public static class Renewer extends TokenRenewer {
-
- static {
- //Ensure that HDFS Configuration files are loaded before trying to use
- // the renewer.
- HdfsConfiguration.init();
- }
-
- @Override
- public boolean handleKind(Text kind) {
- return DelegationTokenIdentifier.HDFS_DELEGATION_KIND.equals(kind);
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public long renew(Token<?> token, Configuration conf) throws IOException {
- Token<DelegationTokenIdentifier> delToken =
- (Token<DelegationTokenIdentifier>) token;
- ClientProtocol nn = getNNProxy(delToken, conf);
- try {
- return nn.renewDelegationToken(delToken);
- } catch (RemoteException re) {
- throw re.unwrapRemoteException(InvalidToken.class,
- AccessControlException.class);
- }
- }
-
- @SuppressWarnings("unchecked")
- @Override
- public void cancel(Token<?> token, Configuration conf) throws IOException {
- Token<DelegationTokenIdentifier> delToken =
- (Token<DelegationTokenIdentifier>) token;
- LOG.info("Cancelling " +
- DelegationTokenIdentifier.stringifyToken(delToken));
- ClientProtocol nn = getNNProxy(delToken, conf);
- try {
- nn.cancelDelegationToken(delToken);
- } catch (RemoteException re) {
- throw re.unwrapRemoteException(InvalidToken.class,
- AccessControlException.class);
- }
- }
-
- private static ClientProtocol getNNProxy(
- Token<DelegationTokenIdentifier> token, Configuration conf)
- throws IOException {
- URI uri = HAUtilClient.getServiceUriFromToken(
- HdfsConstants.HDFS_URI_SCHEME, token);
- if (HAUtilClient.isTokenForLogicalUri(token) &&
- !HAUtilClient.isLogicalUri(conf, uri)) {
- // If the token is for a logical nameservice, but the configuration
- // we have disagrees about that, we can't actually renew it.
- // This can be the case in MR, for example, if the RM doesn't
- // have all of the HA clusters configured in its configuration.
- throw new IOException("Unable to map logical nameservice URI '" +
- uri + "' to a NameNode. Local configuration does not have " +
- "a failover proxy provider configured.");
- }
-
- ProxyAndInfo<ClientProtocol> info =
- NameNodeProxiesClient.createProxyWithClientProtocol(conf, uri, null);
- assert info.getDelegationTokenService().equals(token.getService()) :
- "Returned service '" + info.getDelegationTokenService().toString() +
- "' doesn't match expected service '" +
- token.getService().toString() + "'";
-
- return info.getProxy();
- }
-
- @Override
- public boolean isManaged(Token<?> token) throws IOException {
- return true;
- }
-
- }
-
- /**
- * Report corrupt blocks that were discovered by the client.
- * @see ClientProtocol#reportBadBlocks(LocatedBlock[])
- */
- public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
- namenode.reportBadBlocks(blocks);
- }
-
- public LocatedBlocks getLocatedBlocks(String src, long start)
- throws IOException {
- return getLocatedBlocks(src, start, dfsClientConf.getPrefetchSize());
- }
-
- /*
- * This is just a wrapper around callGetBlockLocations, but non-static so that
- * we can stub it out for tests.
- */
- @VisibleForTesting
- public LocatedBlocks getLocatedBlocks(String src, long start, long length)
- throws IOException {
- TraceScope scope = getPathTraceScope("getBlockLocations", src);
- try {
- return callGetBlockLocations(namenode, src, start, length);
- } finally {
- scope.close();
- }
- }
-
- /**
- * @see ClientProtocol#getBlockLocations(String, long, long)
- */
- static LocatedBlocks callGetBlockLocations(ClientProtocol namenode,
- String src, long start, long length)
- throws IOException {
- try {
- return namenode.getBlockLocations(src, start, length);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- FileNotFoundException.class,
- UnresolvedPathException.class);
- }
- }
-
- /**
- * Recover a file's lease
- * @param src a file's path
- * @return true if the file is already closed
- * @throws IOException
- */
- boolean recoverLease(String src) throws IOException {
- checkOpen();
-
- TraceScope scope = getPathTraceScope("recoverLease", src);
- try {
- return namenode.recoverLease(src, clientName);
- } catch (RemoteException re) {
- throw re.unwrapRemoteException(FileNotFoundException.class,
- AccessControlException.class,
- UnresolvedPathException.class);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Get block location info about file
- *
- * getBlockLocations() returns a list of hostnames that store
- * data for a specific file region. It returns a set of hostnames
- * for every block within the indicated region.
- *
- * This function is very useful when writing code that considers
- * data-placement when performing operations. For example, the
- * MapReduce system tries to schedule tasks on the same machines
- * as the data-block the task processes.
- */
- public BlockLocation[] getBlockLocations(String src, long start,
- long length) throws IOException, UnresolvedLinkException {
- TraceScope scope = getPathTraceScope("getBlockLocations", src);
- try {
- LocatedBlocks blocks = getLocatedBlocks(src, start, length);
- BlockLocation[] locations = DFSUtilClient.locatedBlocks2Locations(blocks);
- HdfsBlockLocation[] hdfsLocations = new HdfsBlockLocation[locations.length];
- for (int i = 0; i < locations.length; i++) {
- hdfsLocations[i] = new HdfsBlockLocation(locations[i], blocks.get(i));
- }
- return hdfsLocations;
- } finally {
- scope.close();
- }
- }
-
- /**
- * Get block location information about a list of {@link HdfsBlockLocation}.
- * Used by {@link DistributedFileSystem#getFileBlockStorageLocations(List)} to
- * get {@link BlockStorageLocation}s for blocks returned by
- * {@link DistributedFileSystem#getFileBlockLocations(org.apache.hadoop.fs.FileStatus, long, long)}
- * .
- *
- * This is done by making a round of RPCs to the associated datanodes, asking
- * the volume of each block replica. The returned array of
- * {@link BlockStorageLocation} expose this information as a
- * {@link VolumeId}.
- *
- * @param blockLocations
- * target blocks on which to query volume location information
- * @return volumeBlockLocations original block array augmented with additional
- * volume location information for each replica.
- */
- public BlockStorageLocation[] getBlockStorageLocations(
- List<BlockLocation> blockLocations) throws IOException,
- UnsupportedOperationException, InvalidBlockTokenException {
- if (!getConf().isHdfsBlocksMetadataEnabled()) {
- throw new UnsupportedOperationException("Datanode-side support for " +
- "getVolumeBlockLocations() must also be enabled in the client " +
- "configuration.");
- }
- // Downcast blockLocations and fetch out required LocatedBlock(s)
- List<LocatedBlock> blocks = new ArrayList<LocatedBlock>();
- for (BlockLocation loc : blockLocations) {
- if (!(loc instanceof HdfsBlockLocation)) {
- throw new ClassCastException("DFSClient#getVolumeBlockLocations " +
- "expected to be passed HdfsBlockLocations");
- }
- HdfsBlockLocation hdfsLoc = (HdfsBlockLocation) loc;
- blocks.add(hdfsLoc.getLocatedBlock());
- }
-
- // Re-group the LocatedBlocks to be grouped by datanodes, with the values
- // a list of the LocatedBlocks on the datanode.
- Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks =
- new LinkedHashMap<DatanodeInfo, List<LocatedBlock>>();
- for (LocatedBlock b : blocks) {
- for (DatanodeInfo info : b.getLocations()) {
- if (!datanodeBlocks.containsKey(info)) {
- datanodeBlocks.put(info, new ArrayList<LocatedBlock>());
- }
- List<LocatedBlock> l = datanodeBlocks.get(info);
- l.add(b);
- }
- }
-
- // Make RPCs to the datanodes to get volume locations for its replicas
- TraceScope scope =
- Trace.startSpan("getBlockStorageLocations", traceSampler);
- Map<DatanodeInfo, HdfsBlocksMetadata> metadatas;
- try {
- metadatas = BlockStorageLocationUtil.
- queryDatanodesForHdfsBlocksMetadata(conf, datanodeBlocks,
- getConf().getFileBlockStorageLocationsNumThreads(),
- getConf().getFileBlockStorageLocationsTimeoutMs(),
- getConf().isConnectToDnViaHostname());
- if (LOG.isTraceEnabled()) {
- LOG.trace("metadata returned: "
- + Joiner.on("\n").withKeyValueSeparator("=").join(metadatas));
- }
- } finally {
- scope.close();
- }
-
- // Regroup the returned VolumeId metadata to again be grouped by
- // LocatedBlock rather than by datanode
- Map<LocatedBlock, List<VolumeId>> blockVolumeIds = BlockStorageLocationUtil
- .associateVolumeIdsWithBlocks(blocks, metadatas);
-
- // Combine original BlockLocations with new VolumeId information
- BlockStorageLocation[] volumeBlockLocations = BlockStorageLocationUtil
- .convertToVolumeBlockLocations(blocks, blockVolumeIds);
-
- return volumeBlockLocations;
- }
-
- /**
- * Decrypts a EDEK by consulting the KeyProvider.
- */
- private KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo
- feInfo) throws IOException {
- TraceScope scope = Trace.startSpan("decryptEDEK", traceSampler);
- try {
- KeyProvider provider = getKeyProvider();
- if (provider == null) {
- throw new IOException("No KeyProvider is configured, cannot access" +
- " an encrypted file");
- }
- EncryptedKeyVersion ekv = EncryptedKeyVersion.createForDecryption(
- feInfo.getKeyName(), feInfo.getEzKeyVersionName(), feInfo.getIV(),
- feInfo.getEncryptedDataEncryptionKey());
- try {
- KeyProviderCryptoExtension cryptoProvider = KeyProviderCryptoExtension
- .createKeyProviderCryptoExtension(provider);
- return cryptoProvider.decryptEncryptedKey(ekv);
- } catch (GeneralSecurityException e) {
- throw new IOException(e);
- }
- } finally {
- scope.close();
- }
- }
-
- /**
- * Obtain the crypto protocol version from the provided FileEncryptionInfo,
- * checking to see if this version is supported by.
- *
- * @param feInfo FileEncryptionInfo
- * @return CryptoProtocolVersion from the feInfo
- * @throws IOException if the protocol version is unsupported.
- */
- private static CryptoProtocolVersion getCryptoProtocolVersion
- (FileEncryptionInfo feInfo) throws IOException {
- final CryptoProtocolVersion version = feInfo.getCryptoProtocolVersion();
- if (!CryptoProtocolVersion.supports(version)) {
- throw new IOException("Client does not support specified " +
- "CryptoProtocolVersion " + version.getDescription() + " version " +
- "number" + version.getVersion());
- }
- return version;
- }
-
- /**
- * Obtain a CryptoCodec based on the CipherSuite set in a FileEncryptionInfo
- * and the available CryptoCodecs configured in the Configuration.
- *
- * @param conf Configuration
- * @param feInfo FileEncryptionInfo
- * @return CryptoCodec
- * @throws IOException if no suitable CryptoCodec for the CipherSuite is
- * available.
- */
- private static CryptoCodec getCryptoCodec(Configuration conf,
- FileEncryptionInfo feInfo) throws IOException {
- final CipherSuite suite = feInfo.getCipherSuite();
- if (suite.equals(CipherSuite.UNKNOWN)) {
- throw new IOException("NameNode specified unknown CipherSuite with ID "
- + suite.getUnknownValue() + ", cannot instantiate CryptoCodec.");
- }
- final CryptoCodec codec = CryptoCodec.getInstance(conf, suite);
- if (codec == null) {
- throw new UnknownCipherSuiteException(
- "No configuration found for the cipher suite "
- + suite.getConfigSuffix() + " prefixed with "
- + HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
- + ". Please see the example configuration "
- + "hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE "
- + "at core-default.xml for details.");
- }
- return codec;
- }
-
- /**
- * Wraps the stream in a CryptoInputStream if the underlying file is
- * encrypted.
- */
- public HdfsDataInputStream createWrappedInputStream(DFSInputStream dfsis)
- throws IOException {
- final FileEncryptionInfo feInfo = dfsis.getFileEncryptionInfo();
- if (feInfo != null) {
- // File is encrypted, wrap the stream in a crypto stream.
- // Currently only one version, so no special logic based on the version #
- getCryptoProtocolVersion(feInfo);
- final CryptoCodec codec = getCryptoCodec(conf, feInfo);
- final KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
- final CryptoInputStream cryptoIn =
- new CryptoInputStream(dfsis, codec, decrypted.getMaterial(),
- feInfo.getIV());
- return new HdfsDataInputStream(cryptoIn);
- } else {
- // No FileEncryptionInfo so no encryption.
- return new HdfsDataInputStream(dfsis);
- }
- }
-
- /**
- * Wraps the stream in a CryptoOutputStream if the underlying file is
- * encrypted.
- */
- public HdfsDataOutputStream createWrappedOutputStream(DFSOutputStream dfsos,
- FileSystem.Statistics statistics) throws IOException {
- return createWrappedOutputStream(dfsos, statistics, 0);
- }
-
- /**
- * Wraps the stream in a CryptoOutputStream if the underlying file is
- * encrypted.
- */
- public HdfsDataOutputStream createWrappedOutputStream(DFSOutputStream dfsos,
- FileSystem.Statistics statistics, long startPos) throws IOException {
- final FileEncryptionInfo feInfo = dfsos.getFileEncryptionInfo();
- if (feInfo != null) {
- // File is encrypted, wrap the stream in a crypto stream.
- // Currently only one version, so no special logic based on the version #
- getCryptoProtocolVersion(feInfo);
- final CryptoCodec codec = getCryptoCodec(conf, feInfo);
- KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
- final CryptoOutputStream cryptoOut =
- new CryptoOutputStream(dfsos, codec,
- decrypted.getMaterial(), feInfo.getIV(), startPos);
- return new HdfsDataOutputStream(cryptoOut, statistics, startPos);
- } else {
- // No FileEncryptionInfo present so no encryption.
- return new HdfsDataOutputStream(dfsos, statistics, startPos);
- }
- }
-
- public DFSInputStream open(String src)
- throws IOException, UnresolvedLinkException {
- return open(src, dfsClientConf.getIoBufferSize(), true, null);
- }
-
- /**
- * Create an input stream that obtains a nodelist from the
- * namenode, and then reads from all the right places. Creates
- * inner subclass of InputStream that does the right out-of-band
- * work.
- * @deprecated Use {@link #open(String, int, boolean)} instead.
- */
- @Deprecated
- public DFSInputStream open(String src, int buffersize, boolean verifyChecksum,
- FileSystem.Statistics stats)
- throws IOException, UnresolvedLinkException {
- return open(src, buffersize, verifyChecksum);
- }
-
-
- /**
- * Create an input stream that obtains a nodelist from the
- * namenode, and then reads from all the right places. Creates
- * inner subclass of InputStream that does the right out-of-band
- * work.
- */
- public DFSInputStream open(String src, int buffersize, boolean verifyChecksum)
- throws IOException, UnresolvedLinkException {
- checkOpen();
- // Get block info from namenode
- TraceScope scope = getPathTraceScope("newDFSInputStream", src);
- try {
- return new DFSInputStream(this, src, verifyChecksum, null);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Get the namenode associated with this DFSClient object
- * @return the namenode associated with this DFSClient object
- */
- public ClientProtocol getNamenode() {
- return namenode;
- }
-
- /**
- * Call {@link #create(String, boolean, short, long, Progressable)} with
- * default <code>replication</code> and <code>blockSize<code> and null <code>
- * progress</code>.
- */
- public OutputStream create(String src, boolean overwrite)
- throws IOException {
- return create(src, overwrite, dfsClientConf.getDefaultReplication(),
- dfsClientConf.getDefaultBlockSize(), null);
- }
-
- /**
- * Call {@link #create(String, boolean, short, long, Progressable)} with
- * default <code>replication</code> and <code>blockSize<code>.
- */
- public OutputStream create(String src,
- boolean overwrite,
- Progressable progress) throws IOException {
- return create(src, overwrite, dfsClientConf.getDefaultReplication(),
- dfsClientConf.getDefaultBlockSize(), progress);
- }
-
- /**
- * Call {@link #create(String, boolean, short, long, Progressable)} with
- * null <code>progress</code>.
- */
- public OutputStream create(String src,
- boolean overwrite,
- short replication,
- long blockSize) throws IOException {
- return create(src, overwrite, replication, blockSize, null);
- }
-
- /**
- * Call {@link #create(String, boolean, short, long, Progressable, int)}
- * with default bufferSize.
- */
- public OutputStream create(String src, boolean overwrite, short replication,
- long blockSize, Progressable progress) throws IOException {
- return create(src, overwrite, replication, blockSize, progress,
- dfsClientConf.getIoBufferSize());
- }
-
- /**
- * Call {@link #create(String, FsPermission, EnumSet, short, long,
- * Progressable, int, ChecksumOpt)} with default <code>permission</code>
- * {@link FsPermission#getFileDefault()}.
- *
- * @param src File name
- * @param overwrite overwrite an existing file if true
- * @param replication replication factor for the file
- * @param blockSize maximum block size
- * @param progress interface for reporting client progress
- * @param buffersize underlying buffersize
- *
- * @return output stream
- */
- public OutputStream create(String src,
- boolean overwrite,
- short replication,
- long blockSize,
- Progressable progress,
- int buffersize)
- throws IOException {
- return create(src, FsPermission.getFileDefault(),
- overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
- : EnumSet.of(CreateFlag.CREATE), replication, blockSize, progress,
- buffersize, null);
- }
-
- /**
- * Call {@link #create(String, FsPermission, EnumSet, boolean, short,
- * long, Progressable, int, ChecksumOpt)} with <code>createParent</code>
- * set to true.
- */
- public DFSOutputStream create(String src,
- FsPermission permission,
- EnumSet<CreateFlag> flag,
- short replication,
- long blockSize,
- Progressable progress,
- int buffersize,
- ChecksumOpt checksumOpt)
- throws IOException {
- return create(src, permission, flag, true,
- replication, blockSize, progress, buffersize, checksumOpt, null);
- }
-
- /**
- * Create a new dfs file with the specified block replication
- * with write-progress reporting and return an output stream for writing
- * into the file.
- *
- * @param src File name
- * @param permission The permission of the directory being created.
- * If null, use default permission {@link FsPermission#getFileDefault()}
- * @param flag indicates create a new file or create/overwrite an
- * existing file or append to an existing file
- * @param createParent create missing parent directory if true
- * @param replication block replication
- * @param blockSize maximum block size
- * @param progress interface for reporting client progress
- * @param buffersize underlying buffer size
- * @param checksumOpt checksum options
- *
- * @return output stream
- *
- * @see ClientProtocol#create for detailed description of exceptions thrown
- */
- public DFSOutputStream create(String src,
- FsPermission permission,
- EnumSet<CreateFlag> flag,
- boolean createParent,
- short replication,
- long blockSize,
- Progressable progress,
- int buffersize,
- ChecksumOpt checksumOpt) throws IOException {
- return create(src, permission, flag, createParent, replication, blockSize,
- progress, buffersize, checksumOpt, null);
- }
-
- private FsPermission applyUMask(FsPermission permission) {
- if (permission == null) {
- permission = FsPermission.getFileDefault();
- }
- return permission.applyUMask(dfsClientConf.getUMask());
- }
-
- /**
- * Same as {@link #create(String, FsPermission, EnumSet, boolean, short, long,
- * Progressable, int, ChecksumOpt)} with the addition of favoredNodes that is
- * a hint to where the namenode should place the file blocks.
- * The favored nodes hint is not persisted in HDFS. Hence it may be honored
- * at the creation time only. HDFS could move the blocks during balancing or
- * replication, to move the blocks from favored nodes. A value of null means
- * no favored nodes for this create
- */
- public DFSOutputStream create(String src,
- FsPermission permission,
- EnumSet<CreateFlag> flag,
- boolean createParent,
- short replication,
- long blockSize,
- Progressable progress,
- int buffersize,
- ChecksumOpt checksumOpt,
- InetSocketAddress[] favoredNodes) throws IOException {
- checkOpen();
- final FsPermission masked = applyUMask(permission);
- if(LOG.isDebugEnabled()) {
- LOG.debug(src + ": masked=" + masked);
- }
- final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this,
- src, masked, flag, createParent, replication, blockSize, progress,
- buffersize, dfsClientConf.createChecksum(checksumOpt),
- getFavoredNodesStr(favoredNodes));
- beginFileLease(result.getFileId(), result);
- return result;
- }
-
- private String[] getFavoredNodesStr(InetSocketAddress[] favoredNodes) {
- String[] favoredNodeStrs = null;
- if (favoredNodes != null) {
- favoredNodeStrs = new String[favoredNodes.length];
- for (int i = 0; i < favoredNodes.length; i++) {
- favoredNodeStrs[i] =
- favoredNodes[i].getHostName() + ":"
- + favoredNodes[i].getPort();
- }
- }
- return favoredNodeStrs;
- }
-
- /**
- * Append to an existing file if {@link CreateFlag#APPEND} is present
- */
- private DFSOutputStream primitiveAppend(String src, EnumSet<CreateFlag> flag,
- int buffersize, Progressable progress) throws IOException {
- if (flag.contains(CreateFlag.APPEND)) {
- HdfsFileStatus stat = getFileInfo(src);
- if (stat == null) { // No file to append to
- // New file needs to be created if create option is present
- if (!flag.contains(CreateFlag.CREATE)) {
- throw new FileNotFoundException("failed to append to non-existent file "
- + src + " on client " + clientName);
- }
- return null;
- }
- return callAppend(src, buffersize, flag, progress, null);
- }
- return null;
- }
-
- /**
- * Same as {{@link #create(String, FsPermission, EnumSet, short, long,
- * Progressable, int, ChecksumOpt)} except that the permission
- * is absolute (ie has already been masked with umask.
- */
- public DFSOutputStream primitiveCreate(String src,
- FsPermission absPermission,
- EnumSet<CreateFlag> flag,
- boolean createParent,
- short replication,
- long blockSize,
- Progressable progress,
- int buffersize,
- ChecksumOpt checksumOpt)
- throws IOException, UnresolvedLinkException {
- checkOpen();
- CreateFlag.validate(flag);
- DFSOutputStream result = primitiveAppend(src, flag, buffersize, progress);
- if (result == null) {
- DataChecksum checksum = dfsClientConf.createChecksum(checksumOpt);
- result = DFSOutputStream.newStreamForCreate(this, src, absPermission,
- flag, createParent, replication, blockSize, progress, buffersize,
- checksum, null);
- }
- beginFileLease(result.getFileId(), result);
- return result;
- }
-
- /**
- * Creates a symbolic link.
- *
- * @see ClientProtocol#createSymlink(String, String,FsPermission, boolean)
- */
- public void createSymlink(String target, String link, boolean createParent)
- throws IOException {
- TraceScope scope = getPathTraceScope("createSymlink", target);
- try {
- final FsPermission dirPerm = applyUMask(null);
- namenode.createSymlink(target, link, dirPerm, createParent);
- } catch (RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- FileAlreadyExistsException.class,
- FileNotFoundException.class,
- ParentNotDirectoryException.class,
- NSQuotaExceededException.class,
- DSQuotaExceededException.class,
- QuotaByStorageTypeExceededException.class,
- UnresolvedPathException.class,
- SnapshotAccessControlException.class);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Resolve the *first* symlink, if any, in the path.
- *
- * @see ClientProtocol#getLinkTarget(String)
- */
- public String getLinkTarget(String path) throws IOException {
- checkOpen();
- TraceScope scope = getPathTraceScope("getLinkTarget", path);
- try {
- return namenode.getLinkTarget(path);
- } catch (RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- FileNotFoundException.class);
- } finally {
- scope.close();
- }
- }
-
- /** Method to get stream returned by append call */
- private DFSOutputStream callAppend(String src, int buffersize,
- EnumSet<CreateFlag> flag, Progressable progress, String[] favoredNodes)
- throws IOException {
- CreateFlag.validateForAppend(flag);
- try {
- LastBlockWithStatus blkWithStatus = namenode.append(src, clientName,
- new EnumSetWritable<>(flag, CreateFlag.class));
- return DFSOutputStream.newStreamForAppend(this, src, flag, buffersize,
- progress, blkWithStatus.getLastBlock(),
- blkWithStatus.getFileStatus(), dfsClientConf.createChecksum(null),
- favoredNodes);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- FileNotFoundException.class,
- SafeModeException.class,
- DSQuotaExceededException.class,
- QuotaByStorageTypeExceededException.class,
- UnsupportedOperationException.class,
- UnresolvedPathException.class,
- SnapshotAccessControlException.class);
- }
- }
-
- /**
- * Append to an existing HDFS file.
- *
- * @param src file name
- * @param buffersize buffer size
- * @param flag indicates whether to append data to a new block instead of
- * the last block
- * @param progress for reporting write-progress; null is acceptable.
- * @param statistics file system statistics; null is acceptable.
- * @return an output stream for writing into the file
- *
- * @see ClientProtocol#append(String, String, EnumSetWritable)
- */
- public HdfsDataOutputStream append(final String src, final int buffersize,
- EnumSet<CreateFlag> flag, final Progressable progress,
- final FileSystem.Statistics statistics) throws IOException {
- final DFSOutputStream out = append(src, buffersize, flag, null, progress);
- return createWrappedOutputStream(out, statistics, out.getInitialLen());
- }
-
- /**
- * Append to an existing HDFS file.
- *
- * @param src file name
- * @param buffersize buffer size
- * @param flag indicates whether to append data to a new block instead of the
- * last block
- * @param progress for reporting write-progress; null is acceptable.
- * @param statistics file system statistics; null is acceptable.
- * @param favoredNodes FavoredNodes for new blocks
- * @return an output stream for writing into the file
- * @see ClientProtocol#append(String, String, EnumSetWritable)
- */
- public HdfsDataOutputStream append(final String src, final int buffersize,
- EnumSet<CreateFlag> flag, final Progressable progress,
- final FileSystem.Statistics statistics,
- final InetSocketAddress[] favoredNodes) throws IOException {
- final DFSOutputStream out = append(src, buffersize, flag,
- getFavoredNodesStr(favoredNodes), progress);
- return createWrappedOutputStream(out, statistics, out.getInitialLen());
- }
-
- private DFSOutputStream append(String src, int buffersize,
- EnumSet<CreateFlag> flag, String[] favoredNodes, Progressable progress)
- throws IOException {
- checkOpen();
- final DFSOutputStream result = callAppend(src, buffersize, flag, progress,
- favoredNodes);
- beginFileLease(result.getFileId(), result);
- return result;
- }
-
- /**
- * Set replication for an existing file.
- * @param src file name
- * @param replication replication to set the file to
- *
- * @see ClientProtocol#setReplication(String, short)
- */
- public boolean setReplication(String src, short replication)
- throws IOException {
- TraceScope scope = getPathTraceScope("setReplication", src);
- try {
- return namenode.setReplication(src, replication);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- FileNotFoundException.class,
- SafeModeException.class,
- DSQuotaExceededException.class,
- QuotaByStorageTypeExceededException.class,
- UnresolvedPathException.class,
- SnapshotAccessControlException.class);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Set storage policy for an existing file/directory
- * @param src file/directory name
- * @param policyName name of the storage policy
- */
- public void setStoragePolicy(String src, String policyName)
- throws IOException {
- TraceScope scope = getPathTraceScope("setStoragePolicy", src);
- try {
- namenode.setStoragePolicy(src, policyName);
- } catch (RemoteException e) {
- throw e.unwrapRemoteException(AccessControlException.class,
- FileNotFoundException.class,
- SafeModeException.class,
- NSQuotaExceededException.class,
- UnresolvedPathException.class,
- SnapshotAccessControlException.class);
- } finally {
- scope.close();
- }
- }
-
- /**
- * @param path file/directory name
- * @return Get the storage policy for specified path
- */
- public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
- checkOpen();
- TraceScope scope = getPathTraceScope("getStoragePolicy", path);
- try {
- return namenode.getStoragePolicy(path);
- } catch (RemoteException e) {
- throw e.unwrapRemoteException(AccessControlException.class,
- FileNotFoundException.class,
- SafeModeException.class,
- UnresolvedPathException.class);
- } finally {
- scope.close();
- }
- }
-
- /**
- * @return All the existing storage policies
- */
- public BlockStoragePolicy[] getStoragePolicies() throws IOException {
- TraceScope scope = Trace.startSpan("getStoragePolicies", traceSampler);
- try {
- return namenode.getStoragePolicies();
- } finally {
- scope.close();
- }
- }
-
- /**
- * Rename file or directory.
- * @see ClientProtocol#rename(String, String)
- * @deprecated Use {@link #rename(String, String, Options.Rename...)} instead.
- */
- @Deprecated
- public boolean rename(String src, String dst) throws IOException {
- checkOpen();
- TraceScope scope = getSrcDstTraceScope("rename", src, dst);
- try {
- return namenode.rename(src, dst);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- NSQuotaExceededException.class,
- DSQuotaExceededException.class,
- QuotaByStorageTypeExceededException.class,
- UnresolvedPathException.class,
- SnapshotAccessControlException.class);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Move blocks from src to trg and delete src
- * See {@link ClientProtocol#concat}.
- */
- public void concat(String trg, String [] srcs) throws IOException {
- checkOpen();
- TraceScope scope = Trace.startSpan("concat", traceSampler);
- try {
- namenode.concat(trg, srcs);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- UnresolvedPathException.class,
- SnapshotAccessControlException.class);
- } finally {
- scope.close();
- }
- }
- /**
- * Rename file or directory.
- * @see ClientProtocol#rename2(String, String, Options.Rename...)
- */
- public void rename(String src, String dst, Options.Rename... options)
- throws IOException {
- checkOpen();
- TraceScope scope = getSrcDstTraceScope("rename2", src, dst);
- try {
- namenode.rename2(src, dst, options);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- DSQuotaExceededException.class,
- QuotaByStorageTypeExceededException.class,
- FileAlreadyExistsException.class,
- FileNotFoundException.class,
- ParentNotDirectoryException.class,
- SafeModeException.class,
- NSQuotaExceededException.class,
- UnresolvedPathException.class,
- SnapshotAccessControlException.class);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Truncate a file to an indicated size
- * See {@link ClientProtocol#truncate}.
- */
- public boolean truncate(String src, long newLength) throws IOException {
- checkOpen();
- if (newLength < 0) {
- throw new HadoopIllegalArgumentException(
- "Cannot truncate to a negative file size: " + newLength + ".");
- }
- try {
- return namenode.truncate(src, newLength, clientName);
- } catch (RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- UnresolvedPathException.class);
- }
- }
-
- /**
- * Delete file or directory.
- * See {@link ClientProtocol#delete(String, boolean)}.
- */
- @Deprecated
- public boolean delete(String src) throws IOException {
- checkOpen();
- return delete(src, true);
- }
-
- /**
- * delete file or directory.
- * delete contents of the directory if non empty and recursive
- * set to true
- *
- * @see ClientProtocol#delete(String, boolean)
- */
- public boolean delete(String src, boolean recursive) throws IOException {
- checkOpen();
- TraceScope scope = getPathTraceScope("delete", src);
- try {
- return namenode.delete(src, recursive);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- FileNotFoundException.class,
- SafeModeException.class,
- UnresolvedPathException.class,
- SnapshotAccessControlException.class);
- } finally {
- scope.close();
- }
- }
-
- /** Implemented using getFileInfo(src)
- */
- public boolean exists(String src) throws IOException {
- checkOpen();
- return getFileInfo(src) != null;
- }
-
- /**
- * Get a partial listing of the indicated directory
- * No block locations need to be fetched
- */
- public DirectoryListing listPaths(String src, byte[] startAfter)
- throws IOException {
- return listPaths(src, startAfter, false);
- }
-
- /**
- * Get a partial listing of the indicated directory
- *
- * Recommend to use HdfsFileStatus.EMPTY_NAME as startAfter
- * if the application wants to fetch a listing starting from
- * the first entry in the directory
- *
- * @see ClientProtocol#getListing(String, byte[], boolean)
- */
- public DirectoryListing listPaths(String src, byte[] startAfter,
- boolean needLocation) throws IOException {
- checkOpen();
- TraceScope scope = getPathTraceScope("listPaths", src);
- try {
- return namenode.getListing(src, startAfter, needLocation);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- FileNotFoundException.class,
- UnresolvedPathException.class);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Get the file info for a specific file or directory.
- * @param src The string representation of the path to the file
- * @return object containing information regarding the file
- * or null if file not found
- *
- * @see ClientProtocol#getFileInfo(String) for description of exceptions
- */
- public HdfsFileStatus getFileInfo(String src) throws IOException {
- checkOpen();
- TraceScope scope = getPathTraceScope("getFileInfo", src);
- try {
- return namenode.getFileInfo(src);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- FileNotFoundException.class,
- UnresolvedPathException.class);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Close status of a file
- * @return true if file is already closed
- */
- public boolean isFileClosed(String src) throws IOException{
- checkOpen();
- TraceScope scope = getPathTraceScope("isFileClosed", src);
- try {
- return namenode.isFileClosed(src);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- FileNotFoundException.class,
- UnresolvedPathException.class);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Get the file info for a specific file or directory. If src
- * refers to a symlink then the FileStatus of the link is returned.
- * @param src path to a file or directory.
- *
- * For description of exceptions thrown
- * @see ClientProtocol#getFileLinkInfo(String)
- */
- public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
- checkOpen();
- TraceScope scope = getPathTraceScope("getFileLinkInfo", src);
- try {
- return namenode.getFileLinkInfo(src);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- UnresolvedPathException.class);
- } finally {
- scope.close();
- }
- }
-
- @InterfaceAudience.Private
- public void clearDataEncryptionKey() {
- LOG.debug("Clearing encryption key");
- synchronized (this) {
- encryptionKey = null;
- }
- }
-
- /**
- * @return true if data sent between this client and DNs should be encrypted,
- * false otherwise.
- * @throws IOException in the event of error communicating with the NN
- */
- boolean shouldEncryptData() throws IOException {
- FsServerDefaults d = getServerDefaults();
- return d == null ? false : d.getEncryptDataTransfer();
- }
-
- @Override
- public DataEncryptionKey newDataEncryptionKey() throws IOException {
- if (shouldEncryptData()) {
- synchronized (this) {
- if (encryptionKey == null ||
- encryptionKey.expiryDate < Time.now()) {
- LOG.debug("Getting new encryption token from NN");
- encryptionKey = namenode.getDataEncryptionKey();
- }
- return encryptionKey;
- }
- } else {
- return null;
- }
- }
-
- /**
- * Get the checksum of the whole file of a range of the file. Note that the
- * range always starts from the beginning of the file.
- * @param src The file path
- * @param length the length of the range, i.e., the range is [0, length]
- * @return The checksum
- * @see DistributedFileSystem#getFileChecksum(Path)
- */
- public MD5MD5CRC32FileChecksum getFileChecksum(String src, long length)
- throws IOException {
- checkOpen();
- Preconditions.checkArgument(length >= 0);
- //get block locations for the file range
- LocatedBlocks blockLocations = callGetBlockLocations(namenode, src, 0,
- length);
- if (null == blockLocations) {
- throw new FileNotFoundException("File does not exist: " + src);
- }
- if (blockLocations.isUnderConstruction()) {
- throw new IOException("Fail to get checksum, since file " + src
- + " is under construction.");
- }
- List<LocatedBlock> locatedblocks = blockLocations.getLocatedBlocks();
- final DataOutputBuffer md5out = new DataOutputBuffer();
- int bytesPerCRC = -1;
- DataChecksum.Type crcType = DataChecksum.Type.DEFAULT;
- long crcPerBlock = 0;
- boolean refetchBlocks = false;
- int lastRetriedIndex = -1;
-
- // get block checksum for each block
- long remaining = length;
- if (src.contains(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR)) {
- remaining = Math.min(length, blockLocations.getFileLength());
- }
- for(int i = 0; i < locatedblocks.size() && remaining > 0; i++) {
- if (refetchBlocks) { // refetch to get fresh tokens
- blockLocations = callGetBlockLocations(namenode, src, 0, length);
- if (null == blockLocations) {
- throw new FileNotFoundException("File does not exist: " + src);
- }
- if (blockLocations.isUnderConstruction()) {
- throw new IOException("Fail to get checksum, since file " + src
- + " is under construction.");
- }
- locatedblocks = blockLocations.getLocatedBlocks();
- refetchBlocks = false;
- }
- LocatedBlock lb = locatedblocks.get(i);
- final ExtendedBlock block = lb.getBlock();
- if (remaining < block.getNumBytes()) {
- block.setNumBytes(remaining);
- }
- remaining -= block.getNumBytes();
- final DatanodeInfo[] datanodes = lb.getLocations();
-
- //try each datanode location of the block
- final int timeout = 3000*datanodes.length + dfsClientConf.getSocketTimeout();
- boolean done = false;
- for(int j = 0; !done && j < datanodes.length; j++) {
- DataOutputStream out = null;
- DataInputStream in = null;
-
- try {
- //connect to a datanode
- IOStreamPair pair = connectToDN(datanodes[j], timeout, lb);
- out = new DataOutputStream(new BufferedOutputStream(pair.out,
- smallBufferSize));
- in = new DataInputStream(pair.in);
-
- if (LOG.isDebugEnabled()) {
- LOG.debug("write to " + datanodes[j] + ": "
- + Op.BLOCK_CHECKSUM + ", block=" + block);
- }
- // get block MD5
- new Sender(out).blockChecksum(block, lb.getBlockToken());
-
- final BlockOpResponseProto reply =
- BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
-
- String logInfo = "for block " + block + " from datanode " + datanodes[j];
- DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
-
- OpBlockChecksumResponseProto checksumData =
- reply.getChecksumResponse();
-
- //read byte-per-checksum
- final int bpc = checksumData.getBytesPerCrc();
- if (i == 0) { //first block
- bytesPerCRC = bpc;
- }
- else if (bpc != bytesPerCRC) {
- throw new IOException("Byte-per-checksum not matched: bpc=" + bpc
- + " but bytesPerCRC=" + bytesPerCRC);
- }
-
- //read crc-per-block
- final long cpb = checksumData.getCrcPerBlock();
- if (locatedblocks.size() > 1 && i == 0) {
- crcPerBlock = cpb;
- }
-
- //read md5
- final MD5Hash md5 = new MD5Hash(
- checksumData.getMd5().toByteArray());
- md5.write(md5out);
-
- // read crc-type
- final DataChecksum.Type ct;
- if (checksumData.hasCrcType()) {
- ct = PBHelperClient.convert(checksumData
- .getCrcType());
- } else {
- LOG.debug("Retrieving checksum from an earlier-version DataNode: " +
- "inferring checksum by reading first byte");
- ct = inferChecksumTypeByReading(lb, datanodes[j]);
- }
-
- if (i == 0) { // first block
- crcType = ct;
- } else if (crcType != DataChecksum.Type.MIXED
- && crcType != ct) {
- // if crc types are mixed in a file
- crcType = DataChecksum.Type.MIXED;
- }
-
- done = true;
-
- if (LOG.isDebugEnabled()) {
- if (i == 0) {
- LOG.debug("set bytesPerCRC=" + bytesPerCRC
- + ", crcPerBlock=" + crcPerBlock);
- }
- LOG.debug("got reply from " + datanodes[j] + ": md5=" + md5);
- }
- } catch (InvalidBlockTokenException ibte) {
- if (i > lastRetriedIndex) {
- if (LOG.isDebugEnabled()) {
- LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
- + "for file " + src + " for block " + block
- + " from datanode " + datanodes[j]
- + ". Will retry the block once.");
- }
- lastRetriedIndex = i;
- done = true; // actually it's not done; but we'll retry
- i--; // repeat at i-th block
- refetchBlocks = true;
- break;
- }
- } catch (IOException ie) {
- LOG.warn("src=" + src + ", datanodes["+j+"]=" + datanodes[j], ie);
- } finally {
- IOUtils.closeStream(in);
- IOUtils.closeStream(out);
- }
- }
-
- if (!done) {
- throw new IOException("Fail to get block MD5 for " + block);
- }
- }
-
- //compute file MD5
- final MD5Hash fileMD5 = MD5Hash.digest(md5out.getData());
- switch (crcType) {
- case CRC32:
- return new MD5MD5CRC32GzipFileChecksum(bytesPerCRC,
- crcPerBlock, fileMD5);
- case CRC32C:
- return new MD5MD5CRC32CastagnoliFileChecksum(bytesPerCRC,
- crcPerBlock, fileMD5);
- default:
- // If there is no block allocated for the file,
- // return one with the magic entry that matches what previous
- // hdfs versions return.
- if (locatedblocks.size() == 0) {
- return new MD5MD5CRC32GzipFileChecksum(0, 0, fileMD5);
- }
-
- // we should never get here since the validity was checked
- // when getCrcType() was called above.
- return null;
- }
- }
-
- /**
- * Connect to the given datanode's datantrasfer port, and return
- * the resulting IOStreamPair. This includes encryption wrapping, etc.
- */
- private IOStreamPair connectToDN(DatanodeInfo dn, int timeout,
- LocatedBlock lb) throws IOException {
- boolean success = false;
- Socket sock = null;
- try {
- sock = socketFactory.createSocket();
- String dnAddr = dn.getXferAddr(getConf().isConnectToDnViaHostname());
- if (LOG.isDebugEnabled()) {
- LOG.debug("Connecting to datanode " + dnAddr);
- }
- NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr), timeout);
- sock.setSoTimeout(timeout);
-
- OutputStream unbufOut = NetUtils.getOutputStream(sock);
- InputStream unbufIn = NetUtils.getInputStream(sock);
- IOStreamPair ret = saslClient.newSocketSend(sock, unbufOut, unbufIn, this,
- lb.getBlockToken(), dn);
- success = true;
- return ret;
- } finally {
- if (!success) {
- IOUtils.closeSocket(sock);
- }
- }
- }
-
- /**
- * Infer the checksum type for a replica by sending an OP_READ_BLOCK
- * for the first byte of that replica. This is used for compatibility
- * with older HDFS versions which did not include the checksum type in
- * OpBlockChecksumResponseProto.
- *
- * @param lb the located block
- * @param dn the connected datanode
- * @return the inferred checksum type
- * @throws IOException if an error occurs
- */
- private Type inferChecksumTypeByReading(LocatedBlock lb, DatanodeInfo dn)
- throws IOException {
- IOStreamPair pair = connectToDN(dn, dfsClientConf.getSocketTimeout(), lb);
-
- try {
- DataOutputStream out = new DataOutputStream(new BufferedOutputStream(pair.out,
- smallBufferSize));
- DataInputStream in = new DataInputStream(pair.in);
-
- new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName,
- 0, 1, true, CachingStrategy.newDefaultStrategy());
- final BlockOpResponseProto reply =
- BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
- String logInfo = "trying to read " + lb.getBlock() + " from datanode " + dn;
- DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
-
- return PBHelperClient.convert(reply.getReadOpChecksumInfo().getChecksum().getType());
- } finally {
- IOUtils.cleanup(null, pair.in, pair.out);
- }
- }
-
- /**
- * Set permissions to a file or directory.
- * @param src path name.
- * @param permission permission to set to
- *
- * @see ClientProtocol#setPermission(String, FsPermission)
- */
- public void setPermission(String src, FsPermission permission)
- throws IOException {
- checkOpen();
- TraceScope scope = getPathTraceScope("setPermission", src);
- try {
- namenode.setPermission(src, permission);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- FileNotFoundException.class,
- SafeModeException.class,
- UnresolvedPathException.class,
- SnapshotAccessControlException.class);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Set file or directory owner.
- * @param src path name.
- * @param username user id.
- * @param groupname user group.
- *
- * @see ClientProtocol#setOwner(String, String, String)
- */
- public void setOwner(String src, String username, String groupname)
- throws IOException {
- checkOpen();
- TraceScope scope = getPathTraceScope("setOwner", src);
- try {
- namenode.setOwner(src, username, groupname);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException(AccessControlException.class,
- FileNotFoundException.class,
- SafeModeException.class,
- UnresolvedPathException.class,
- SnapshotAccessControlException.class);
- } finally {
- scope.close();
- }
- }
-
- private long[] callGetStats() throws IOException {
- checkOpen();
- TraceScope scope = Trace.startSpan("getStats", traceSampler);
- try {
- return namenode.getStats();
- } finally {
- scope.close();
- }
- }
-
- /**
- * @see ClientProtocol#getStats()
- */
- public FsStatus getDiskStatus() throws IOException {
- long rawNums[] = callGetStats();
- return new FsStatus(rawNums[0], rawNums[1], rawNums[2]);
- }
-
- /**
- * Returns count of blocks with no good replicas left. Normally should be
- * zero.
- * @throws IOException
- */
- public long getMissingBlocksCount() throws IOException {
- return callGetStats()[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX];
- }
-
- /**
- * Returns count of blocks with replication factor 1 and have
- * lost the only replica.
- * @throws IOException
- */
- public long getMissingReplOneBlocksCount() throws IOException {
- return callGetStats()[ClientProtocol.
- GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX];
- }
-
- /**
- * Returns count of blocks with one of more replica missing.
- * @throws IOException
- */
- public long getUnderReplicatedBlocksCount() throws IOException {
- return callGetStats()[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX];
- }
-
- /**
- * Returns count of blocks with at least one replica marked corrupt.
- * @throws IOException
- */
- public long getCorruptBlocksCount() throws IOException {
- return callGetStats()[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX];
- }
-
- /**
- * @return a list in which each entry describes a corrupt file/block
- * @throws IOException
- */
- public CorruptFileBlocks listCorruptFileBlocks(String path,
- String cookie)
- throws IOException {
- checkOpen();
- TraceScope scope = getPathTraceScope("listCorruptFileBlocks", path);
- try {
- return namenode.listCorruptFileBlocks(path, cookie);
- } finally {
- scope.close();
- }
- }
-
- public DatanodeInfo[] datanodeReport(DatanodeReportType type)
- throws IOException {
- checkOpen();
- TraceScope scope = Trace.startSpan("datanodeReport", traceSampler);
- try {
- return namenode.getDatanodeReport(type);
- } finally {
- scope.close();
- }
- }
-
- public DatanodeStorageReport[] getDatanodeStorageReport(
- DatanodeReportType type) throws IOException {
- checkOpen();
- TraceScope scope =
- Trace.startSpan("datanodeStorageReport", traceSampler);
- try {
- return namenode.getDatanodeStorageReport(type);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Enter, leave or get safe mode.
- *
- * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction,boolean)
- */
- public boolean setSafeMode(SafeModeAction action) throws IOException {
- return setSafeMode(action, false);
- }
-
- /**
- * Enter, leave or get safe mode.
- *
- * @param action
- * One of SafeModeAction.GET, SafeModeAction.ENTER and
- * SafeModeActiob.LEAVE
- * @param isChecked
- * If true, then check only active namenode's safemode status, else
- * check first namenode's status.
- * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean)
- */
- public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
- TraceScope scope =
- Trace.startSpan("setSafeMode", traceSampler);
- try {
- return namenode.setSafeMode(action, isChecked);
- } finally {
- scope.close();
- }
- }
-
- /**
- * Create one snapshot.
- *
- * @param snapshotRoot The directory where the snapshot is to be taken
- * @param snapshotName Name of the snapshot
- * @return the snapshot path.
- * @see ClientProtocol#createSnapshot(String, String)
- */
- public String createSnapshot(String snapshotRoot, String snapshotName)
- throws IOException {
- checkOpen();
- TraceScope scope = Trace.startSpan("createSnapshot", traceSampler);
- try {
- return namenode.createSnapshot(snapshotRoot, snapshotName);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException();
- } finally {
- scope.close();
- }
- }
-
- /**
- * Delete a snapshot of a snapshottable directory.
- *
- * @param snapshotRoot The snapshottable directory that the
- * to-be-deleted snapshot belongs to
- * @param snapshotName The name of the to-be-deleted snapshot
- * @throws IOException
- * @see ClientProtocol#deleteSnapshot(String, String)
- */
- public void deleteSnapshot(String snapshotRoot, String snapshotName)
- throws IOException {
- checkOpen();
- TraceScope scope = Trace.startSpan("deleteSnapshot", traceSampler);
- try {
- namenode.deleteSnapshot(snapshotRoot, snapshotName);
- } catch(RemoteException re) {
- throw re.unwrapRemoteException();
- } finally {
<TRUNCATED>
[11/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and
related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
new file mode 100644
index 0000000..7cfc854
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -0,0 +1,3203 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_READS;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_DROP_BEHIND_WRITES;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CACHE_READAHEAD;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT;
+import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_CLIENT_CONTEXT_DEFAULT;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.Proxy;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.net.SocketAddress;
+import java.net.URI;
+import java.net.UnknownHostException;
+import java.security.GeneralSecurityException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.HashMap;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import javax.net.SocketFactory;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CipherSuite;
+import org.apache.hadoop.crypto.CryptoCodec;
+import org.apache.hadoop.crypto.CryptoInputStream;
+import org.apache.hadoop.crypto.CryptoOutputStream;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProvider.KeyVersion;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension;
+import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.BlockStorageLocation;
+import org.apache.hadoop.fs.CacheFlag;
+import org.apache.hadoop.fs.ContentSummary;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.HdfsBlockLocation;
+import org.apache.hadoop.fs.InvalidPathException;
+import org.apache.hadoop.fs.MD5MD5CRC32CastagnoliFileChecksum;
+import org.apache.hadoop.fs.MD5MD5CRC32FileChecksum;
+import org.apache.hadoop.fs.MD5MD5CRC32GzipFileChecksum;
+import org.apache.hadoop.fs.Options;
+import org.apache.hadoop.fs.Options.ChecksumOpt;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RemoteIterator;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.VolumeId;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.fs.permission.AclEntry;
+import org.apache.hadoop.fs.permission.AclStatus;
+import org.apache.hadoop.fs.permission.FsAction;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.NameNodeProxiesClient.ProxyAndInfo;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.client.impl.LeaseRenewer;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.AclException;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
+import org.apache.hadoop.hdfs.protocol.CacheDirectiveIterator;
+import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
+import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
+import org.apache.hadoop.hdfs.protocol.CachePoolIterator;
+import org.apache.hadoop.hdfs.protocol.ClientProtocol;
+import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
+import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Op;
+import org.apache.hadoop.hdfs.protocol.datatransfer.ReplaceDatanodeOnFailure;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.datatransfer.TrustedChannelResolver;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataTransferSaslUtil;
+import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.SaslDataTransferClient;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.OpBlockChecksumResponseProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
+import org.apache.hadoop.hdfs.util.IOUtilsClient;
+import org.apache.hadoop.io.DataOutputBuffer;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.io.MD5Hash;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.ipc.RpcInvocationHandler;
+import org.apache.hadoop.net.DNS;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenRenewer;
+import org.apache.hadoop.tracing.SpanReceiverHost;
+import org.apache.hadoop.tracing.TraceUtils;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.DataChecksum.Type;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.SamplerBuilder;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import com.google.common.net.InetAddresses;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/********************************************************
+ * DFSClient can connect to a Hadoop Filesystem and
+ * perform basic file tasks. It uses the ClientProtocol
+ * to communicate with a NameNode daemon, and connects
+ * directly to DataNodes to read/write block data.
+ *
+ * Hadoop DFS users should obtain an instance of
+ * DistributedFileSystem, which uses DFSClient to handle
+ * filesystem tasks.
+ *
+ ********************************************************/
+@InterfaceAudience.Private
+public class DFSClient implements java.io.Closeable, RemotePeerFactory,
+ DataEncryptionKeyFactory {
+ public static final Logger LOG = LoggerFactory.getLogger(DFSClient.class);
+ public static final long SERVER_DEFAULTS_VALIDITY_PERIOD = 60 * 60 * 1000L; // 1 hour
+
+ private final Configuration conf;
+ private final DfsClientConf dfsClientConf;
+ final ClientProtocol namenode;
+ /* The service used for delegation tokens */
+ private Text dtService;
+
+ final UserGroupInformation ugi;
+ volatile boolean clientRunning = true;
+ volatile long lastLeaseRenewal;
+ private volatile FsServerDefaults serverDefaults;
+ private volatile long serverDefaultsLastUpdate;
+ final String clientName;
+ final SocketFactory socketFactory;
+ final ReplaceDatanodeOnFailure dtpReplaceDatanodeOnFailure;
+ final FileSystem.Statistics stats;
+ private final String authority;
+ private final Random r = new Random();
+ private SocketAddress[] localInterfaceAddrs;
+ private DataEncryptionKey encryptionKey;
+ final SaslDataTransferClient saslClient;
+ private final CachingStrategy defaultReadCachingStrategy;
+ private final CachingStrategy defaultWriteCachingStrategy;
+ private final ClientContext clientContext;
+
+ private static final DFSHedgedReadMetrics HEDGED_READ_METRIC =
+ new DFSHedgedReadMetrics();
+ private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
+ private final Sampler<?> traceSampler;
+ private final int smallBufferSize;
+
+ public DfsClientConf getConf() {
+ return dfsClientConf;
+ }
+
+ Configuration getConfiguration() {
+ return conf;
+ }
+
+ /**
+ * A map from file names to {@link DFSOutputStream} objects
+ * that are currently being written by this client.
+ * Note that a file can only be written by a single client.
+ */
+ private final Map<Long, DFSOutputStream> filesBeingWritten
+ = new HashMap<Long, DFSOutputStream>();
+
+ /**
+ * Same as this(NameNode.getNNAddress(conf), conf);
+ * @see #DFSClient(InetSocketAddress, Configuration)
+ * @deprecated Deprecated at 0.21
+ */
+ @Deprecated
+ public DFSClient(Configuration conf) throws IOException {
+ this(DFSUtilClient.getNNAddress(conf), conf);
+ }
+
+ public DFSClient(InetSocketAddress address, Configuration conf) throws IOException {
+ this(DFSUtilClient.getNNUri(address), conf);
+ }
+
+ /**
+ * Same as this(nameNodeUri, conf, null);
+ * @see #DFSClient(URI, Configuration, FileSystem.Statistics)
+ */
+ public DFSClient(URI nameNodeUri, Configuration conf
+ ) throws IOException {
+ this(nameNodeUri, conf, null);
+ }
+
+ /**
+ * Same as this(nameNodeUri, null, conf, stats);
+ * @see #DFSClient(URI, ClientProtocol, Configuration, FileSystem.Statistics)
+ */
+ public DFSClient(URI nameNodeUri, Configuration conf,
+ FileSystem.Statistics stats)
+ throws IOException {
+ this(nameNodeUri, null, conf, stats);
+ }
+
+ /**
+ * Create a new DFSClient connected to the given nameNodeUri or rpcNamenode.
+ * If HA is enabled and a positive value is set for
+ * {@link HdfsClientConfigKeys#DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY}
+ * in the configuration, the DFSClient will use
+ * {@link LossyRetryInvocationHandler} as its RetryInvocationHandler.
+ * Otherwise one of nameNodeUri or rpcNamenode must be null.
+ */
+ @VisibleForTesting
+ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode,
+ Configuration conf, FileSystem.Statistics stats)
+ throws IOException {
+ SpanReceiverHost.get(conf, HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX);
+ traceSampler = new SamplerBuilder(TraceUtils.
+ wrapHadoopConf(HdfsClientConfigKeys.DFS_CLIENT_HTRACE_PREFIX, conf))
+ .build();
+ // Copy only the required DFSClient configuration
+ this.dfsClientConf = new DfsClientConf(conf);
+ this.conf = conf;
+ this.stats = stats;
+ this.socketFactory = NetUtils.getSocketFactory(conf, ClientProtocol.class);
+ this.dtpReplaceDatanodeOnFailure = ReplaceDatanodeOnFailure.get(conf);
+ this.smallBufferSize = DFSUtilClient.getSmallBufferSize(conf);
+
+ this.ugi = UserGroupInformation.getCurrentUser();
+
+ this.authority = nameNodeUri == null? "null": nameNodeUri.getAuthority();
+ this.clientName = "DFSClient_" + dfsClientConf.getTaskId() + "_" +
+ ThreadLocalRandom.current().nextInt() + "_" +
+ Thread.currentThread().getId();
+ int numResponseToDrop = conf.getInt(
+ HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY,
+ HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_DEFAULT);
+ ProxyAndInfo<ClientProtocol> proxyInfo = null;
+ AtomicBoolean nnFallbackToSimpleAuth = new AtomicBoolean(false);
+
+ if (numResponseToDrop > 0) {
+ // This case is used for testing.
+ LOG.warn(HdfsClientConfigKeys.DFS_CLIENT_TEST_DROP_NAMENODE_RESPONSE_NUM_KEY
+ + " is set to " + numResponseToDrop
+ + ", this hacked client will proactively drop responses");
+ proxyInfo = NameNodeProxiesClient.createProxyWithLossyRetryHandler(conf,
+ nameNodeUri, ClientProtocol.class, numResponseToDrop,
+ nnFallbackToSimpleAuth);
+ }
+
+ if (proxyInfo != null) {
+ this.dtService = proxyInfo.getDelegationTokenService();
+ this.namenode = proxyInfo.getProxy();
+ } else if (rpcNamenode != null) {
+ // This case is used for testing.
+ Preconditions.checkArgument(nameNodeUri == null);
+ this.namenode = rpcNamenode;
+ dtService = null;
+ } else {
+ Preconditions.checkArgument(nameNodeUri != null,
+ "null URI");
+ proxyInfo = NameNodeProxiesClient.createProxyWithClientProtocol(conf,
+ nameNodeUri, nnFallbackToSimpleAuth);
+ this.dtService = proxyInfo.getDelegationTokenService();
+ this.namenode = proxyInfo.getProxy();
+ }
+
+ String localInterfaces[] =
+ conf.getTrimmedStrings(HdfsClientConfigKeys.DFS_CLIENT_LOCAL_INTERFACES);
+ localInterfaceAddrs = getLocalInterfaceAddrs(localInterfaces);
+ if (LOG.isDebugEnabled() && 0 != localInterfaces.length) {
+ LOG.debug("Using local interfaces [" +
+ Joiner.on(',').join(localInterfaces)+ "] with addresses [" +
+ Joiner.on(',').join(localInterfaceAddrs) + "]");
+ }
+
+ Boolean readDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_READS) == null) ?
+ null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_READS, false);
+ Long readahead = (conf.get(DFS_CLIENT_CACHE_READAHEAD) == null) ?
+ null : conf.getLong(DFS_CLIENT_CACHE_READAHEAD, 0);
+ Boolean writeDropBehind = (conf.get(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES) == null) ?
+ null : conf.getBoolean(DFS_CLIENT_CACHE_DROP_BEHIND_WRITES, false);
+ this.defaultReadCachingStrategy =
+ new CachingStrategy(readDropBehind, readahead);
+ this.defaultWriteCachingStrategy =
+ new CachingStrategy(writeDropBehind, readahead);
+ this.clientContext = ClientContext.get(
+ conf.get(DFS_CLIENT_CONTEXT, DFS_CLIENT_CONTEXT_DEFAULT),
+ dfsClientConf);
+
+ if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) {
+ this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize());
+ }
+ this.saslClient = new SaslDataTransferClient(
+ conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf),
+ TrustedChannelResolver.getInstance(conf), nnFallbackToSimpleAuth);
+ }
+
+ /**
+ * Return the socket addresses to use with each configured
+ * local interface. Local interfaces may be specified by IP
+ * address, IP address range using CIDR notation, interface
+ * name (e.g. eth0) or sub-interface name (e.g. eth0:0).
+ * The socket addresses consist of the IPs for the interfaces
+ * and the ephemeral port (port 0). If an IP, IP range, or
+ * interface name matches an interface with sub-interfaces
+ * only the IP of the interface is used. Sub-interfaces can
+ * be used by specifying them explicitly (by IP or name).
+ *
+ * @return SocketAddresses for the configured local interfaces,
+ * or an empty array if none are configured
+ * @throws UnknownHostException if a given interface name is invalid
+ */
+ private static SocketAddress[] getLocalInterfaceAddrs(
+ String interfaceNames[]) throws UnknownHostException {
+ List<SocketAddress> localAddrs = new ArrayList<SocketAddress>();
+ for (String interfaceName : interfaceNames) {
+ if (InetAddresses.isInetAddress(interfaceName)) {
+ localAddrs.add(new InetSocketAddress(interfaceName, 0));
+ } else if (NetUtils.isValidSubnet(interfaceName)) {
+ for (InetAddress addr : NetUtils.getIPs(interfaceName, false)) {
+ localAddrs.add(new InetSocketAddress(addr, 0));
+ }
+ } else {
+ for (String ip : DNS.getIPs(interfaceName, false)) {
+ localAddrs.add(new InetSocketAddress(ip, 0));
+ }
+ }
+ }
+ return localAddrs.toArray(new SocketAddress[localAddrs.size()]);
+ }
+
+ /**
+ * Select one of the configured local interfaces at random. We use a random
+ * interface because other policies like round-robin are less effective
+ * given that we cache connections to datanodes.
+ *
+ * @return one of the local interface addresses at random, or null if no
+ * local interfaces are configured
+ */
+ SocketAddress getRandomLocalInterfaceAddr() {
+ if (localInterfaceAddrs.length == 0) {
+ return null;
+ }
+ final int idx = r.nextInt(localInterfaceAddrs.length);
+ final SocketAddress addr = localInterfaceAddrs[idx];
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Using local interface " + addr);
+ }
+ return addr;
+ }
+
+ /**
+ * Return the timeout that clients should use when writing to datanodes.
+ * @param numNodes the number of nodes in the pipeline.
+ */
+ int getDatanodeWriteTimeout(int numNodes) {
+ final int t = dfsClientConf.getDatanodeSocketWriteTimeout();
+ return t > 0? t + HdfsConstants.WRITE_TIMEOUT_EXTENSION*numNodes: 0;
+ }
+
+ int getDatanodeReadTimeout(int numNodes) {
+ final int t = dfsClientConf.getSocketTimeout();
+ return t > 0? HdfsConstants.READ_TIMEOUT_EXTENSION*numNodes + t: 0;
+ }
+
+ @VisibleForTesting
+ public String getClientName() {
+ return clientName;
+ }
+
+ void checkOpen() throws IOException {
+ if (!clientRunning) {
+ IOException result = new IOException("Filesystem closed");
+ throw result;
+ }
+ }
+
+ /** Return the lease renewer instance. The renewer thread won't start
+ * until the first output stream is created. The same instance will
+ * be returned until all output streams are closed.
+ */
+ public LeaseRenewer getLeaseRenewer() throws IOException {
+ return LeaseRenewer.getInstance(authority, ugi, this);
+ }
+
+ /** Get a lease and start automatic renewal */
+ private void beginFileLease(final long inodeId, final DFSOutputStream out)
+ throws IOException {
+ getLeaseRenewer().put(inodeId, out, this);
+ }
+
+ /** Stop renewal of lease for the file. */
+ void endFileLease(final long inodeId) throws IOException {
+ getLeaseRenewer().closeFile(inodeId, this);
+ }
+
+
+ /** Put a file. Only called from LeaseRenewer, where proper locking is
+ * enforced to consistently update its local dfsclients array and
+ * client's filesBeingWritten map.
+ */
+ public void putFileBeingWritten(final long inodeId, final DFSOutputStream out) {
+ synchronized(filesBeingWritten) {
+ filesBeingWritten.put(inodeId, out);
+ // update the last lease renewal time only when there was no
+ // writes. once there is one write stream open, the lease renewer
+ // thread keeps it updated well with in anyone's expiration time.
+ if (lastLeaseRenewal == 0) {
+ updateLastLeaseRenewal();
+ }
+ }
+ }
+
+ /** Remove a file. Only called from LeaseRenewer. */
+ public void removeFileBeingWritten(final long inodeId) {
+ synchronized(filesBeingWritten) {
+ filesBeingWritten.remove(inodeId);
+ if (filesBeingWritten.isEmpty()) {
+ lastLeaseRenewal = 0;
+ }
+ }
+ }
+
+ /** Is file-being-written map empty? */
+ public boolean isFilesBeingWrittenEmpty() {
+ synchronized(filesBeingWritten) {
+ return filesBeingWritten.isEmpty();
+ }
+ }
+
+ /** @return true if the client is running */
+ public boolean isClientRunning() {
+ return clientRunning;
+ }
+
+ long getLastLeaseRenewal() {
+ return lastLeaseRenewal;
+ }
+
+ void updateLastLeaseRenewal() {
+ synchronized(filesBeingWritten) {
+ if (filesBeingWritten.isEmpty()) {
+ return;
+ }
+ lastLeaseRenewal = Time.monotonicNow();
+ }
+ }
+
+ /**
+ * Renew leases.
+ * @return true if lease was renewed. May return false if this
+ * client has been closed or has no files open.
+ **/
+ public boolean renewLease() throws IOException {
+ if (clientRunning && !isFilesBeingWrittenEmpty()) {
+ try {
+ namenode.renewLease(clientName);
+ updateLastLeaseRenewal();
+ return true;
+ } catch (IOException e) {
+ // Abort if the lease has already expired.
+ final long elapsed = Time.monotonicNow() - getLastLeaseRenewal();
+ if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
+ LOG.warn("Failed to renew lease for " + clientName + " for "
+ + (elapsed/1000) + " seconds (>= hard-limit ="
+ + (HdfsConstants.LEASE_HARDLIMIT_PERIOD / 1000) + " seconds.) "
+ + "Closing all files being written ...", e);
+ closeAllFilesBeingWritten(true);
+ } else {
+ // Let the lease renewer handle it and retry.
+ throw e;
+ }
+ }
+ }
+ return false;
+ }
+
+ /**
+ * Close connections the Namenode.
+ */
+ void closeConnectionToNamenode() {
+ RPC.stopProxy(namenode);
+ }
+
+ /** Close/abort all files being written. */
+ public void closeAllFilesBeingWritten(final boolean abort) {
+ for(;;) {
+ final long inodeId;
+ final DFSOutputStream out;
+ synchronized(filesBeingWritten) {
+ if (filesBeingWritten.isEmpty()) {
+ return;
+ }
+ inodeId = filesBeingWritten.keySet().iterator().next();
+ out = filesBeingWritten.remove(inodeId);
+ }
+ if (out != null) {
+ try {
+ if (abort) {
+ out.abort();
+ } else {
+ out.close();
+ }
+ } catch(IOException ie) {
+ LOG.error("Failed to " + (abort ? "abort" : "close") + " file: "
+ + out.getSrc() + " with inode: " + inodeId, ie);
+ }
+ }
+ }
+ }
+
+ /**
+ * Close the file system, abandoning all of the leases and files being
+ * created and close connections to the namenode.
+ */
+ @Override
+ public synchronized void close() throws IOException {
+ if(clientRunning) {
+ closeAllFilesBeingWritten(false);
+ clientRunning = false;
+ getLeaseRenewer().closeClient(this);
+ // close connections to the namenode
+ closeConnectionToNamenode();
+ }
+ }
+
+ /**
+ * Close all open streams, abandoning all of the leases and files being
+ * created.
+ * @param abort whether streams should be gracefully closed
+ */
+ public void closeOutputStreams(boolean abort) {
+ if (clientRunning) {
+ closeAllFilesBeingWritten(abort);
+ }
+ }
+
+ /**
+ * @see ClientProtocol#getPreferredBlockSize(String)
+ */
+ public long getBlockSize(String f) throws IOException {
+ TraceScope scope = getPathTraceScope("getBlockSize", f);
+ try {
+ return namenode.getPreferredBlockSize(f);
+ } catch (IOException ie) {
+ LOG.warn("Problem getting block size", ie);
+ throw ie;
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Get server default values for a number of configuration params.
+ * @see ClientProtocol#getServerDefaults()
+ */
+ public FsServerDefaults getServerDefaults() throws IOException {
+ long now = Time.monotonicNow();
+ if ((serverDefaults == null) ||
+ (now - serverDefaultsLastUpdate > SERVER_DEFAULTS_VALIDITY_PERIOD)) {
+ serverDefaults = namenode.getServerDefaults();
+ serverDefaultsLastUpdate = now;
+ }
+ assert serverDefaults != null;
+ return serverDefaults;
+ }
+
+ /**
+ * Get a canonical token service name for this client's tokens. Null should
+ * be returned if the client is not using tokens.
+ * @return the token service for the client
+ */
+ @InterfaceAudience.LimitedPrivate( { "HDFS" })
+ public String getCanonicalServiceName() {
+ return (dtService != null) ? dtService.toString() : null;
+ }
+
+ /**
+ * @see ClientProtocol#getDelegationToken(Text)
+ */
+ public Token<DelegationTokenIdentifier> getDelegationToken(Text renewer)
+ throws IOException {
+ assert dtService != null;
+ TraceScope scope = Trace.startSpan("getDelegationToken", traceSampler);
+ try {
+ Token<DelegationTokenIdentifier> token =
+ namenode.getDelegationToken(renewer);
+ if (token != null) {
+ token.setService(this.dtService);
+ LOG.info("Created " + DelegationTokenIdentifier.stringifyToken(token));
+ } else {
+ LOG.info("Cannot get delegation token from " + renewer);
+ }
+ return token;
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Renew a delegation token
+ * @param token the token to renew
+ * @return the new expiration time
+ * @throws InvalidToken
+ * @throws IOException
+ * @deprecated Use Token.renew instead.
+ */
+ @Deprecated
+ public long renewDelegationToken(Token<DelegationTokenIdentifier> token)
+ throws InvalidToken, IOException {
+ LOG.info("Renewing " + DelegationTokenIdentifier.stringifyToken(token));
+ try {
+ return token.renew(conf);
+ } catch (InterruptedException ie) {
+ throw new RuntimeException("caught interrupted", ie);
+ } catch (RemoteException re) {
+ throw re.unwrapRemoteException(InvalidToken.class,
+ AccessControlException.class);
+ }
+ }
+
+ /**
+ * Cancel a delegation token
+ * @param token the token to cancel
+ * @throws InvalidToken
+ * @throws IOException
+ * @deprecated Use Token.cancel instead.
+ */
+ @Deprecated
+ public void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
+ throws InvalidToken, IOException {
+ LOG.info("Cancelling " + DelegationTokenIdentifier.stringifyToken(token));
+ try {
+ token.cancel(conf);
+ } catch (InterruptedException ie) {
+ throw new RuntimeException("caught interrupted", ie);
+ } catch (RemoteException re) {
+ throw re.unwrapRemoteException(InvalidToken.class,
+ AccessControlException.class);
+ }
+ }
+
+ @InterfaceAudience.Private
+ public static class Renewer extends TokenRenewer {
+
+ static {
+ //Ensure that HDFS Configuration files are loaded before trying to use
+ // the renewer.
+ HdfsConfigurationLoader.init();
+ }
+
+ @Override
+ public boolean handleKind(Text kind) {
+ return DelegationTokenIdentifier.HDFS_DELEGATION_KIND.equals(kind);
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public long renew(Token<?> token, Configuration conf) throws IOException {
+ Token<DelegationTokenIdentifier> delToken =
+ (Token<DelegationTokenIdentifier>) token;
+ ClientProtocol nn = getNNProxy(delToken, conf);
+ try {
+ return nn.renewDelegationToken(delToken);
+ } catch (RemoteException re) {
+ throw re.unwrapRemoteException(InvalidToken.class,
+ AccessControlException.class);
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ @Override
+ public void cancel(Token<?> token, Configuration conf) throws IOException {
+ Token<DelegationTokenIdentifier> delToken =
+ (Token<DelegationTokenIdentifier>) token;
+ LOG.info("Cancelling " +
+ DelegationTokenIdentifier.stringifyToken(delToken));
+ ClientProtocol nn = getNNProxy(delToken, conf);
+ try {
+ nn.cancelDelegationToken(delToken);
+ } catch (RemoteException re) {
+ throw re.unwrapRemoteException(InvalidToken.class,
+ AccessControlException.class);
+ }
+ }
+
+ private static ClientProtocol getNNProxy(
+ Token<DelegationTokenIdentifier> token, Configuration conf)
+ throws IOException {
+ URI uri = HAUtilClient.getServiceUriFromToken(
+ HdfsConstants.HDFS_URI_SCHEME, token);
+ if (HAUtilClient.isTokenForLogicalUri(token) &&
+ !HAUtilClient.isLogicalUri(conf, uri)) {
+ // If the token is for a logical nameservice, but the configuration
+ // we have disagrees about that, we can't actually renew it.
+ // This can be the case in MR, for example, if the RM doesn't
+ // have all of the HA clusters configured in its configuration.
+ throw new IOException("Unable to map logical nameservice URI '" +
+ uri + "' to a NameNode. Local configuration does not have " +
+ "a failover proxy provider configured.");
+ }
+
+ ProxyAndInfo<ClientProtocol> info =
+ NameNodeProxiesClient.createProxyWithClientProtocol(conf, uri, null);
+ assert info.getDelegationTokenService().equals(token.getService()) :
+ "Returned service '" + info.getDelegationTokenService().toString() +
+ "' doesn't match expected service '" +
+ token.getService().toString() + "'";
+
+ return info.getProxy();
+ }
+
+ @Override
+ public boolean isManaged(Token<?> token) throws IOException {
+ return true;
+ }
+
+ }
+
+ /**
+ * Report corrupt blocks that were discovered by the client.
+ * @see ClientProtocol#reportBadBlocks(LocatedBlock[])
+ */
+ public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
+ namenode.reportBadBlocks(blocks);
+ }
+
+ public LocatedBlocks getLocatedBlocks(String src, long start)
+ throws IOException {
+ return getLocatedBlocks(src, start, dfsClientConf.getPrefetchSize());
+ }
+
+ /*
+ * This is just a wrapper around callGetBlockLocations, but non-static so that
+ * we can stub it out for tests.
+ */
+ @VisibleForTesting
+ public LocatedBlocks getLocatedBlocks(String src, long start, long length)
+ throws IOException {
+ TraceScope scope = getPathTraceScope("getBlockLocations", src);
+ try {
+ return callGetBlockLocations(namenode, src, start, length);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * @see ClientProtocol#getBlockLocations(String, long, long)
+ */
+ static LocatedBlocks callGetBlockLocations(ClientProtocol namenode,
+ String src, long start, long length)
+ throws IOException {
+ try {
+ return namenode.getBlockLocations(src, start, length);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ FileNotFoundException.class,
+ UnresolvedPathException.class);
+ }
+ }
+
+ /**
+ * Recover a file's lease
+ * @param src a file's path
+ * @return true if the file is already closed
+ * @throws IOException
+ */
+ boolean recoverLease(String src) throws IOException {
+ checkOpen();
+
+ TraceScope scope = getPathTraceScope("recoverLease", src);
+ try {
+ return namenode.recoverLease(src, clientName);
+ } catch (RemoteException re) {
+ throw re.unwrapRemoteException(FileNotFoundException.class,
+ AccessControlException.class,
+ UnresolvedPathException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Get block location info about file
+ *
+ * getBlockLocations() returns a list of hostnames that store
+ * data for a specific file region. It returns a set of hostnames
+ * for every block within the indicated region.
+ *
+ * This function is very useful when writing code that considers
+ * data-placement when performing operations. For example, the
+ * MapReduce system tries to schedule tasks on the same machines
+ * as the data-block the task processes.
+ */
+ public BlockLocation[] getBlockLocations(String src, long start,
+ long length) throws IOException, UnresolvedLinkException {
+ TraceScope scope = getPathTraceScope("getBlockLocations", src);
+ try {
+ LocatedBlocks blocks = getLocatedBlocks(src, start, length);
+ BlockLocation[] locations = DFSUtilClient.locatedBlocks2Locations(blocks);
+ HdfsBlockLocation[] hdfsLocations = new HdfsBlockLocation[locations.length];
+ for (int i = 0; i < locations.length; i++) {
+ hdfsLocations[i] = new HdfsBlockLocation(locations[i], blocks.get(i));
+ }
+ return hdfsLocations;
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Get block location information about a list of {@link HdfsBlockLocation}.
+ * Used by {@link DistributedFileSystem#getFileBlockStorageLocations(List)} to
+ * get {@link BlockStorageLocation}s for blocks returned by
+ * {@link DistributedFileSystem#getFileBlockLocations(org.apache.hadoop.fs.FileStatus, long, long)}
+ * .
+ *
+ * This is done by making a round of RPCs to the associated datanodes, asking
+ * the volume of each block replica. The returned array of
+ * {@link BlockStorageLocation} expose this information as a
+ * {@link VolumeId}.
+ *
+ * @param blockLocations
+ * target blocks on which to query volume location information
+ * @return volumeBlockLocations original block array augmented with additional
+ * volume location information for each replica.
+ */
+ public BlockStorageLocation[] getBlockStorageLocations(
+ List<BlockLocation> blockLocations) throws IOException,
+ UnsupportedOperationException, InvalidBlockTokenException {
+ if (!getConf().isHdfsBlocksMetadataEnabled()) {
+ throw new UnsupportedOperationException("Datanode-side support for " +
+ "getVolumeBlockLocations() must also be enabled in the client " +
+ "configuration.");
+ }
+ // Downcast blockLocations and fetch out required LocatedBlock(s)
+ List<LocatedBlock> blocks = new ArrayList<LocatedBlock>();
+ for (BlockLocation loc : blockLocations) {
+ if (!(loc instanceof HdfsBlockLocation)) {
+ throw new ClassCastException("DFSClient#getVolumeBlockLocations " +
+ "expected to be passed HdfsBlockLocations");
+ }
+ HdfsBlockLocation hdfsLoc = (HdfsBlockLocation) loc;
+ blocks.add(hdfsLoc.getLocatedBlock());
+ }
+
+ // Re-group the LocatedBlocks to be grouped by datanodes, with the values
+ // a list of the LocatedBlocks on the datanode.
+ Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks =
+ new LinkedHashMap<DatanodeInfo, List<LocatedBlock>>();
+ for (LocatedBlock b : blocks) {
+ for (DatanodeInfo info : b.getLocations()) {
+ if (!datanodeBlocks.containsKey(info)) {
+ datanodeBlocks.put(info, new ArrayList<LocatedBlock>());
+ }
+ List<LocatedBlock> l = datanodeBlocks.get(info);
+ l.add(b);
+ }
+ }
+
+ // Make RPCs to the datanodes to get volume locations for its replicas
+ TraceScope scope =
+ Trace.startSpan("getBlockStorageLocations", traceSampler);
+ Map<DatanodeInfo, HdfsBlocksMetadata> metadatas;
+ try {
+ metadatas = BlockStorageLocationUtil.
+ queryDatanodesForHdfsBlocksMetadata(conf, datanodeBlocks,
+ getConf().getFileBlockStorageLocationsNumThreads(),
+ getConf().getFileBlockStorageLocationsTimeoutMs(),
+ getConf().isConnectToDnViaHostname());
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("metadata returned: "
+ + Joiner.on("\n").withKeyValueSeparator("=").join(metadatas));
+ }
+ } finally {
+ scope.close();
+ }
+
+ // Regroup the returned VolumeId metadata to again be grouped by
+ // LocatedBlock rather than by datanode
+ Map<LocatedBlock, List<VolumeId>> blockVolumeIds = BlockStorageLocationUtil
+ .associateVolumeIdsWithBlocks(blocks, metadatas);
+
+ // Combine original BlockLocations with new VolumeId information
+ BlockStorageLocation[] volumeBlockLocations = BlockStorageLocationUtil
+ .convertToVolumeBlockLocations(blocks, blockVolumeIds);
+
+ return volumeBlockLocations;
+ }
+
+ /**
+ * Decrypts a EDEK by consulting the KeyProvider.
+ */
+ private KeyVersion decryptEncryptedDataEncryptionKey(FileEncryptionInfo
+ feInfo) throws IOException {
+ TraceScope scope = Trace.startSpan("decryptEDEK", traceSampler);
+ try {
+ KeyProvider provider = getKeyProvider();
+ if (provider == null) {
+ throw new IOException("No KeyProvider is configured, cannot access" +
+ " an encrypted file");
+ }
+ EncryptedKeyVersion ekv = EncryptedKeyVersion.createForDecryption(
+ feInfo.getKeyName(), feInfo.getEzKeyVersionName(), feInfo.getIV(),
+ feInfo.getEncryptedDataEncryptionKey());
+ try {
+ KeyProviderCryptoExtension cryptoProvider = KeyProviderCryptoExtension
+ .createKeyProviderCryptoExtension(provider);
+ return cryptoProvider.decryptEncryptedKey(ekv);
+ } catch (GeneralSecurityException e) {
+ throw new IOException(e);
+ }
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Obtain the crypto protocol version from the provided FileEncryptionInfo,
+ * checking to see if this version is supported by.
+ *
+ * @param feInfo FileEncryptionInfo
+ * @return CryptoProtocolVersion from the feInfo
+ * @throws IOException if the protocol version is unsupported.
+ */
+ private static CryptoProtocolVersion getCryptoProtocolVersion
+ (FileEncryptionInfo feInfo) throws IOException {
+ final CryptoProtocolVersion version = feInfo.getCryptoProtocolVersion();
+ if (!CryptoProtocolVersion.supports(version)) {
+ throw new IOException("Client does not support specified " +
+ "CryptoProtocolVersion " + version.getDescription() + " version " +
+ "number" + version.getVersion());
+ }
+ return version;
+ }
+
+ /**
+ * Obtain a CryptoCodec based on the CipherSuite set in a FileEncryptionInfo
+ * and the available CryptoCodecs configured in the Configuration.
+ *
+ * @param conf Configuration
+ * @param feInfo FileEncryptionInfo
+ * @return CryptoCodec
+ * @throws IOException if no suitable CryptoCodec for the CipherSuite is
+ * available.
+ */
+ private static CryptoCodec getCryptoCodec(Configuration conf,
+ FileEncryptionInfo feInfo) throws IOException {
+ final CipherSuite suite = feInfo.getCipherSuite();
+ if (suite.equals(CipherSuite.UNKNOWN)) {
+ throw new IOException("NameNode specified unknown CipherSuite with ID "
+ + suite.getUnknownValue() + ", cannot instantiate CryptoCodec.");
+ }
+ final CryptoCodec codec = CryptoCodec.getInstance(conf, suite);
+ if (codec == null) {
+ throw new UnknownCipherSuiteException(
+ "No configuration found for the cipher suite "
+ + suite.getConfigSuffix() + " prefixed with "
+ + HADOOP_SECURITY_CRYPTO_CODEC_CLASSES_KEY_PREFIX
+ + ". Please see the example configuration "
+ + "hadoop.security.crypto.codec.classes.EXAMPLECIPHERSUITE "
+ + "at core-default.xml for details.");
+ }
+ return codec;
+ }
+
+ /**
+ * Wraps the stream in a CryptoInputStream if the underlying file is
+ * encrypted.
+ */
+ public HdfsDataInputStream createWrappedInputStream(DFSInputStream dfsis)
+ throws IOException {
+ final FileEncryptionInfo feInfo = dfsis.getFileEncryptionInfo();
+ if (feInfo != null) {
+ // File is encrypted, wrap the stream in a crypto stream.
+ // Currently only one version, so no special logic based on the version #
+ getCryptoProtocolVersion(feInfo);
+ final CryptoCodec codec = getCryptoCodec(conf, feInfo);
+ final KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
+ final CryptoInputStream cryptoIn =
+ new CryptoInputStream(dfsis, codec, decrypted.getMaterial(),
+ feInfo.getIV());
+ return new HdfsDataInputStream(cryptoIn);
+ } else {
+ // No FileEncryptionInfo so no encryption.
+ return new HdfsDataInputStream(dfsis);
+ }
+ }
+
+ /**
+ * Wraps the stream in a CryptoOutputStream if the underlying file is
+ * encrypted.
+ */
+ public HdfsDataOutputStream createWrappedOutputStream(DFSOutputStream dfsos,
+ FileSystem.Statistics statistics) throws IOException {
+ return createWrappedOutputStream(dfsos, statistics, 0);
+ }
+
+ /**
+ * Wraps the stream in a CryptoOutputStream if the underlying file is
+ * encrypted.
+ */
+ public HdfsDataOutputStream createWrappedOutputStream(DFSOutputStream dfsos,
+ FileSystem.Statistics statistics, long startPos) throws IOException {
+ final FileEncryptionInfo feInfo = dfsos.getFileEncryptionInfo();
+ if (feInfo != null) {
+ // File is encrypted, wrap the stream in a crypto stream.
+ // Currently only one version, so no special logic based on the version #
+ getCryptoProtocolVersion(feInfo);
+ final CryptoCodec codec = getCryptoCodec(conf, feInfo);
+ KeyVersion decrypted = decryptEncryptedDataEncryptionKey(feInfo);
+ final CryptoOutputStream cryptoOut =
+ new CryptoOutputStream(dfsos, codec,
+ decrypted.getMaterial(), feInfo.getIV(), startPos);
+ return new HdfsDataOutputStream(cryptoOut, statistics, startPos);
+ } else {
+ // No FileEncryptionInfo present so no encryption.
+ return new HdfsDataOutputStream(dfsos, statistics, startPos);
+ }
+ }
+
+ public DFSInputStream open(String src)
+ throws IOException, UnresolvedLinkException {
+ return open(src, dfsClientConf.getIoBufferSize(), true, null);
+ }
+
+ /**
+ * Create an input stream that obtains a nodelist from the
+ * namenode, and then reads from all the right places. Creates
+ * inner subclass of InputStream that does the right out-of-band
+ * work.
+ * @deprecated Use {@link #open(String, int, boolean)} instead.
+ */
+ @Deprecated
+ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum,
+ FileSystem.Statistics stats)
+ throws IOException, UnresolvedLinkException {
+ return open(src, buffersize, verifyChecksum);
+ }
+
+
+ /**
+ * Create an input stream that obtains a nodelist from the
+ * namenode, and then reads from all the right places. Creates
+ * inner subclass of InputStream that does the right out-of-band
+ * work.
+ */
+ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum)
+ throws IOException, UnresolvedLinkException {
+ checkOpen();
+ // Get block info from namenode
+ TraceScope scope = getPathTraceScope("newDFSInputStream", src);
+ try {
+ return new DFSInputStream(this, src, verifyChecksum, null);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Get the namenode associated with this DFSClient object
+ * @return the namenode associated with this DFSClient object
+ */
+ public ClientProtocol getNamenode() {
+ return namenode;
+ }
+
+ /**
+ * Call {@link #create(String, boolean, short, long, Progressable)} with
+ * default <code>replication</code> and <code>blockSize<code> and null <code>
+ * progress</code>.
+ */
+ public OutputStream create(String src, boolean overwrite)
+ throws IOException {
+ return create(src, overwrite, dfsClientConf.getDefaultReplication(),
+ dfsClientConf.getDefaultBlockSize(), null);
+ }
+
+ /**
+ * Call {@link #create(String, boolean, short, long, Progressable)} with
+ * default <code>replication</code> and <code>blockSize<code>.
+ */
+ public OutputStream create(String src,
+ boolean overwrite,
+ Progressable progress) throws IOException {
+ return create(src, overwrite, dfsClientConf.getDefaultReplication(),
+ dfsClientConf.getDefaultBlockSize(), progress);
+ }
+
+ /**
+ * Call {@link #create(String, boolean, short, long, Progressable)} with
+ * null <code>progress</code>.
+ */
+ public OutputStream create(String src,
+ boolean overwrite,
+ short replication,
+ long blockSize) throws IOException {
+ return create(src, overwrite, replication, blockSize, null);
+ }
+
+ /**
+ * Call {@link #create(String, boolean, short, long, Progressable, int)}
+ * with default bufferSize.
+ */
+ public OutputStream create(String src, boolean overwrite, short replication,
+ long blockSize, Progressable progress) throws IOException {
+ return create(src, overwrite, replication, blockSize, progress,
+ dfsClientConf.getIoBufferSize());
+ }
+
+ /**
+ * Call {@link #create(String, FsPermission, EnumSet, short, long,
+ * Progressable, int, ChecksumOpt)} with default <code>permission</code>
+ * {@link FsPermission#getFileDefault()}.
+ *
+ * @param src File name
+ * @param overwrite overwrite an existing file if true
+ * @param replication replication factor for the file
+ * @param blockSize maximum block size
+ * @param progress interface for reporting client progress
+ * @param buffersize underlying buffersize
+ *
+ * @return output stream
+ */
+ public OutputStream create(String src,
+ boolean overwrite,
+ short replication,
+ long blockSize,
+ Progressable progress,
+ int buffersize)
+ throws IOException {
+ return create(src, FsPermission.getFileDefault(),
+ overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
+ : EnumSet.of(CreateFlag.CREATE), replication, blockSize, progress,
+ buffersize, null);
+ }
+
+ /**
+ * Call {@link #create(String, FsPermission, EnumSet, boolean, short,
+ * long, Progressable, int, ChecksumOpt)} with <code>createParent</code>
+ * set to true.
+ */
+ public DFSOutputStream create(String src,
+ FsPermission permission,
+ EnumSet<CreateFlag> flag,
+ short replication,
+ long blockSize,
+ Progressable progress,
+ int buffersize,
+ ChecksumOpt checksumOpt)
+ throws IOException {
+ return create(src, permission, flag, true,
+ replication, blockSize, progress, buffersize, checksumOpt, null);
+ }
+
+ /**
+ * Create a new dfs file with the specified block replication
+ * with write-progress reporting and return an output stream for writing
+ * into the file.
+ *
+ * @param src File name
+ * @param permission The permission of the directory being created.
+ * If null, use default permission {@link FsPermission#getFileDefault()}
+ * @param flag indicates create a new file or create/overwrite an
+ * existing file or append to an existing file
+ * @param createParent create missing parent directory if true
+ * @param replication block replication
+ * @param blockSize maximum block size
+ * @param progress interface for reporting client progress
+ * @param buffersize underlying buffer size
+ * @param checksumOpt checksum options
+ *
+ * @return output stream
+ *
+ * @see ClientProtocol#create for detailed description of exceptions thrown
+ */
+ public DFSOutputStream create(String src,
+ FsPermission permission,
+ EnumSet<CreateFlag> flag,
+ boolean createParent,
+ short replication,
+ long blockSize,
+ Progressable progress,
+ int buffersize,
+ ChecksumOpt checksumOpt) throws IOException {
+ return create(src, permission, flag, createParent, replication, blockSize,
+ progress, buffersize, checksumOpt, null);
+ }
+
+ private FsPermission applyUMask(FsPermission permission) {
+ if (permission == null) {
+ permission = FsPermission.getFileDefault();
+ }
+ return permission.applyUMask(dfsClientConf.getUMask());
+ }
+
+ /**
+ * Same as {@link #create(String, FsPermission, EnumSet, boolean, short, long,
+ * Progressable, int, ChecksumOpt)} with the addition of favoredNodes that is
+ * a hint to where the namenode should place the file blocks.
+ * The favored nodes hint is not persisted in HDFS. Hence it may be honored
+ * at the creation time only. HDFS could move the blocks during balancing or
+ * replication, to move the blocks from favored nodes. A value of null means
+ * no favored nodes for this create
+ */
+ public DFSOutputStream create(String src,
+ FsPermission permission,
+ EnumSet<CreateFlag> flag,
+ boolean createParent,
+ short replication,
+ long blockSize,
+ Progressable progress,
+ int buffersize,
+ ChecksumOpt checksumOpt,
+ InetSocketAddress[] favoredNodes) throws IOException {
+ checkOpen();
+ final FsPermission masked = applyUMask(permission);
+ if(LOG.isDebugEnabled()) {
+ LOG.debug(src + ": masked=" + masked);
+ }
+ final DFSOutputStream result = DFSOutputStream.newStreamForCreate(this,
+ src, masked, flag, createParent, replication, blockSize, progress,
+ buffersize, dfsClientConf.createChecksum(checksumOpt),
+ getFavoredNodesStr(favoredNodes));
+ beginFileLease(result.getFileId(), result);
+ return result;
+ }
+
+ private String[] getFavoredNodesStr(InetSocketAddress[] favoredNodes) {
+ String[] favoredNodeStrs = null;
+ if (favoredNodes != null) {
+ favoredNodeStrs = new String[favoredNodes.length];
+ for (int i = 0; i < favoredNodes.length; i++) {
+ favoredNodeStrs[i] =
+ favoredNodes[i].getHostName() + ":"
+ + favoredNodes[i].getPort();
+ }
+ }
+ return favoredNodeStrs;
+ }
+
+ /**
+ * Append to an existing file if {@link CreateFlag#APPEND} is present
+ */
+ private DFSOutputStream primitiveAppend(String src, EnumSet<CreateFlag> flag,
+ int buffersize, Progressable progress) throws IOException {
+ if (flag.contains(CreateFlag.APPEND)) {
+ HdfsFileStatus stat = getFileInfo(src);
+ if (stat == null) { // No file to append to
+ // New file needs to be created if create option is present
+ if (!flag.contains(CreateFlag.CREATE)) {
+ throw new FileNotFoundException("failed to append to non-existent file "
+ + src + " on client " + clientName);
+ }
+ return null;
+ }
+ return callAppend(src, buffersize, flag, progress, null);
+ }
+ return null;
+ }
+
+ /**
+ * Same as {{@link #create(String, FsPermission, EnumSet, short, long,
+ * Progressable, int, ChecksumOpt)} except that the permission
+ * is absolute (ie has already been masked with umask.
+ */
+ public DFSOutputStream primitiveCreate(String src,
+ FsPermission absPermission,
+ EnumSet<CreateFlag> flag,
+ boolean createParent,
+ short replication,
+ long blockSize,
+ Progressable progress,
+ int buffersize,
+ ChecksumOpt checksumOpt)
+ throws IOException, UnresolvedLinkException {
+ checkOpen();
+ CreateFlag.validate(flag);
+ DFSOutputStream result = primitiveAppend(src, flag, buffersize, progress);
+ if (result == null) {
+ DataChecksum checksum = dfsClientConf.createChecksum(checksumOpt);
+ result = DFSOutputStream.newStreamForCreate(this, src, absPermission,
+ flag, createParent, replication, blockSize, progress, buffersize,
+ checksum, null);
+ }
+ beginFileLease(result.getFileId(), result);
+ return result;
+ }
+
+ /**
+ * Creates a symbolic link.
+ *
+ * @see ClientProtocol#createSymlink(String, String,FsPermission, boolean)
+ */
+ public void createSymlink(String target, String link, boolean createParent)
+ throws IOException {
+ TraceScope scope = getPathTraceScope("createSymlink", target);
+ try {
+ final FsPermission dirPerm = applyUMask(null);
+ namenode.createSymlink(target, link, dirPerm, createParent);
+ } catch (RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ FileAlreadyExistsException.class,
+ FileNotFoundException.class,
+ ParentNotDirectoryException.class,
+ NSQuotaExceededException.class,
+ DSQuotaExceededException.class,
+ QuotaByStorageTypeExceededException.class,
+ UnresolvedPathException.class,
+ SnapshotAccessControlException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Resolve the *first* symlink, if any, in the path.
+ *
+ * @see ClientProtocol#getLinkTarget(String)
+ */
+ public String getLinkTarget(String path) throws IOException {
+ checkOpen();
+ TraceScope scope = getPathTraceScope("getLinkTarget", path);
+ try {
+ return namenode.getLinkTarget(path);
+ } catch (RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ FileNotFoundException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /** Method to get stream returned by append call */
+ private DFSOutputStream callAppend(String src, int buffersize,
+ EnumSet<CreateFlag> flag, Progressable progress, String[] favoredNodes)
+ throws IOException {
+ CreateFlag.validateForAppend(flag);
+ try {
+ LastBlockWithStatus blkWithStatus = namenode.append(src, clientName,
+ new EnumSetWritable<>(flag, CreateFlag.class));
+ return DFSOutputStream.newStreamForAppend(this, src, flag, buffersize,
+ progress, blkWithStatus.getLastBlock(),
+ blkWithStatus.getFileStatus(), dfsClientConf.createChecksum(null),
+ favoredNodes);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ FileNotFoundException.class,
+ SafeModeException.class,
+ DSQuotaExceededException.class,
+ QuotaByStorageTypeExceededException.class,
+ UnsupportedOperationException.class,
+ UnresolvedPathException.class,
+ SnapshotAccessControlException.class);
+ }
+ }
+
+ /**
+ * Append to an existing HDFS file.
+ *
+ * @param src file name
+ * @param buffersize buffer size
+ * @param flag indicates whether to append data to a new block instead of
+ * the last block
+ * @param progress for reporting write-progress; null is acceptable.
+ * @param statistics file system statistics; null is acceptable.
+ * @return an output stream for writing into the file
+ *
+ * @see ClientProtocol#append(String, String, EnumSetWritable)
+ */
+ public HdfsDataOutputStream append(final String src, final int buffersize,
+ EnumSet<CreateFlag> flag, final Progressable progress,
+ final FileSystem.Statistics statistics) throws IOException {
+ final DFSOutputStream out = append(src, buffersize, flag, null, progress);
+ return createWrappedOutputStream(out, statistics, out.getInitialLen());
+ }
+
+ /**
+ * Append to an existing HDFS file.
+ *
+ * @param src file name
+ * @param buffersize buffer size
+ * @param flag indicates whether to append data to a new block instead of the
+ * last block
+ * @param progress for reporting write-progress; null is acceptable.
+ * @param statistics file system statistics; null is acceptable.
+ * @param favoredNodes FavoredNodes for new blocks
+ * @return an output stream for writing into the file
+ * @see ClientProtocol#append(String, String, EnumSetWritable)
+ */
+ public HdfsDataOutputStream append(final String src, final int buffersize,
+ EnumSet<CreateFlag> flag, final Progressable progress,
+ final FileSystem.Statistics statistics,
+ final InetSocketAddress[] favoredNodes) throws IOException {
+ final DFSOutputStream out = append(src, buffersize, flag,
+ getFavoredNodesStr(favoredNodes), progress);
+ return createWrappedOutputStream(out, statistics, out.getInitialLen());
+ }
+
+ private DFSOutputStream append(String src, int buffersize,
+ EnumSet<CreateFlag> flag, String[] favoredNodes, Progressable progress)
+ throws IOException {
+ checkOpen();
+ final DFSOutputStream result = callAppend(src, buffersize, flag, progress,
+ favoredNodes);
+ beginFileLease(result.getFileId(), result);
+ return result;
+ }
+
+ /**
+ * Set replication for an existing file.
+ * @param src file name
+ * @param replication replication to set the file to
+ *
+ * @see ClientProtocol#setReplication(String, short)
+ */
+ public boolean setReplication(String src, short replication)
+ throws IOException {
+ TraceScope scope = getPathTraceScope("setReplication", src);
+ try {
+ return namenode.setReplication(src, replication);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ FileNotFoundException.class,
+ SafeModeException.class,
+ DSQuotaExceededException.class,
+ QuotaByStorageTypeExceededException.class,
+ UnresolvedPathException.class,
+ SnapshotAccessControlException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Set storage policy for an existing file/directory
+ * @param src file/directory name
+ * @param policyName name of the storage policy
+ */
+ public void setStoragePolicy(String src, String policyName)
+ throws IOException {
+ TraceScope scope = getPathTraceScope("setStoragePolicy", src);
+ try {
+ namenode.setStoragePolicy(src, policyName);
+ } catch (RemoteException e) {
+ throw e.unwrapRemoteException(AccessControlException.class,
+ FileNotFoundException.class,
+ SafeModeException.class,
+ NSQuotaExceededException.class,
+ UnresolvedPathException.class,
+ SnapshotAccessControlException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * @param path file/directory name
+ * @return Get the storage policy for specified path
+ */
+ public BlockStoragePolicy getStoragePolicy(String path) throws IOException {
+ checkOpen();
+ TraceScope scope = getPathTraceScope("getStoragePolicy", path);
+ try {
+ return namenode.getStoragePolicy(path);
+ } catch (RemoteException e) {
+ throw e.unwrapRemoteException(AccessControlException.class,
+ FileNotFoundException.class,
+ SafeModeException.class,
+ UnresolvedPathException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * @return All the existing storage policies
+ */
+ public BlockStoragePolicy[] getStoragePolicies() throws IOException {
+ TraceScope scope = Trace.startSpan("getStoragePolicies", traceSampler);
+ try {
+ return namenode.getStoragePolicies();
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Rename file or directory.
+ * @see ClientProtocol#rename(String, String)
+ * @deprecated Use {@link #rename(String, String, Options.Rename...)} instead.
+ */
+ @Deprecated
+ public boolean rename(String src, String dst) throws IOException {
+ checkOpen();
+ TraceScope scope = getSrcDstTraceScope("rename", src, dst);
+ try {
+ return namenode.rename(src, dst);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ NSQuotaExceededException.class,
+ DSQuotaExceededException.class,
+ QuotaByStorageTypeExceededException.class,
+ UnresolvedPathException.class,
+ SnapshotAccessControlException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Move blocks from src to trg and delete src
+ * See {@link ClientProtocol#concat}.
+ */
+ public void concat(String trg, String [] srcs) throws IOException {
+ checkOpen();
+ TraceScope scope = Trace.startSpan("concat", traceSampler);
+ try {
+ namenode.concat(trg, srcs);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ UnresolvedPathException.class,
+ SnapshotAccessControlException.class);
+ } finally {
+ scope.close();
+ }
+ }
+ /**
+ * Rename file or directory.
+ * @see ClientProtocol#rename2(String, String, Options.Rename...)
+ */
+ public void rename(String src, String dst, Options.Rename... options)
+ throws IOException {
+ checkOpen();
+ TraceScope scope = getSrcDstTraceScope("rename2", src, dst);
+ try {
+ namenode.rename2(src, dst, options);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ DSQuotaExceededException.class,
+ QuotaByStorageTypeExceededException.class,
+ FileAlreadyExistsException.class,
+ FileNotFoundException.class,
+ ParentNotDirectoryException.class,
+ SafeModeException.class,
+ NSQuotaExceededException.class,
+ UnresolvedPathException.class,
+ SnapshotAccessControlException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Truncate a file to an indicated size
+ * See {@link ClientProtocol#truncate}.
+ */
+ public boolean truncate(String src, long newLength) throws IOException {
+ checkOpen();
+ if (newLength < 0) {
+ throw new HadoopIllegalArgumentException(
+ "Cannot truncate to a negative file size: " + newLength + ".");
+ }
+ try {
+ return namenode.truncate(src, newLength, clientName);
+ } catch (RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ UnresolvedPathException.class);
+ }
+ }
+
+ /**
+ * Delete file or directory.
+ * See {@link ClientProtocol#delete(String, boolean)}.
+ */
+ @Deprecated
+ public boolean delete(String src) throws IOException {
+ checkOpen();
+ return delete(src, true);
+ }
+
+ /**
+ * delete file or directory.
+ * delete contents of the directory if non empty and recursive
+ * set to true
+ *
+ * @see ClientProtocol#delete(String, boolean)
+ */
+ public boolean delete(String src, boolean recursive) throws IOException {
+ checkOpen();
+ TraceScope scope = getPathTraceScope("delete", src);
+ try {
+ return namenode.delete(src, recursive);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ FileNotFoundException.class,
+ SafeModeException.class,
+ UnresolvedPathException.class,
+ SnapshotAccessControlException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /** Implemented using getFileInfo(src)
+ */
+ public boolean exists(String src) throws IOException {
+ checkOpen();
+ return getFileInfo(src) != null;
+ }
+
+ /**
+ * Get a partial listing of the indicated directory
+ * No block locations need to be fetched
+ */
+ public DirectoryListing listPaths(String src, byte[] startAfter)
+ throws IOException {
+ return listPaths(src, startAfter, false);
+ }
+
+ /**
+ * Get a partial listing of the indicated directory
+ *
+ * Recommend to use HdfsFileStatus.EMPTY_NAME as startAfter
+ * if the application wants to fetch a listing starting from
+ * the first entry in the directory
+ *
+ * @see ClientProtocol#getListing(String, byte[], boolean)
+ */
+ public DirectoryListing listPaths(String src, byte[] startAfter,
+ boolean needLocation) throws IOException {
+ checkOpen();
+ TraceScope scope = getPathTraceScope("listPaths", src);
+ try {
+ return namenode.getListing(src, startAfter, needLocation);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ FileNotFoundException.class,
+ UnresolvedPathException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Get the file info for a specific file or directory.
+ * @param src The string representation of the path to the file
+ * @return object containing information regarding the file
+ * or null if file not found
+ *
+ * @see ClientProtocol#getFileInfo(String) for description of exceptions
+ */
+ public HdfsFileStatus getFileInfo(String src) throws IOException {
+ checkOpen();
+ TraceScope scope = getPathTraceScope("getFileInfo", src);
+ try {
+ return namenode.getFileInfo(src);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ FileNotFoundException.class,
+ UnresolvedPathException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Close status of a file
+ * @return true if file is already closed
+ */
+ public boolean isFileClosed(String src) throws IOException{
+ checkOpen();
+ TraceScope scope = getPathTraceScope("isFileClosed", src);
+ try {
+ return namenode.isFileClosed(src);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ FileNotFoundException.class,
+ UnresolvedPathException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Get the file info for a specific file or directory. If src
+ * refers to a symlink then the FileStatus of the link is returned.
+ * @param src path to a file or directory.
+ *
+ * For description of exceptions thrown
+ * @see ClientProtocol#getFileLinkInfo(String)
+ */
+ public HdfsFileStatus getFileLinkInfo(String src) throws IOException {
+ checkOpen();
+ TraceScope scope = getPathTraceScope("getFileLinkInfo", src);
+ try {
+ return namenode.getFileLinkInfo(src);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ UnresolvedPathException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ @InterfaceAudience.Private
+ public void clearDataEncryptionKey() {
+ LOG.debug("Clearing encryption key");
+ synchronized (this) {
+ encryptionKey = null;
+ }
+ }
+
+ /**
+ * @return true if data sent between this client and DNs should be encrypted,
+ * false otherwise.
+ * @throws IOException in the event of error communicating with the NN
+ */
+ boolean shouldEncryptData() throws IOException {
+ FsServerDefaults d = getServerDefaults();
+ return d == null ? false : d.getEncryptDataTransfer();
+ }
+
+ @Override
+ public DataEncryptionKey newDataEncryptionKey() throws IOException {
+ if (shouldEncryptData()) {
+ synchronized (this) {
+ if (encryptionKey == null ||
+ encryptionKey.expiryDate < Time.now()) {
+ LOG.debug("Getting new encryption token from NN");
+ encryptionKey = namenode.getDataEncryptionKey();
+ }
+ return encryptionKey;
+ }
+ } else {
+ return null;
+ }
+ }
+
+ /**
+ * Get the checksum of the whole file of a range of the file. Note that the
+ * range always starts from the beginning of the file.
+ * @param src The file path
+ * @param length the length of the range, i.e., the range is [0, length]
+ * @return The checksum
+ * @see DistributedFileSystem#getFileChecksum(Path)
+ */
+ public MD5MD5CRC32FileChecksum getFileChecksum(String src, long length)
+ throws IOException {
+ checkOpen();
+ Preconditions.checkArgument(length >= 0);
+ //get block locations for the file range
+ LocatedBlocks blockLocations = callGetBlockLocations(namenode, src, 0,
+ length);
+ if (null == blockLocations) {
+ throw new FileNotFoundException("File does not exist: " + src);
+ }
+ if (blockLocations.isUnderConstruction()) {
+ throw new IOException("Fail to get checksum, since file " + src
+ + " is under construction.");
+ }
+ List<LocatedBlock> locatedblocks = blockLocations.getLocatedBlocks();
+ final DataOutputBuffer md5out = new DataOutputBuffer();
+ int bytesPerCRC = -1;
+ DataChecksum.Type crcType = DataChecksum.Type.DEFAULT;
+ long crcPerBlock = 0;
+ boolean refetchBlocks = false;
+ int lastRetriedIndex = -1;
+
+ // get block checksum for each block
+ long remaining = length;
+ if (src.contains(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR_SEPARATOR)) {
+ remaining = Math.min(length, blockLocations.getFileLength());
+ }
+ for(int i = 0; i < locatedblocks.size() && remaining > 0; i++) {
+ if (refetchBlocks) { // refetch to get fresh tokens
+ blockLocations = callGetBlockLocations(namenode, src, 0, length);
+ if (null == blockLocations) {
+ throw new FileNotFoundException("File does not exist: " + src);
+ }
+ if (blockLocations.isUnderConstruction()) {
+ throw new IOException("Fail to get checksum, since file " + src
+ + " is under construction.");
+ }
+ locatedblocks = blockLocations.getLocatedBlocks();
+ refetchBlocks = false;
+ }
+ LocatedBlock lb = locatedblocks.get(i);
+ final ExtendedBlock block = lb.getBlock();
+ if (remaining < block.getNumBytes()) {
+ block.setNumBytes(remaining);
+ }
+ remaining -= block.getNumBytes();
+ final DatanodeInfo[] datanodes = lb.getLocations();
+
+ //try each datanode location of the block
+ final int timeout = 3000*datanodes.length + dfsClientConf.getSocketTimeout();
+ boolean done = false;
+ for(int j = 0; !done && j < datanodes.length; j++) {
+ DataOutputStream out = null;
+ DataInputStream in = null;
+
+ try {
+ //connect to a datanode
+ IOStreamPair pair = connectToDN(datanodes[j], timeout, lb);
+ out = new DataOutputStream(new BufferedOutputStream(pair.out,
+ smallBufferSize));
+ in = new DataInputStream(pair.in);
+
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("write to " + datanodes[j] + ": "
+ + Op.BLOCK_CHECKSUM + ", block=" + block);
+ }
+ // get block MD5
+ new Sender(out).blockChecksum(block, lb.getBlockToken());
+
+ final BlockOpResponseProto reply =
+ BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
+
+ String logInfo = "for block " + block + " from datanode " + datanodes[j];
+ DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
+
+ OpBlockChecksumResponseProto checksumData =
+ reply.getChecksumResponse();
+
+ //read byte-per-checksum
+ final int bpc = checksumData.getBytesPerCrc();
+ if (i == 0) { //first block
+ bytesPerCRC = bpc;
+ }
+ else if (bpc != bytesPerCRC) {
+ throw new IOException("Byte-per-checksum not matched: bpc=" + bpc
+ + " but bytesPerCRC=" + bytesPerCRC);
+ }
+
+ //read crc-per-block
+ final long cpb = checksumData.getCrcPerBlock();
+ if (locatedblocks.size() > 1 && i == 0) {
+ crcPerBlock = cpb;
+ }
+
+ //read md5
+ final MD5Hash md5 = new MD5Hash(
+ checksumData.getMd5().toByteArray());
+ md5.write(md5out);
+
+ // read crc-type
+ final DataChecksum.Type ct;
+ if (checksumData.hasCrcType()) {
+ ct = PBHelperClient.convert(checksumData
+ .getCrcType());
+ } else {
+ LOG.debug("Retrieving checksum from an earlier-version DataNode: " +
+ "inferring checksum by reading first byte");
+ ct = inferChecksumTypeByReading(lb, datanodes[j]);
+ }
+
+ if (i == 0) { // first block
+ crcType = ct;
+ } else if (crcType != DataChecksum.Type.MIXED
+ && crcType != ct) {
+ // if crc types are mixed in a file
+ crcType = DataChecksum.Type.MIXED;
+ }
+
+ done = true;
+
+ if (LOG.isDebugEnabled()) {
+ if (i == 0) {
+ LOG.debug("set bytesPerCRC=" + bytesPerCRC
+ + ", crcPerBlock=" + crcPerBlock);
+ }
+ LOG.debug("got reply from " + datanodes[j] + ": md5=" + md5);
+ }
+ } catch (InvalidBlockTokenException ibte) {
+ if (i > lastRetriedIndex) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Got access token error in response to OP_BLOCK_CHECKSUM "
+ + "for file " + src + " for block " + block
+ + " from datanode " + datanodes[j]
+ + ". Will retry the block once.");
+ }
+ lastRetriedIndex = i;
+ done = true; // actually it's not done; but we'll retry
+ i--; // repeat at i-th block
+ refetchBlocks = true;
+ break;
+ }
+ } catch (IOException ie) {
+ LOG.warn("src=" + src + ", datanodes["+j+"]=" + datanodes[j], ie);
+ } finally {
+ IOUtils.closeStream(in);
+ IOUtils.closeStream(out);
+ }
+ }
+
+ if (!done) {
+ throw new IOException("Fail to get block MD5 for " + block);
+ }
+ }
+
+ //compute file MD5
+ final MD5Hash fileMD5 = MD5Hash.digest(md5out.getData());
+ switch (crcType) {
+ case CRC32:
+ return new MD5MD5CRC32GzipFileChecksum(bytesPerCRC,
+ crcPerBlock, fileMD5);
+ case CRC32C:
+ return new MD5MD5CRC32CastagnoliFileChecksum(bytesPerCRC,
+ crcPerBlock, fileMD5);
+ default:
+ // If there is no block allocated for the file,
+ // return one with the magic entry that matches what previous
+ // hdfs versions return.
+ if (locatedblocks.size() == 0) {
+ return new MD5MD5CRC32GzipFileChecksum(0, 0, fileMD5);
+ }
+
+ // we should never get here since the validity was checked
+ // when getCrcType() was called above.
+ return null;
+ }
+ }
+
+ /**
+ * Connect to the given datanode's datantrasfer port, and return
+ * the resulting IOStreamPair. This includes encryption wrapping, etc.
+ */
+ private IOStreamPair connectToDN(DatanodeInfo dn, int timeout,
+ LocatedBlock lb) throws IOException {
+ boolean success = false;
+ Socket sock = null;
+ try {
+ sock = socketFactory.createSocket();
+ String dnAddr = dn.getXferAddr(getConf().isConnectToDnViaHostname());
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Connecting to datanode " + dnAddr);
+ }
+ NetUtils.connect(sock, NetUtils.createSocketAddr(dnAddr), timeout);
+ sock.setSoTimeout(timeout);
+
+ OutputStream unbufOut = NetUtils.getOutputStream(sock);
+ InputStream unbufIn = NetUtils.getInputStream(sock);
+ IOStreamPair ret = saslClient.newSocketSend(sock, unbufOut, unbufIn, this,
+ lb.getBlockToken(), dn);
+ success = true;
+ return ret;
+ } finally {
+ if (!success) {
+ IOUtils.closeSocket(sock);
+ }
+ }
+ }
+
+ /**
+ * Infer the checksum type for a replica by sending an OP_READ_BLOCK
+ * for the first byte of that replica. This is used for compatibility
+ * with older HDFS versions which did not include the checksum type in
+ * OpBlockChecksumResponseProto.
+ *
+ * @param lb the located block
+ * @param dn the connected datanode
+ * @return the inferred checksum type
+ * @throws IOException if an error occurs
+ */
+ private Type inferChecksumTypeByReading(LocatedBlock lb, DatanodeInfo dn)
+ throws IOException {
+ IOStreamPair pair = connectToDN(dn, dfsClientConf.getSocketTimeout(), lb);
+
+ try {
+ DataOutputStream out = new DataOutputStream(new BufferedOutputStream(pair.out,
+ smallBufferSize));
+ DataInputStream in = new DataInputStream(pair.in);
+
+ new Sender(out).readBlock(lb.getBlock(), lb.getBlockToken(), clientName,
+ 0, 1, true, CachingStrategy.newDefaultStrategy());
+ final BlockOpResponseProto reply =
+ BlockOpResponseProto.parseFrom(PBHelperClient.vintPrefixed(in));
+ String logInfo = "trying to read " + lb.getBlock() + " from datanode " + dn;
+ DataTransferProtoUtil.checkBlockOpStatus(reply, logInfo);
+
+ return PBHelperClient.convert(reply.getReadOpChecksumInfo().getChecksum().getType());
+ } finally {
+ IOUtilsClient.cleanup(null, pair.in, pair.out);
+ }
+ }
+
+ /**
+ * Set permissions to a file or directory.
+ * @param src path name.
+ * @param permission permission to set to
+ *
+ * @see ClientProtocol#setPermission(String, FsPermission)
+ */
+ public void setPermission(String src, FsPermission permission)
+ throws IOException {
+ checkOpen();
+ TraceScope scope = getPathTraceScope("setPermission", src);
+ try {
+ namenode.setPermission(src, permission);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ FileNotFoundException.class,
+ SafeModeException.class,
+ UnresolvedPathException.class,
+ SnapshotAccessControlException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Set file or directory owner.
+ * @param src path name.
+ * @param username user id.
+ * @param groupname user group.
+ *
+ * @see ClientProtocol#setOwner(String, String, String)
+ */
+ public void setOwner(String src, String username, String groupname)
+ throws IOException {
+ checkOpen();
+ TraceScope scope = getPathTraceScope("setOwner", src);
+ try {
+ namenode.setOwner(src, username, groupname);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException(AccessControlException.class,
+ FileNotFoundException.class,
+ SafeModeException.class,
+ UnresolvedPathException.class,
+ SnapshotAccessControlException.class);
+ } finally {
+ scope.close();
+ }
+ }
+
+ private long[] callGetStats() throws IOException {
+ checkOpen();
+ TraceScope scope = Trace.startSpan("getStats", traceSampler);
+ try {
+ return namenode.getStats();
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * @see ClientProtocol#getStats()
+ */
+ public FsStatus getDiskStatus() throws IOException {
+ long rawNums[] = callGetStats();
+ return new FsStatus(rawNums[0], rawNums[1], rawNums[2]);
+ }
+
+ /**
+ * Returns count of blocks with no good replicas left. Normally should be
+ * zero.
+ * @throws IOException
+ */
+ public long getMissingBlocksCount() throws IOException {
+ return callGetStats()[ClientProtocol.GET_STATS_MISSING_BLOCKS_IDX];
+ }
+
+ /**
+ * Returns count of blocks with replication factor 1 and have
+ * lost the only replica.
+ * @throws IOException
+ */
+ public long getMissingReplOneBlocksCount() throws IOException {
+ return callGetStats()[ClientProtocol.
+ GET_STATS_MISSING_REPL_ONE_BLOCKS_IDX];
+ }
+
+ /**
+ * Returns count of blocks with one of more replica missing.
+ * @throws IOException
+ */
+ public long getUnderReplicatedBlocksCount() throws IOException {
+ return callGetStats()[ClientProtocol.GET_STATS_UNDER_REPLICATED_IDX];
+ }
+
+ /**
+ * Returns count of blocks with at least one replica marked corrupt.
+ * @throws IOException
+ */
+ public long getCorruptBlocksCount() throws IOException {
+ return callGetStats()[ClientProtocol.GET_STATS_CORRUPT_BLOCKS_IDX];
+ }
+
+ /**
+ * @return a list in which each entry describes a corrupt file/block
+ * @throws IOException
+ */
+ public CorruptFileBlocks listCorruptFileBlocks(String path,
+ String cookie)
+ throws IOException {
+ checkOpen();
+ TraceScope scope = getPathTraceScope("listCorruptFileBlocks", path);
+ try {
+ return namenode.listCorruptFileBlocks(path, cookie);
+ } finally {
+ scope.close();
+ }
+ }
+
+ public DatanodeInfo[] datanodeReport(DatanodeReportType type)
+ throws IOException {
+ checkOpen();
+ TraceScope scope = Trace.startSpan("datanodeReport", traceSampler);
+ try {
+ return namenode.getDatanodeReport(type);
+ } finally {
+ scope.close();
+ }
+ }
+
+ public DatanodeStorageReport[] getDatanodeStorageReport(
+ DatanodeReportType type) throws IOException {
+ checkOpen();
+ TraceScope scope =
+ Trace.startSpan("datanodeStorageReport", traceSampler);
+ try {
+ return namenode.getDatanodeStorageReport(type);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Enter, leave or get safe mode.
+ *
+ * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction,boolean)
+ */
+ public boolean setSafeMode(SafeModeAction action) throws IOException {
+ return setSafeMode(action, false);
+ }
+
+ /**
+ * Enter, leave or get safe mode.
+ *
+ * @param action
+ * One of SafeModeAction.GET, SafeModeAction.ENTER and
+ * SafeModeActiob.LEAVE
+ * @param isChecked
+ * If true, then check only active namenode's safemode status, else
+ * check first namenode's status.
+ * @see ClientProtocol#setSafeMode(HdfsConstants.SafeModeAction, boolean)
+ */
+ public boolean setSafeMode(SafeModeAction action, boolean isChecked) throws IOException{
+ TraceScope scope =
+ Trace.startSpan("setSafeMode", traceSampler);
+ try {
+ return namenode.setSafeMode(action, isChecked);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Create one snapshot.
+ *
+ * @param snapshotRoot The directory where the snapshot is to be taken
+ * @param snapshotName Name of the snapshot
+ * @return the snapshot path.
+ * @see ClientProtocol#createSnapshot(String, String)
+ */
+ public String createSnapshot(String snapshotRoot, String snapshotName)
+ throws IOException {
+ checkOpen();
+ TraceScope scope = Trace.startSpan("createSnapshot", traceSampler);
+ try {
+ return namenode.createSnapshot(snapshotRoot, snapshotName);
+ } catch(RemoteException re) {
+ throw re.unwrapRemoteException();
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Delete a snapshot of a snapshottable directory.
+ *
+ * @param snapshotRoot The snapshottable directory that the
+ * to-be-deleted snapshot belongs to
+ * @param snapshotName The name of the to-be-deleted snapshot
+ * @throws IOException
+ * @see ClientProtocol#deleteSnapshot(String, String)
+ */
+ public void deleteSnapshot(String snapshotRoot, String snapshotName)
+ throws IOException {
+ checkOpen();
+ TraceScope scope = Trace.startSpan("deleteSnapshot", traceSampler);
+ try {
+ namenode.deleteSnapshot(snapshotRoot, snapshotName);
+ } catch(R
<TRUNCATED>
[09/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and
related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
new file mode 100755
index 0000000..5466936
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -0,0 +1,923 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.channels.ClosedChannelException;
+import java.util.EnumSet;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.crypto.CryptoProtocolVersion;
+import org.apache.hadoop.fs.CanSetDropBehind;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.fs.FSOutputSummer;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Syncable;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.NSQuotaExceededException;
+import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException;
+import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.protocol.UnresolvedPathException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.server.namenode.RetryStartFileException;
+import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
+import org.apache.hadoop.hdfs.util.ByteArrayManager;
+import org.apache.hadoop.io.EnumSetWritable;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.DataChecksum.Type;
+import org.apache.hadoop.util.Progressable;
+import org.apache.hadoop.util.Time;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+
+/****************************************************************
+ * DFSOutputStream creates files from a stream of bytes.
+ *
+ * The client application writes data that is cached internally by
+ * this stream. Data is broken up into packets, each packet is
+ * typically 64K in size. A packet comprises of chunks. Each chunk
+ * is typically 512 bytes and has an associated checksum with it.
+ *
+ * When a client application fills up the currentPacket, it is
+ * enqueued into the dataQueue of DataStreamer. DataStreamer is a
+ * thread that picks up packets from the dataQueue and sends it to
+ * the first datanode in the pipeline.
+ *
+ ****************************************************************/
+@InterfaceAudience.Private
+public class DFSOutputStream extends FSOutputSummer
+ implements Syncable, CanSetDropBehind {
+ static final Logger LOG = LoggerFactory.getLogger(DFSOutputStream.class);
+ /**
+ * Number of times to retry creating a file when there are transient
+ * errors (typically related to encryption zones and KeyProvider operations).
+ */
+ @VisibleForTesting
+ static final int CREATE_RETRY_COUNT = 10;
+ @VisibleForTesting
+ static CryptoProtocolVersion[] SUPPORTED_CRYPTO_VERSIONS =
+ CryptoProtocolVersion.supported();
+
+ protected final DFSClient dfsClient;
+ protected final ByteArrayManager byteArrayManager;
+ // closed is accessed by different threads under different locks.
+ protected volatile boolean closed = false;
+
+ protected final String src;
+ protected final long fileId;
+ protected final long blockSize;
+ protected final int bytesPerChecksum;
+
+ protected DFSPacket currentPacket = null;
+ private DataStreamer streamer;
+ protected int packetSize = 0; // write packet size, not including the header.
+ protected int chunksPerPacket = 0;
+ protected long lastFlushOffset = 0; // offset when flush was invoked
+ private long initialFileSize = 0; // at time of file open
+ private final short blockReplication; // replication factor of file
+ protected boolean shouldSyncBlock = false; // force blocks to disk upon close
+ protected final AtomicReference<CachingStrategy> cachingStrategy;
+ private FileEncryptionInfo fileEncryptionInfo;
+
+ /** Use {@link ByteArrayManager} to create buffer for non-heartbeat packets.*/
+ protected DFSPacket createPacket(int packetSize, int chunksPerPkt, long offsetInBlock,
+ long seqno, boolean lastPacketInBlock) throws InterruptedIOException {
+ final byte[] buf;
+ final int bufferSize = PacketHeader.PKT_MAX_HEADER_LEN + packetSize;
+
+ try {
+ buf = byteArrayManager.newByteArray(bufferSize);
+ } catch (InterruptedException ie) {
+ final InterruptedIOException iioe = new InterruptedIOException(
+ "seqno=" + seqno);
+ iioe.initCause(ie);
+ throw iioe;
+ }
+
+ return new DFSPacket(buf, chunksPerPkt, offsetInBlock, seqno,
+ getChecksumSize(), lastPacketInBlock);
+ }
+
+ @Override
+ protected void checkClosed() throws IOException {
+ if (isClosed()) {
+ getStreamer().getLastException().throwException4Close();
+ }
+ }
+
+ //
+ // returns the list of targets, if any, that is being currently used.
+ //
+ @VisibleForTesting
+ public synchronized DatanodeInfo[] getPipeline() {
+ if (getStreamer().streamerClosed()) {
+ return null;
+ }
+ DatanodeInfo[] currentNodes = getStreamer().getNodes();
+ if (currentNodes == null) {
+ return null;
+ }
+ DatanodeInfo[] value = new DatanodeInfo[currentNodes.length];
+ for (int i = 0; i < currentNodes.length; i++) {
+ value[i] = currentNodes[i];
+ }
+ return value;
+ }
+
+ /**
+ * @return the object for computing checksum.
+ * The type is NULL if checksum is not computed.
+ */
+ private static DataChecksum getChecksum4Compute(DataChecksum checksum,
+ HdfsFileStatus stat) {
+ if (DataStreamer.isLazyPersist(stat) && stat.getReplication() == 1) {
+ // do not compute checksum for writing to single replica to memory
+ return DataChecksum.newDataChecksum(Type.NULL,
+ checksum.getBytesPerChecksum());
+ }
+ return checksum;
+ }
+
+ private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress,
+ HdfsFileStatus stat, DataChecksum checksum) throws IOException {
+ super(getChecksum4Compute(checksum, stat));
+ this.dfsClient = dfsClient;
+ this.src = src;
+ this.fileId = stat.getFileId();
+ this.blockSize = stat.getBlockSize();
+ this.blockReplication = stat.getReplication();
+ this.fileEncryptionInfo = stat.getFileEncryptionInfo();
+ this.cachingStrategy = new AtomicReference<CachingStrategy>(
+ dfsClient.getDefaultWriteCachingStrategy());
+ if ((progress != null) && DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug(
+ "Set non-null progress callback on DFSOutputStream " + src);
+ }
+
+ this.bytesPerChecksum = checksum.getBytesPerChecksum();
+ if (bytesPerChecksum <= 0) {
+ throw new HadoopIllegalArgumentException(
+ "Invalid value: bytesPerChecksum = " + bytesPerChecksum + " <= 0");
+ }
+ if (blockSize % bytesPerChecksum != 0) {
+ throw new HadoopIllegalArgumentException("Invalid values: "
+ + HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + bytesPerChecksum
+ + ") must divide block size (=" + blockSize + ").");
+ }
+ this.byteArrayManager = dfsClient.getClientContext().getByteArrayManager();
+ }
+
+ /** Construct a new output stream for creating a file. */
+ protected DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
+ EnumSet<CreateFlag> flag, Progressable progress,
+ DataChecksum checksum, String[] favoredNodes) throws IOException {
+ this(dfsClient, src, progress, stat, checksum);
+ this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK);
+
+ computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum);
+
+ streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum,
+ cachingStrategy, byteArrayManager, favoredNodes);
+ }
+
+ static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src,
+ FsPermission masked, EnumSet<CreateFlag> flag, boolean createParent,
+ short replication, long blockSize, Progressable progress, int buffersize,
+ DataChecksum checksum, String[] favoredNodes) throws IOException {
+ TraceScope scope =
+ dfsClient.getPathTraceScope("newStreamForCreate", src);
+ try {
+ HdfsFileStatus stat = null;
+
+ // Retry the create if we get a RetryStartFileException up to a maximum
+ // number of times
+ boolean shouldRetry = true;
+ int retryCount = CREATE_RETRY_COUNT;
+ while (shouldRetry) {
+ shouldRetry = false;
+ try {
+ stat = dfsClient.namenode.create(src, masked, dfsClient.clientName,
+ new EnumSetWritable<CreateFlag>(flag), createParent, replication,
+ blockSize, SUPPORTED_CRYPTO_VERSIONS);
+ break;
+ } catch (RemoteException re) {
+ IOException e = re.unwrapRemoteException(
+ AccessControlException.class,
+ DSQuotaExceededException.class,
+ QuotaByStorageTypeExceededException.class,
+ FileAlreadyExistsException.class,
+ FileNotFoundException.class,
+ ParentNotDirectoryException.class,
+ NSQuotaExceededException.class,
+ RetryStartFileException.class,
+ SafeModeException.class,
+ UnresolvedPathException.class,
+ SnapshotAccessControlException.class,
+ UnknownCryptoProtocolVersionException.class);
+ if (e instanceof RetryStartFileException) {
+ if (retryCount > 0) {
+ shouldRetry = true;
+ retryCount--;
+ } else {
+ throw new IOException("Too many retries because of encryption" +
+ " zone operations", e);
+ }
+ } else {
+ throw e;
+ }
+ }
+ }
+ Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!");
+ final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
+ flag, progress, checksum, favoredNodes);
+ out.start();
+ return out;
+ } finally {
+ scope.close();
+ }
+ }
+
+ /** Construct a new output stream for append. */
+ private DFSOutputStream(DFSClient dfsClient, String src,
+ EnumSet<CreateFlag> flags, Progressable progress, LocatedBlock lastBlock,
+ HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes)
+ throws IOException {
+ this(dfsClient, src, progress, stat, checksum);
+ initialFileSize = stat.getLen(); // length of file when opened
+ this.shouldSyncBlock = flags.contains(CreateFlag.SYNC_BLOCK);
+
+ boolean toNewBlock = flags.contains(CreateFlag.NEW_BLOCK);
+
+ this.fileEncryptionInfo = stat.getFileEncryptionInfo();
+
+ // The last partial block of the file has to be filled.
+ if (!toNewBlock && lastBlock != null) {
+ // indicate that we are appending to an existing block
+ streamer = new DataStreamer(lastBlock, stat, dfsClient, src, progress, checksum,
+ cachingStrategy, byteArrayManager);
+ getStreamer().setBytesCurBlock(lastBlock.getBlockSize());
+ adjustPacketChunkSize(stat);
+ getStreamer().setPipelineInConstruction(lastBlock);
+ } else {
+ computePacketChunkSize(dfsClient.getConf().getWritePacketSize(),
+ bytesPerChecksum);
+ streamer = new DataStreamer(stat, lastBlock != null ? lastBlock.getBlock() : null,
+ dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
+ favoredNodes);
+ }
+ }
+
+ private void adjustPacketChunkSize(HdfsFileStatus stat) throws IOException{
+
+ long usedInLastBlock = stat.getLen() % blockSize;
+ int freeInLastBlock = (int)(blockSize - usedInLastBlock);
+
+ // calculate the amount of free space in the pre-existing
+ // last crc chunk
+ int usedInCksum = (int)(stat.getLen() % bytesPerChecksum);
+ int freeInCksum = bytesPerChecksum - usedInCksum;
+
+ // if there is space in the last block, then we have to
+ // append to that block
+ if (freeInLastBlock == blockSize) {
+ throw new IOException("The last block for file " +
+ src + " is full.");
+ }
+
+ if (usedInCksum > 0 && freeInCksum > 0) {
+ // if there is space in the last partial chunk, then
+ // setup in such a way that the next packet will have only
+ // one chunk that fills up the partial chunk.
+ //
+ computePacketChunkSize(0, freeInCksum);
+ setChecksumBufSize(freeInCksum);
+ getStreamer().setAppendChunk(true);
+ } else {
+ // if the remaining space in the block is smaller than
+ // that expected size of of a packet, then create
+ // smaller size packet.
+ //
+ computePacketChunkSize(
+ Math.min(dfsClient.getConf().getWritePacketSize(), freeInLastBlock),
+ bytesPerChecksum);
+ }
+ }
+
+ static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src,
+ EnumSet<CreateFlag> flags, int bufferSize, Progressable progress,
+ LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum,
+ String[] favoredNodes) throws IOException {
+ TraceScope scope =
+ dfsClient.getPathTraceScope("newStreamForAppend", src);
+ try {
+ final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
+ progress, lastBlock, stat, checksum, favoredNodes);
+ out.start();
+ return out;
+ } finally {
+ scope.close();
+ }
+ }
+
+ protected void computePacketChunkSize(int psize, int csize) {
+ final int bodySize = psize - PacketHeader.PKT_MAX_HEADER_LEN;
+ final int chunkSize = csize + getChecksumSize();
+ chunksPerPacket = Math.max(bodySize/chunkSize, 1);
+ packetSize = chunkSize*chunksPerPacket;
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("computePacketChunkSize: src=" + src +
+ ", chunkSize=" + chunkSize +
+ ", chunksPerPacket=" + chunksPerPacket +
+ ", packetSize=" + packetSize);
+ }
+ }
+
+ protected TraceScope createWriteTraceScope() {
+ return dfsClient.getPathTraceScope("DFSOutputStream#write", src);
+ }
+
+ // @see FSOutputSummer#writeChunk()
+ @Override
+ protected synchronized void writeChunk(byte[] b, int offset, int len,
+ byte[] checksum, int ckoff, int cklen) throws IOException {
+ dfsClient.checkOpen();
+ checkClosed();
+
+ if (len > bytesPerChecksum) {
+ throw new IOException("writeChunk() buffer size is " + len +
+ " is larger than supported bytesPerChecksum " +
+ bytesPerChecksum);
+ }
+ if (cklen != 0 && cklen != getChecksumSize()) {
+ throw new IOException("writeChunk() checksum size is supposed to be " +
+ getChecksumSize() + " but found to be " + cklen);
+ }
+
+ if (currentPacket == null) {
+ currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer()
+ .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false);
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno=" +
+ currentPacket.getSeqno() +
+ ", src=" + src +
+ ", packetSize=" + packetSize +
+ ", chunksPerPacket=" + chunksPerPacket +
+ ", bytesCurBlock=" + getStreamer().getBytesCurBlock());
+ }
+ }
+
+ currentPacket.writeChecksum(checksum, ckoff, cklen);
+ currentPacket.writeData(b, offset, len);
+ currentPacket.incNumChunks();
+ getStreamer().incBytesCurBlock(len);
+
+ // If packet is full, enqueue it for transmission
+ //
+ if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() ||
+ getStreamer().getBytesCurBlock() == blockSize) {
+ enqueueCurrentPacketFull();
+ }
+ }
+
+ void enqueueCurrentPacket() throws IOException {
+ getStreamer().waitAndQueuePacket(currentPacket);
+ currentPacket = null;
+ }
+
+ void enqueueCurrentPacketFull() throws IOException {
+ LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={},"
+ + " appendChunk={}, {}", currentPacket, src, getStreamer()
+ .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(),
+ getStreamer());
+ enqueueCurrentPacket();
+ adjustChunkBoundary();
+ endBlock();
+ }
+
+ /** create an empty packet to mark the end of the block. */
+ void setCurrentPacketToEmpty() throws InterruptedIOException {
+ currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
+ getStreamer().getAndIncCurrentSeqno(), true);
+ currentPacket.setSyncBlock(shouldSyncBlock);
+ }
+
+ /**
+ * If the reopened file did not end at chunk boundary and the above
+ * write filled up its partial chunk. Tell the summer to generate full
+ * crc chunks from now on.
+ */
+ protected void adjustChunkBoundary() {
+ if (getStreamer().getAppendChunk() &&
+ getStreamer().getBytesCurBlock() % bytesPerChecksum == 0) {
+ getStreamer().setAppendChunk(false);
+ resetChecksumBufSize();
+ }
+
+ if (!getStreamer().getAppendChunk()) {
+ int psize = Math.min((int)(blockSize- getStreamer().getBytesCurBlock()),
+ dfsClient.getConf().getWritePacketSize());
+ computePacketChunkSize(psize, bytesPerChecksum);
+ }
+ }
+
+ /**
+ * if encountering a block boundary, send an empty packet to
+ * indicate the end of block and reset bytesCurBlock.
+ *
+ * @throws IOException
+ */
+ protected void endBlock() throws IOException {
+ if (getStreamer().getBytesCurBlock() == blockSize) {
+ setCurrentPacketToEmpty();
+ enqueueCurrentPacket();
+ getStreamer().setBytesCurBlock(0);
+ lastFlushOffset = 0;
+ }
+ }
+
+ @Deprecated
+ public void sync() throws IOException {
+ hflush();
+ }
+
+ /**
+ * Flushes out to all replicas of the block. The data is in the buffers
+ * of the DNs but not necessarily in the DN's OS buffers.
+ *
+ * It is a synchronous operation. When it returns,
+ * it guarantees that flushed data become visible to new readers.
+ * It is not guaranteed that data has been flushed to
+ * persistent store on the datanode.
+ * Block allocations are persisted on namenode.
+ */
+ @Override
+ public void hflush() throws IOException {
+ TraceScope scope =
+ dfsClient.getPathTraceScope("hflush", src);
+ try {
+ flushOrSync(false, EnumSet.noneOf(SyncFlag.class));
+ } finally {
+ scope.close();
+ }
+ }
+
+ @Override
+ public void hsync() throws IOException {
+ TraceScope scope =
+ dfsClient.getPathTraceScope("hsync", src);
+ try {
+ flushOrSync(true, EnumSet.noneOf(SyncFlag.class));
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * The expected semantics is all data have flushed out to all replicas
+ * and all replicas have done posix fsync equivalent - ie the OS has
+ * flushed it to the disk device (but the disk may have it in its cache).
+ *
+ * Note that only the current block is flushed to the disk device.
+ * To guarantee durable sync across block boundaries the stream should
+ * be created with {@link CreateFlag#SYNC_BLOCK}.
+ *
+ * @param syncFlags
+ * Indicate the semantic of the sync. Currently used to specify
+ * whether or not to update the block length in NameNode.
+ */
+ public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
+ TraceScope scope =
+ dfsClient.getPathTraceScope("hsync", src);
+ try {
+ flushOrSync(true, syncFlags);
+ } finally {
+ scope.close();
+ }
+ }
+
+ /**
+ * Flush/Sync buffered data to DataNodes.
+ *
+ * @param isSync
+ * Whether or not to require all replicas to flush data to the disk
+ * device
+ * @param syncFlags
+ * Indicate extra detailed semantic of the flush/sync. Currently
+ * mainly used to specify whether or not to update the file length in
+ * the NameNode
+ * @throws IOException
+ */
+ private void flushOrSync(boolean isSync, EnumSet<SyncFlag> syncFlags)
+ throws IOException {
+ dfsClient.checkOpen();
+ checkClosed();
+ try {
+ long toWaitFor;
+ long lastBlockLength = -1L;
+ boolean updateLength = syncFlags.contains(SyncFlag.UPDATE_LENGTH);
+ boolean endBlock = syncFlags.contains(SyncFlag.END_BLOCK);
+ synchronized (this) {
+ // flush checksum buffer, but keep checksum buffer intact if we do not
+ // need to end the current block
+ int numKept = flushBuffer(!endBlock, true);
+ // bytesCurBlock potentially incremented if there was buffered data
+
+ if (DFSClient.LOG.isDebugEnabled()) {
+ DFSClient.LOG.debug("DFSClient flush(): "
+ + " bytesCurBlock=" + getStreamer().getBytesCurBlock()
+ + " lastFlushOffset=" + lastFlushOffset
+ + " createNewBlock=" + endBlock);
+ }
+ // Flush only if we haven't already flushed till this offset.
+ if (lastFlushOffset != getStreamer().getBytesCurBlock()) {
+ assert getStreamer().getBytesCurBlock() > lastFlushOffset;
+ // record the valid offset of this flush
+ lastFlushOffset = getStreamer().getBytesCurBlock();
+ if (isSync && currentPacket == null && !endBlock) {
+ // Nothing to send right now,
+ // but sync was requested.
+ // Send an empty packet if we do not end the block right now
+ currentPacket = createPacket(packetSize, chunksPerPacket,
+ getStreamer().getBytesCurBlock(), getStreamer()
+ .getAndIncCurrentSeqno(), false);
+ }
+ } else {
+ if (isSync && getStreamer().getBytesCurBlock() > 0 && !endBlock) {
+ // Nothing to send right now,
+ // and the block was partially written,
+ // and sync was requested.
+ // So send an empty sync packet if we do not end the block right
+ // now
+ currentPacket = createPacket(packetSize, chunksPerPacket,
+ getStreamer().getBytesCurBlock(), getStreamer()
+ .getAndIncCurrentSeqno(), false);
+ } else if (currentPacket != null) {
+ // just discard the current packet since it is already been sent.
+ currentPacket.releaseBuffer(byteArrayManager);
+ currentPacket = null;
+ }
+ }
+ if (currentPacket != null) {
+ currentPacket.setSyncBlock(isSync);
+ enqueueCurrentPacket();
+ }
+ if (endBlock && getStreamer().getBytesCurBlock() > 0) {
+ // Need to end the current block, thus send an empty packet to
+ // indicate this is the end of the block and reset bytesCurBlock
+ currentPacket = createPacket(0, 0, getStreamer().getBytesCurBlock(),
+ getStreamer().getAndIncCurrentSeqno(), true);
+ currentPacket.setSyncBlock(shouldSyncBlock || isSync);
+ enqueueCurrentPacket();
+ getStreamer().setBytesCurBlock(0);
+ lastFlushOffset = 0;
+ } else {
+ // Restore state of stream. Record the last flush offset
+ // of the last full chunk that was flushed.
+ getStreamer().setBytesCurBlock(
+ getStreamer().getBytesCurBlock() - numKept);
+ }
+
+ toWaitFor = getStreamer().getLastQueuedSeqno();
+ } // end synchronized
+
+ getStreamer().waitForAckedSeqno(toWaitFor);
+
+ // update the block length first time irrespective of flag
+ if (updateLength || getStreamer().getPersistBlocks().get()) {
+ synchronized (this) {
+ if (!getStreamer().streamerClosed()
+ && getStreamer().getBlock() != null) {
+ lastBlockLength = getStreamer().getBlock().getNumBytes();
+ }
+ }
+ }
+ // If 1) any new blocks were allocated since the last flush, or 2) to
+ // update length in NN is required, then persist block locations on
+ // namenode.
+ if (getStreamer().getPersistBlocks().getAndSet(false) || updateLength) {
+ try {
+ dfsClient.namenode.fsync(src, fileId, dfsClient.clientName,
+ lastBlockLength);
+ } catch (IOException ioe) {
+ DFSClient.LOG.warn("Unable to persist blocks in hflush for " + src, ioe);
+ // If we got an error here, it might be because some other thread called
+ // close before our hflush completed. In that case, we should throw an
+ // exception that the stream is closed.
+ checkClosed();
+ // If we aren't closed but failed to sync, we should expose that to the
+ // caller.
+ throw ioe;
+ }
+ }
+
+ synchronized(this) {
+ if (!getStreamer().streamerClosed()) {
+ getStreamer().setHflush();
+ }
+ }
+ } catch (InterruptedIOException interrupt) {
+ // This kind of error doesn't mean that the stream itself is broken - just the
+ // flushing thread got interrupted. So, we shouldn't close down the writer,
+ // but instead just propagate the error
+ throw interrupt;
+ } catch (IOException e) {
+ DFSClient.LOG.warn("Error while syncing", e);
+ synchronized (this) {
+ if (!isClosed()) {
+ getStreamer().getLastException().set(e);
+ closeThreads(true);
+ }
+ }
+ throw e;
+ }
+ }
+
+ /**
+ * @deprecated use {@link HdfsDataOutputStream#getCurrentBlockReplication()}.
+ */
+ @Deprecated
+ public synchronized int getNumCurrentReplicas() throws IOException {
+ return getCurrentBlockReplication();
+ }
+
+ /**
+ * Note that this is not a public API;
+ * use {@link HdfsDataOutputStream#getCurrentBlockReplication()} instead.
+ *
+ * @return the number of valid replicas of the current block
+ */
+ public synchronized int getCurrentBlockReplication() throws IOException {
+ dfsClient.checkOpen();
+ checkClosed();
+ if (getStreamer().streamerClosed()) {
+ return blockReplication; // no pipeline, return repl factor of file
+ }
+ DatanodeInfo[] currentNodes = getStreamer().getNodes();
+ if (currentNodes == null) {
+ return blockReplication; // no pipeline, return repl factor of file
+ }
+ return currentNodes.length;
+ }
+
+ /**
+ * Waits till all existing data is flushed and confirmations
+ * received from datanodes.
+ */
+ protected void flushInternal() throws IOException {
+ long toWaitFor;
+ synchronized (this) {
+ dfsClient.checkOpen();
+ checkClosed();
+ //
+ // If there is data in the current buffer, send it across
+ //
+ getStreamer().queuePacket(currentPacket);
+ currentPacket = null;
+ toWaitFor = getStreamer().getLastQueuedSeqno();
+ }
+
+ getStreamer().waitForAckedSeqno(toWaitFor);
+ }
+
+ protected synchronized void start() {
+ getStreamer().start();
+ }
+
+ /**
+ * Aborts this output stream and releases any system
+ * resources associated with this stream.
+ */
+ synchronized void abort() throws IOException {
+ if (isClosed()) {
+ return;
+ }
+ getStreamer().getLastException().set(new IOException("Lease timeout of "
+ + (dfsClient.getConf().getHdfsTimeout()/1000) + " seconds expired."));
+ closeThreads(true);
+ dfsClient.endFileLease(fileId);
+ }
+
+ boolean isClosed() {
+ return closed || getStreamer().streamerClosed();
+ }
+
+ void setClosed() {
+ closed = true;
+ getStreamer().release();
+ }
+
+ // shutdown datastreamer and responseprocessor threads.
+ // interrupt datastreamer if force is true
+ protected void closeThreads(boolean force) throws IOException {
+ try {
+ getStreamer().close(force);
+ getStreamer().join();
+ getStreamer().closeSocket();
+ } catch (InterruptedException e) {
+ throw new IOException("Failed to shutdown streamer");
+ } finally {
+ getStreamer().setSocketToNull();
+ setClosed();
+ }
+ }
+
+ /**
+ * Closes this output stream and releases any system
+ * resources associated with this stream.
+ */
+ @Override
+ public synchronized void close() throws IOException {
+ TraceScope scope =
+ dfsClient.getPathTraceScope("DFSOutputStream#close", src);
+ try {
+ closeImpl();
+ } finally {
+ scope.close();
+ }
+ }
+
+ protected synchronized void closeImpl() throws IOException {
+ if (isClosed()) {
+ getStreamer().getLastException().check(true);
+ return;
+ }
+
+ try {
+ flushBuffer(); // flush from all upper layers
+
+ if (currentPacket != null) {
+ enqueueCurrentPacket();
+ }
+
+ if (getStreamer().getBytesCurBlock() != 0) {
+ setCurrentPacketToEmpty();
+ }
+
+ flushInternal(); // flush all data to Datanodes
+ // get last block before destroying the streamer
+ ExtendedBlock lastBlock = getStreamer().getBlock();
+ closeThreads(false);
+ TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
+ try {
+ completeFile(lastBlock);
+ } finally {
+ scope.close();
+ }
+ dfsClient.endFileLease(fileId);
+ } catch (ClosedChannelException e) {
+ } finally {
+ setClosed();
+ }
+ }
+
+ // should be called holding (this) lock since setTestFilename() may
+ // be called during unit tests
+ protected void completeFile(ExtendedBlock last) throws IOException {
+ long localstart = Time.monotonicNow();
+ final DfsClientConf conf = dfsClient.getConf();
+ long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs();
+ boolean fileComplete = false;
+ int retries = conf.getNumBlockWriteLocateFollowingRetry();
+ while (!fileComplete) {
+ fileComplete =
+ dfsClient.namenode.complete(src, dfsClient.clientName, last, fileId);
+ if (!fileComplete) {
+ final int hdfsTimeout = conf.getHdfsTimeout();
+ if (!dfsClient.clientRunning
+ || (hdfsTimeout > 0
+ && localstart + hdfsTimeout < Time.monotonicNow())) {
+ String msg = "Unable to close file because dfsclient " +
+ " was unable to contact the HDFS servers." +
+ " clientRunning " + dfsClient.clientRunning +
+ " hdfsTimeout " + hdfsTimeout;
+ DFSClient.LOG.info(msg);
+ throw new IOException(msg);
+ }
+ try {
+ if (retries == 0) {
+ throw new IOException("Unable to close file because the last block"
+ + " does not have enough number of replicas.");
+ }
+ retries--;
+ Thread.sleep(sleeptime);
+ sleeptime *= 2;
+ if (Time.monotonicNow() - localstart > 5000) {
+ DFSClient.LOG.info("Could not complete " + src + " retrying...");
+ }
+ } catch (InterruptedException ie) {
+ DFSClient.LOG.warn("Caught exception ", ie);
+ }
+ }
+ }
+ }
+
+ @VisibleForTesting
+ public void setArtificialSlowdown(long period) {
+ getStreamer().setArtificialSlowdown(period);
+ }
+
+ @VisibleForTesting
+ public synchronized void setChunksPerPacket(int value) {
+ chunksPerPacket = Math.min(chunksPerPacket, value);
+ packetSize = (bytesPerChecksum + getChecksumSize()) * chunksPerPacket;
+ }
+
+ /**
+ * Returns the size of a file as it was when this stream was opened
+ */
+ public long getInitialLen() {
+ return initialFileSize;
+ }
+
+ /**
+ * @return the FileEncryptionInfo for this stream, or null if not encrypted.
+ */
+ public FileEncryptionInfo getFileEncryptionInfo() {
+ return fileEncryptionInfo;
+ }
+
+ /**
+ * Returns the access token currently used by streamer, for testing only
+ */
+ synchronized Token<BlockTokenIdentifier> getBlockToken() {
+ return getStreamer().getBlockToken();
+ }
+
+ @Override
+ public void setDropBehind(Boolean dropBehind) throws IOException {
+ CachingStrategy prevStrategy, nextStrategy;
+ // CachingStrategy is immutable. So build a new CachingStrategy with the
+ // modifications we want, and compare-and-swap it in.
+ do {
+ prevStrategy = this.cachingStrategy.get();
+ nextStrategy = new CachingStrategy.Builder(prevStrategy).
+ setDropBehind(dropBehind).build();
+ } while (!this.cachingStrategy.compareAndSet(prevStrategy, nextStrategy));
+ }
+
+ @VisibleForTesting
+ ExtendedBlock getBlock() {
+ return getStreamer().getBlock();
+ }
+
+ @VisibleForTesting
+ public long getFileId() {
+ return fileId;
+ }
+
+ /**
+ * Return the source of stream.
+ */
+ String getSrc() {
+ return src;
+ }
+
+ /**
+ * Returns the data streamer object.
+ */
+ protected DataStreamer getStreamer() {
+ return streamer;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
new file mode 100755
index 0000000..22055c3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
@@ -0,0 +1,345 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.BufferOverflowException;
+import java.nio.channels.ClosedChannelException;
+import java.util.Arrays;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader;
+import org.apache.hadoop.hdfs.util.ByteArrayManager;
+import org.apache.htrace.Span;
+
+/****************************************************************
+ * DFSPacket is used by DataStreamer and DFSOutputStream.
+ * DFSOutputStream generates packets and then ask DatStreamer
+ * to send them to datanodes.
+ ****************************************************************/
+
+@InterfaceAudience.Private
+class DFSPacket {
+ public static final long HEART_BEAT_SEQNO = -1L;
+ private static long[] EMPTY = new long[0];
+ private final long seqno; // sequence number of buffer in block
+ private final long offsetInBlock; // offset in block
+ private boolean syncBlock; // this packet forces the current block to disk
+ private int numChunks; // number of chunks currently in packet
+ private final int maxChunks; // max chunks in packet
+ private byte[] buf;
+ private final boolean lastPacketInBlock; // is this the last packet in block?
+
+ /**
+ * buf is pointed into like follows:
+ * (C is checksum data, D is payload data)
+ *
+ * [_________CCCCCCCCC________________DDDDDDDDDDDDDDDD___]
+ * ^ ^ ^ ^
+ * | checksumPos dataStart dataPos
+ * checksumStart
+ *
+ * Right before sending, we move the checksum data to immediately precede
+ * the actual data, and then insert the header into the buffer immediately
+ * preceding the checksum data, so we make sure to keep enough space in
+ * front of the checksum data to support the largest conceivable header.
+ */
+ private int checksumStart;
+ private int checksumPos;
+ private final int dataStart;
+ private int dataPos;
+ private long[] traceParents = EMPTY;
+ private int traceParentsUsed;
+ private Span span;
+
+ /**
+ * Create a new packet.
+ *
+ * @param buf the buffer storing data and checksums
+ * @param chunksPerPkt maximum number of chunks per packet.
+ * @param offsetInBlock offset in bytes into the HDFS block.
+ * @param seqno the sequence number of this packet
+ * @param checksumSize the size of checksum
+ * @param lastPacketInBlock if this is the last packet
+ */
+ DFSPacket(byte[] buf, int chunksPerPkt, long offsetInBlock, long seqno,
+ int checksumSize, boolean lastPacketInBlock) {
+ this.lastPacketInBlock = lastPacketInBlock;
+ this.numChunks = 0;
+ this.offsetInBlock = offsetInBlock;
+ this.seqno = seqno;
+
+ this.buf = buf;
+
+ checksumStart = PacketHeader.PKT_MAX_HEADER_LEN;
+ checksumPos = checksumStart;
+ dataStart = checksumStart + (chunksPerPkt * checksumSize);
+ dataPos = dataStart;
+ maxChunks = chunksPerPkt;
+ }
+
+ /**
+ * Write data to this packet.
+ *
+ * @param inarray input array of data
+ * @param off the offset of data to write
+ * @param len the length of data to write
+ * @throws ClosedChannelException
+ */
+ synchronized void writeData(byte[] inarray, int off, int len)
+ throws ClosedChannelException {
+ checkBuffer();
+ if (dataPos + len > buf.length) {
+ throw new BufferOverflowException();
+ }
+ System.arraycopy(inarray, off, buf, dataPos, len);
+ dataPos += len;
+ }
+
+ /**
+ * Write checksums to this packet
+ *
+ * @param inarray input array of checksums
+ * @param off the offset of checksums to write
+ * @param len the length of checksums to write
+ * @throws ClosedChannelException
+ */
+ synchronized void writeChecksum(byte[] inarray, int off, int len)
+ throws ClosedChannelException {
+ checkBuffer();
+ if (len == 0) {
+ return;
+ }
+ if (checksumPos + len > dataStart) {
+ throw new BufferOverflowException();
+ }
+ System.arraycopy(inarray, off, buf, checksumPos, len);
+ checksumPos += len;
+ }
+
+ /**
+ * Write the full packet, including the header, to the given output stream.
+ *
+ * @param stm
+ * @throws IOException
+ */
+ synchronized void writeTo(DataOutputStream stm) throws IOException {
+ checkBuffer();
+
+ final int dataLen = dataPos - dataStart;
+ final int checksumLen = checksumPos - checksumStart;
+ final int pktLen = HdfsConstants.BYTES_IN_INTEGER + dataLen + checksumLen;
+
+ PacketHeader header = new PacketHeader(
+ pktLen, offsetInBlock, seqno, lastPacketInBlock, dataLen, syncBlock);
+
+ if (checksumPos != dataStart) {
+ // Move the checksum to cover the gap. This can happen for the last
+ // packet or during an hflush/hsync call.
+ System.arraycopy(buf, checksumStart, buf,
+ dataStart - checksumLen , checksumLen);
+ checksumPos = dataStart;
+ checksumStart = checksumPos - checksumLen;
+ }
+
+ final int headerStart = checksumStart - header.getSerializedSize();
+ assert checksumStart + 1 >= header.getSerializedSize();
+ assert headerStart >= 0;
+ assert headerStart + header.getSerializedSize() == checksumStart;
+
+ // Copy the header data into the buffer immediately preceding the checksum
+ // data.
+ System.arraycopy(header.getBytes(), 0, buf, headerStart,
+ header.getSerializedSize());
+
+ // corrupt the data for testing.
+ if (DFSClientFaultInjector.get().corruptPacket()) {
+ buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
+ }
+
+ // Write the now contiguous full packet to the output stream.
+ stm.write(buf, headerStart, header.getSerializedSize() + checksumLen + dataLen);
+
+ // undo corruption.
+ if (DFSClientFaultInjector.get().uncorruptPacket()) {
+ buf[headerStart+header.getSerializedSize() + checksumLen + dataLen-1] ^= 0xff;
+ }
+ }
+
+ private synchronized void checkBuffer() throws ClosedChannelException {
+ if (buf == null) {
+ throw new ClosedChannelException();
+ }
+ }
+
+ /**
+ * Release the buffer in this packet to ByteArrayManager.
+ *
+ * @param bam
+ */
+ synchronized void releaseBuffer(ByteArrayManager bam) {
+ bam.release(buf);
+ buf = null;
+ }
+
+ /**
+ * get the packet's last byte's offset in the block
+ *
+ * @return the packet's last byte's offset in the block
+ */
+ synchronized long getLastByteOffsetBlock() {
+ return offsetInBlock + dataPos - dataStart;
+ }
+
+ /**
+ * Check if this packet is a heart beat packet
+ *
+ * @return true if the sequence number is HEART_BEAT_SEQNO
+ */
+ boolean isHeartbeatPacket() {
+ return seqno == HEART_BEAT_SEQNO;
+ }
+
+ /**
+ * check if this packet is the last packet in block
+ *
+ * @return true if the packet is the last packet
+ */
+ boolean isLastPacketInBlock(){
+ return lastPacketInBlock;
+ }
+
+ /**
+ * get sequence number of this packet
+ *
+ * @return the sequence number of this packet
+ */
+ long getSeqno(){
+ return seqno;
+ }
+
+ /**
+ * get the number of chunks this packet contains
+ *
+ * @return the number of chunks in this packet
+ */
+ synchronized int getNumChunks(){
+ return numChunks;
+ }
+
+ /**
+ * increase the number of chunks by one
+ */
+ synchronized void incNumChunks(){
+ numChunks++;
+ }
+
+ /**
+ * get the maximum number of packets
+ *
+ * @return the maximum number of packets
+ */
+ int getMaxChunks(){
+ return maxChunks;
+ }
+
+ /**
+ * set if to sync block
+ *
+ * @param syncBlock if to sync block
+ */
+ synchronized void setSyncBlock(boolean syncBlock){
+ this.syncBlock = syncBlock;
+ }
+
+ @Override
+ public String toString() {
+ return "packet seqno: " + this.seqno +
+ " offsetInBlock: " + this.offsetInBlock +
+ " lastPacketInBlock: " + this.lastPacketInBlock +
+ " lastByteOffsetInBlock: " + this.getLastByteOffsetBlock();
+ }
+
+ /**
+ * Add a trace parent span for this packet.<p/>
+ *
+ * Trace parent spans for a packet are the trace spans responsible for
+ * adding data to that packet. We store them as an array of longs for
+ * efficiency.<p/>
+ *
+ * Protected by the DFSOutputStream dataQueue lock.
+ */
+ public void addTraceParent(Span span) {
+ if (span == null) {
+ return;
+ }
+ addTraceParent(span.getSpanId());
+ }
+
+ public void addTraceParent(long id) {
+ if (traceParentsUsed == traceParents.length) {
+ int newLength = (traceParents.length == 0) ? 8 :
+ traceParents.length * 2;
+ traceParents = Arrays.copyOf(traceParents, newLength);
+ }
+ traceParents[traceParentsUsed] = id;
+ traceParentsUsed++;
+ }
+
+ /**
+ * Get the trace parent spans for this packet.<p/>
+ *
+ * Will always be non-null.<p/>
+ *
+ * Protected by the DFSOutputStream dataQueue lock.
+ */
+ public long[] getTraceParents() {
+ // Remove duplicates from the array.
+ int len = traceParentsUsed;
+ Arrays.sort(traceParents, 0, len);
+ int i = 0, j = 0;
+ long prevVal = 0; // 0 is not a valid span id
+ while (true) {
+ if (i == len) {
+ break;
+ }
+ long val = traceParents[i];
+ if (val != prevVal) {
+ traceParents[j] = val;
+ j++;
+ prevVal = val;
+ }
+ i++;
+ }
+ if (j < traceParents.length) {
+ traceParents = Arrays.copyOf(traceParents, j);
+ traceParentsUsed = traceParents.length;
+ }
+ return traceParents;
+ }
+
+ public void setTraceSpan(Span span) {
+ this.span = span;
+ }
+
+ public Span getTraceSpan() {
+ return span;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
index 359886e..e275afb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.crypto.key.KeyProvider;
import org.apache.hadoop.crypto.key.KeyProviderFactory;
import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
@@ -590,6 +591,29 @@ public class DFSUtilClient {
}
}
+ public static int getIoFileBufferSize(Configuration conf) {
+ return conf.getInt(
+ CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
+ CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
+ }
+
+ public static int getSmallBufferSize(Configuration conf) {
+ return Math.min(getIoFileBufferSize(conf) / 2, 512);
+ }
+
+ /**
+ * Probe for HDFS Encryption being enabled; this uses the value of
+ * the option {@link HdfsClientConfigKeys#DFS_ENCRYPTION_KEY_PROVIDER_URI},
+ * returning true if that property contains a non-empty, non-whitespace
+ * string.
+ * @param conf configuration to probe
+ * @return true if encryption is considered enabled.
+ */
+ public static boolean isHDFSEncryptionEnabled(Configuration conf) {
+ return !conf.getTrimmed(
+ HdfsClientConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, "").isEmpty();
+ }
+
public static InetSocketAddress getNNAddress(String address) {
return NetUtils.createSocketAddr(address,
HdfsClientConfigKeys.DFS_NAMENODE_RPC_PORT_DEFAULT);
[07/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and
related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
new file mode 100644
index 0000000..745ca7e
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsDataOutputStream.java
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.client;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.EnumSet;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.crypto.CryptoOutputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSOutputStream;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * The Hdfs implementation of {@link FSDataOutputStream}.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class HdfsDataOutputStream extends FSDataOutputStream {
+ public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats,
+ long startPosition) throws IOException {
+ super(out, stats, startPosition);
+ }
+
+ public HdfsDataOutputStream(DFSOutputStream out, FileSystem.Statistics stats
+ ) throws IOException {
+ this(out, stats, 0L);
+ }
+
+ public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats,
+ long startPosition) throws IOException {
+ super(out, stats, startPosition);
+ Preconditions.checkArgument(out.getWrappedStream() instanceof DFSOutputStream,
+ "CryptoOutputStream should wrap a DFSOutputStream");
+ }
+
+ public HdfsDataOutputStream(CryptoOutputStream out, FileSystem.Statistics stats)
+ throws IOException {
+ this(out, stats, 0L);
+ }
+
+ /**
+ * Get the actual number of replicas of the current block.
+ *
+ * This can be different from the designated replication factor of the file
+ * because the namenode does not maintain replication for the blocks which are
+ * currently being written to. Depending on the configuration, the client may
+ * continue to write to a block even if a few datanodes in the write pipeline
+ * have failed, or the client may add a new datanodes once a datanode has
+ * failed.
+ *
+ * @return the number of valid replicas of the current block
+ */
+ public synchronized int getCurrentBlockReplication() throws IOException {
+ OutputStream wrappedStream = getWrappedStream();
+ if (wrappedStream instanceof CryptoOutputStream) {
+ wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
+ }
+ return ((DFSOutputStream) wrappedStream).getCurrentBlockReplication();
+ }
+
+ /**
+ * Sync buffered data to DataNodes (flush to disk devices).
+ *
+ * @param syncFlags
+ * Indicate the detailed semantic and actions of the hsync.
+ * @throws IOException
+ * @see FSDataOutputStream#hsync()
+ */
+ public void hsync(EnumSet<SyncFlag> syncFlags) throws IOException {
+ OutputStream wrappedStream = getWrappedStream();
+ if (wrappedStream instanceof CryptoOutputStream) {
+ ((CryptoOutputStream) wrappedStream).flush();
+ wrappedStream = ((CryptoOutputStream) wrappedStream).getWrappedStream();
+ }
+ ((DFSOutputStream) wrappedStream).hsync(syncFlags);
+ }
+
+ public static enum SyncFlag {
+
+ /**
+ * When doing sync to DataNodes, also update the metadata (block length) in
+ * the NameNode.
+ */
+ UPDATE_LENGTH,
+
+ /**
+ * Sync the data to DataNode, close the current block, and allocate a new
+ * block
+ */
+ END_BLOCK;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
new file mode 100644
index 0000000..c3d2cfc
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/LeaseRenewer.java
@@ -0,0 +1,524 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.client.impl;
+
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.DFSClient;
+import org.apache.hadoop.hdfs.DFSOutputStream;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Time;
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * Used by {@link org.apache.hadoop.hdfs.DFSClient} for renewing file-being-written leases
+ * on the namenode.
+ * When a file is opened for write (create or append),
+ * namenode stores a file lease for recording the identity of the writer.
+ * The writer (i.e. the DFSClient) is required to renew the lease periodically.
+ * When the lease is not renewed before it expires,
+ * the namenode considers the writer as failed and then it may either let
+ * another writer to obtain the lease or close the file.
+ * </p>
+ * <p>
+ * This class also provides the following functionality:
+ * <ul>
+ * <li>
+ * It maintains a map from (namenode, user) pairs to lease renewers.
+ * The same {@link LeaseRenewer} instance is used for renewing lease
+ * for all the {@link org.apache.hadoop.hdfs.DFSClient} to the same namenode and the same user.
+ * </li>
+ * <li>
+ * Each renewer maintains a list of {@link org.apache.hadoop.hdfs.DFSClient}.
+ * Periodically the leases for all the clients are renewed.
+ * A client is removed from the list when the client is closed.
+ * </li>
+ * <li>
+ * A thread per namenode per user is used by the {@link LeaseRenewer}
+ * to renew the leases.
+ * </li>
+ * </ul>
+ * </p>
+ */
+@InterfaceAudience.Private
+public class LeaseRenewer {
+ static final Logger LOG = LoggerFactory.getLogger(LeaseRenewer.class);
+
+ static final long LEASE_RENEWER_GRACE_DEFAULT = 60*1000L;
+ static final long LEASE_RENEWER_SLEEP_DEFAULT = 1000L;
+
+ /** Get a {@link LeaseRenewer} instance */
+ public static LeaseRenewer getInstance(final String authority,
+ final UserGroupInformation ugi, final DFSClient dfsc) throws IOException {
+ final LeaseRenewer r = Factory.INSTANCE.get(authority, ugi);
+ r.addClient(dfsc);
+ return r;
+ }
+
+ /**
+ * A factory for sharing {@link LeaseRenewer} objects
+ * among {@link DFSClient} instances
+ * so that there is only one renewer per authority per user.
+ */
+ private static class Factory {
+ private static final Factory INSTANCE = new Factory();
+
+ private static class Key {
+ /** Namenode info */
+ final String authority;
+ /** User info */
+ final UserGroupInformation ugi;
+
+ private Key(final String authority, final UserGroupInformation ugi) {
+ if (authority == null) {
+ throw new HadoopIllegalArgumentException("authority == null");
+ } else if (ugi == null) {
+ throw new HadoopIllegalArgumentException("ugi == null");
+ }
+
+ this.authority = authority;
+ this.ugi = ugi;
+ }
+
+ @Override
+ public int hashCode() {
+ return authority.hashCode() ^ ugi.hashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj == this) {
+ return true;
+ }
+ if (obj != null && obj instanceof Key) {
+ final Key that = (Key)obj;
+ return this.authority.equals(that.authority)
+ && this.ugi.equals(that.ugi);
+ }
+ return false;
+ }
+
+ @Override
+ public String toString() {
+ return ugi.getShortUserName() + "@" + authority;
+ }
+ }
+
+ /** A map for per user per namenode renewers. */
+ private final Map<Key, LeaseRenewer> renewers = new HashMap<Key, LeaseRenewer>();
+
+ /** Get a renewer. */
+ private synchronized LeaseRenewer get(final String authority,
+ final UserGroupInformation ugi) {
+ final Key k = new Key(authority, ugi);
+ LeaseRenewer r = renewers.get(k);
+ if (r == null) {
+ r = new LeaseRenewer(k);
+ renewers.put(k, r);
+ }
+ return r;
+ }
+
+ /** Remove the given renewer. */
+ private synchronized void remove(final LeaseRenewer r) {
+ final LeaseRenewer stored = renewers.get(r.factorykey);
+ //Since a renewer may expire, the stored renewer can be different.
+ if (r == stored) {
+ if (!r.clientsRunning()) {
+ renewers.remove(r.factorykey);
+ }
+ }
+ }
+ }
+
+ /** The time in milliseconds that the map became empty. */
+ private long emptyTime = Long.MAX_VALUE;
+ /** A fixed lease renewal time period in milliseconds */
+ private long renewal = HdfsConstants.LEASE_SOFTLIMIT_PERIOD / 2;
+
+ /** A daemon for renewing lease */
+ private Daemon daemon = null;
+ /** Only the daemon with currentId should run. */
+ private int currentId = 0;
+
+ /**
+ * A period in milliseconds that the lease renewer thread should run
+ * after the map became empty.
+ * In other words,
+ * if the map is empty for a time period longer than the grace period,
+ * the renewer should terminate.
+ */
+ private long gracePeriod;
+ /**
+ * The time period in milliseconds
+ * that the renewer sleeps for each iteration.
+ */
+ private long sleepPeriod;
+
+ private final Factory.Key factorykey;
+
+ /** A list of clients corresponding to this renewer. */
+ private final List<DFSClient> dfsclients = new ArrayList<DFSClient>();
+
+ /**
+ * A stringified stack trace of the call stack when the Lease Renewer
+ * was instantiated. This is only generated if trace-level logging is
+ * enabled on this class.
+ */
+ private final String instantiationTrace;
+
+ private LeaseRenewer(Factory.Key factorykey) {
+ this.factorykey = factorykey;
+ unsyncSetGraceSleepPeriod(LEASE_RENEWER_GRACE_DEFAULT);
+
+ if (LOG.isTraceEnabled()) {
+ instantiationTrace = StringUtils.stringifyException(
+ new Throwable("TRACE"));
+ } else {
+ instantiationTrace = null;
+ }
+ }
+
+ /** @return the renewal time in milliseconds. */
+ private synchronized long getRenewalTime() {
+ return renewal;
+ }
+
+ /** Used for testing only. */
+ @VisibleForTesting
+ public synchronized void setRenewalTime(final long renewal) {
+ this.renewal = renewal;
+ }
+
+ /** Add a client. */
+ private synchronized void addClient(final DFSClient dfsc) {
+ for(DFSClient c : dfsclients) {
+ if (c == dfsc) {
+ //client already exists, nothing to do.
+ return;
+ }
+ }
+ //client not found, add it
+ dfsclients.add(dfsc);
+
+ //update renewal time
+ final int hdfsTimeout = dfsc.getConf().getHdfsTimeout();
+ if (hdfsTimeout > 0) {
+ final long half = hdfsTimeout/2;
+ if (half < renewal) {
+ this.renewal = half;
+ }
+ }
+ }
+
+ private synchronized boolean clientsRunning() {
+ for(Iterator<DFSClient> i = dfsclients.iterator(); i.hasNext(); ) {
+ if (!i.next().isClientRunning()) {
+ i.remove();
+ }
+ }
+ return !dfsclients.isEmpty();
+ }
+
+ private synchronized long getSleepPeriod() {
+ return sleepPeriod;
+ }
+
+ /** Set the grace period and adjust the sleep period accordingly. */
+ synchronized void setGraceSleepPeriod(final long gracePeriod) {
+ unsyncSetGraceSleepPeriod(gracePeriod);
+ }
+
+ private void unsyncSetGraceSleepPeriod(final long gracePeriod) {
+ if (gracePeriod < 100L) {
+ throw new HadoopIllegalArgumentException(gracePeriod
+ + " = gracePeriod < 100ms is too small.");
+ }
+ this.gracePeriod = gracePeriod;
+ final long half = gracePeriod/2;
+ this.sleepPeriod = half < LEASE_RENEWER_SLEEP_DEFAULT?
+ half: LEASE_RENEWER_SLEEP_DEFAULT;
+ }
+
+ /** Is the daemon running? */
+ synchronized boolean isRunning() {
+ return daemon != null && daemon.isAlive();
+ }
+
+ /** Does this renewer have nothing to renew? */
+ public boolean isEmpty() {
+ return dfsclients.isEmpty();
+ }
+
+ /** Used only by tests */
+ synchronized String getDaemonName() {
+ return daemon.getName();
+ }
+
+ /** Is the empty period longer than the grace period? */
+ private synchronized boolean isRenewerExpired() {
+ return emptyTime != Long.MAX_VALUE
+ && Time.monotonicNow() - emptyTime > gracePeriod;
+ }
+
+ public synchronized void put(final long inodeId, final DFSOutputStream out,
+ final DFSClient dfsc) {
+ if (dfsc.isClientRunning()) {
+ if (!isRunning() || isRenewerExpired()) {
+ //start a new deamon with a new id.
+ final int id = ++currentId;
+ daemon = new Daemon(new Runnable() {
+ @Override
+ public void run() {
+ try {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Lease renewer daemon for " + clientsString()
+ + " with renew id " + id + " started");
+ }
+ LeaseRenewer.this.run(id);
+ } catch(InterruptedException e) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(LeaseRenewer.this.getClass().getSimpleName()
+ + " is interrupted.", e);
+ }
+ } finally {
+ synchronized(LeaseRenewer.this) {
+ Factory.INSTANCE.remove(LeaseRenewer.this);
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Lease renewer daemon for " + clientsString()
+ + " with renew id " + id + " exited");
+ }
+ }
+ }
+
+ @Override
+ public String toString() {
+ return String.valueOf(LeaseRenewer.this);
+ }
+ });
+ daemon.start();
+ }
+ dfsc.putFileBeingWritten(inodeId, out);
+ emptyTime = Long.MAX_VALUE;
+ }
+ }
+
+ @VisibleForTesting
+ synchronized void setEmptyTime(long time) {
+ emptyTime = time;
+ }
+
+ /** Close a file. */
+ public void closeFile(final long inodeId, final DFSClient dfsc) {
+ dfsc.removeFileBeingWritten(inodeId);
+
+ synchronized(this) {
+ if (dfsc.isFilesBeingWrittenEmpty()) {
+ dfsclients.remove(dfsc);
+ }
+ //update emptyTime if necessary
+ if (emptyTime == Long.MAX_VALUE) {
+ for(DFSClient c : dfsclients) {
+ if (!c.isFilesBeingWrittenEmpty()) {
+ //found a non-empty file-being-written map
+ return;
+ }
+ }
+ //discover the first time that all file-being-written maps are empty.
+ emptyTime = Time.monotonicNow();
+ }
+ }
+ }
+
+ /** Close the given client. */
+ public synchronized void closeClient(final DFSClient dfsc) {
+ dfsclients.remove(dfsc);
+ if (dfsclients.isEmpty()) {
+ if (!isRunning() || isRenewerExpired()) {
+ Factory.INSTANCE.remove(LeaseRenewer.this);
+ return;
+ }
+ if (emptyTime == Long.MAX_VALUE) {
+ //discover the first time that the client list is empty.
+ emptyTime = Time.monotonicNow();
+ }
+ }
+
+ //update renewal time
+ if (renewal == dfsc.getConf().getHdfsTimeout()/2) {
+ long min = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
+ for(DFSClient c : dfsclients) {
+ final int timeout = c.getConf().getHdfsTimeout();
+ if (timeout > 0 && timeout < min) {
+ min = timeout;
+ }
+ }
+ renewal = min/2;
+ }
+ }
+
+ public void interruptAndJoin() throws InterruptedException {
+ Daemon daemonCopy = null;
+ synchronized (this) {
+ if (isRunning()) {
+ daemon.interrupt();
+ daemonCopy = daemon;
+ }
+ }
+
+ if (daemonCopy != null) {
+ if(LOG.isDebugEnabled()) {
+ LOG.debug("Wait for lease checker to terminate");
+ }
+ daemonCopy.join();
+ }
+ }
+
+ private void renew() throws IOException {
+ final List<DFSClient> copies;
+ synchronized(this) {
+ copies = new ArrayList<DFSClient>(dfsclients);
+ }
+ //sort the client names for finding out repeated names.
+ Collections.sort(copies, new Comparator<DFSClient>() {
+ @Override
+ public int compare(final DFSClient left, final DFSClient right) {
+ return left.getClientName().compareTo(right.getClientName());
+ }
+ });
+ String previousName = "";
+ for(int i = 0; i < copies.size(); i++) {
+ final DFSClient c = copies.get(i);
+ //skip if current client name is the same as the previous name.
+ if (!c.getClientName().equals(previousName)) {
+ if (!c.renewLease()) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Did not renew lease for client " +
+ c);
+ }
+ continue;
+ }
+ previousName = c.getClientName();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Lease renewed for client " + previousName);
+ }
+ }
+ }
+ }
+
+ /**
+ * Periodically check in with the namenode and renew all the leases
+ * when the lease period is half over.
+ */
+ private void run(final int id) throws InterruptedException {
+ for(long lastRenewed = Time.monotonicNow(); !Thread.interrupted();
+ Thread.sleep(getSleepPeriod())) {
+ final long elapsed = Time.monotonicNow() - lastRenewed;
+ if (elapsed >= getRenewalTime()) {
+ try {
+ renew();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Lease renewer daemon for " + clientsString()
+ + " with renew id " + id + " executed");
+ }
+ lastRenewed = Time.monotonicNow();
+ } catch (SocketTimeoutException ie) {
+ LOG.warn("Failed to renew lease for " + clientsString() + " for "
+ + (elapsed/1000) + " seconds. Aborting ...", ie);
+ synchronized (this) {
+ while (!dfsclients.isEmpty()) {
+ DFSClient dfsClient = dfsclients.get(0);
+ dfsClient.closeAllFilesBeingWritten(true);
+ closeClient(dfsClient);
+ }
+ //Expire the current LeaseRenewer thread.
+ emptyTime = 0;
+ }
+ break;
+ } catch (IOException ie) {
+ LOG.warn("Failed to renew lease for " + clientsString() + " for "
+ + (elapsed/1000) + " seconds. Will retry shortly ...", ie);
+ }
+ }
+
+ synchronized(this) {
+ if (id != currentId || isRenewerExpired()) {
+ if (LOG.isDebugEnabled()) {
+ if (id != currentId) {
+ LOG.debug("Lease renewer daemon for " + clientsString()
+ + " with renew id " + id + " is not current");
+ } else {
+ LOG.debug("Lease renewer daemon for " + clientsString()
+ + " with renew id " + id + " expired");
+ }
+ }
+ //no longer the current daemon or expired
+ return;
+ }
+
+ // if no clients are in running state or there is no more clients
+ // registered with this renewer, stop the daemon after the grace
+ // period.
+ if (!clientsRunning() && emptyTime == Long.MAX_VALUE) {
+ emptyTime = Time.monotonicNow();
+ }
+ }
+ }
+ }
+
+ @Override
+ public String toString() {
+ String s = getClass().getSimpleName() + ":" + factorykey;
+ if (LOG.isTraceEnabled()) {
+ return s + ", clients=" + clientsString()
+ + ", created at " + instantiationTrace;
+ }
+ return s;
+ }
+
+ /** Get the names of all clients */
+ private synchronized String clientsString() {
+ if (dfsclients.isEmpty()) {
+ return "[]";
+ } else {
+ final StringBuilder b = new StringBuilder("[").append(
+ dfsclients.get(0).getClientName());
+ for(int i = 1; i < dfsclients.size(); i++) {
+ b.append(", ").append(dfsclients.get(i).getClientName());
+ }
+ return b.append("]").toString();
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java
new file mode 100644
index 0000000..e4b51c5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/inotify/MissingEventsException.java
@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.inotify;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public class MissingEventsException extends Exception {
+ private static final long serialVersionUID = 1L;
+
+ private long expectedTxid;
+ private long actualTxid;
+
+ public MissingEventsException() {}
+
+ public MissingEventsException(long expectedTxid, long actualTxid) {
+ this.expectedTxid = expectedTxid;
+ this.actualTxid = actualTxid;
+ }
+
+ public long getExpectedTxid() {
+ return expectedTxid;
+ }
+
+ public long getActualTxid() {
+ return actualTxid;
+ }
+
+ @Override
+ public String toString() {
+ return "We expected the next batch of events to start with transaction ID "
+ + expectedTxid + ", but it instead started with transaction ID " +
+ actualTxid + ". Most likely the intervening transactions were cleaned "
+ + "up as part of checkpointing.";
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java
new file mode 100644
index 0000000..1210999
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/AclException.java
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+/**
+ * Indicates a failure manipulating an ACL.
+ */
+@InterfaceAudience.Private
+public class AclException extends IOException {
+ private static final long serialVersionUID = 1L;
+
+ /**
+ * Creates a new AclException.
+ *
+ * @param message String message
+ */
+ public AclException(String message) {
+ super(message);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
new file mode 100644
index 0000000..923cdb4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CacheDirectiveIterator.java
@@ -0,0 +1,130 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.apache.hadoop.fs.InvalidRequestException;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * CacheDirectiveIterator is a remote iterator that iterates cache directives.
+ * It supports retrying in case of namenode failover.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class CacheDirectiveIterator
+ extends BatchedRemoteIterator<Long, CacheDirectiveEntry> {
+
+ private CacheDirectiveInfo filter;
+ private final ClientProtocol namenode;
+ private final Sampler<?> traceSampler;
+
+ public CacheDirectiveIterator(ClientProtocol namenode,
+ CacheDirectiveInfo filter, Sampler<?> traceSampler) {
+ super(0L);
+ this.namenode = namenode;
+ this.filter = filter;
+ this.traceSampler = traceSampler;
+ }
+
+ private static CacheDirectiveInfo removeIdFromFilter(CacheDirectiveInfo filter) {
+ CacheDirectiveInfo.Builder builder = new CacheDirectiveInfo.Builder(filter);
+ builder.setId(null);
+ return builder.build();
+ }
+
+ /**
+ * Used for compatibility when communicating with a server version that
+ * does not support filtering directives by ID.
+ */
+ private static class SingleEntry implements
+ BatchedEntries<CacheDirectiveEntry> {
+
+ private final CacheDirectiveEntry entry;
+
+ public SingleEntry(final CacheDirectiveEntry entry) {
+ this.entry = entry;
+ }
+
+ @Override
+ public CacheDirectiveEntry get(int i) {
+ if (i > 0) {
+ return null;
+ }
+ return entry;
+ }
+
+ @Override
+ public int size() {
+ return 1;
+ }
+
+ @Override
+ public boolean hasMore() {
+ return false;
+ }
+ }
+
+ @Override
+ public BatchedEntries<CacheDirectiveEntry> makeRequest(Long prevKey)
+ throws IOException {
+ BatchedEntries<CacheDirectiveEntry> entries = null;
+ TraceScope scope = Trace.startSpan("listCacheDirectives", traceSampler);
+ try {
+ entries = namenode.listCacheDirectives(prevKey, filter);
+ } catch (IOException e) {
+ if (e.getMessage().contains("Filtering by ID is unsupported")) {
+ // Retry case for old servers, do the filtering client-side
+ long id = filter.getId();
+ filter = removeIdFromFilter(filter);
+ // Using id - 1 as prevId should get us a window containing the id
+ // This is somewhat brittle, since it depends on directives being
+ // returned in order of ascending ID.
+ entries = namenode.listCacheDirectives(id - 1, filter);
+ for (int i=0; i<entries.size(); i++) {
+ CacheDirectiveEntry entry = entries.get(i);
+ if (entry.getInfo().getId().equals((Long)id)) {
+ return new SingleEntry(entry);
+ }
+ }
+ throw new RemoteException(InvalidRequestException.class.getName(),
+ "Did not find requested id " + id);
+ }
+ throw e;
+ } finally {
+ scope.close();
+ }
+ Preconditions.checkNotNull(entries);
+ return entries;
+ }
+
+ @Override
+ public Long elementToPrevKey(CacheDirectiveEntry entry) {
+ return entry.getInfo().getId();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
new file mode 100644
index 0000000..e9481f7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/CachePoolIterator.java
@@ -0,0 +1,63 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+/**
+ * CachePoolIterator is a remote iterator that iterates cache pools.
+ * It supports retrying in case of namenode failover.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class CachePoolIterator
+ extends BatchedRemoteIterator<String, CachePoolEntry> {
+
+ private final ClientProtocol namenode;
+ private final Sampler traceSampler;
+
+ public CachePoolIterator(ClientProtocol namenode, Sampler traceSampler) {
+ super("");
+ this.namenode = namenode;
+ this.traceSampler = traceSampler;
+ }
+
+ @Override
+ public BatchedEntries<CachePoolEntry> makeRequest(String prevKey)
+ throws IOException {
+ TraceScope scope = Trace.startSpan("listCachePools", traceSampler);
+ try {
+ return namenode.listCachePools(prevKey);
+ } finally {
+ scope.close();
+ }
+ }
+
+ @Override
+ public String elementToPrevKey(CachePoolEntry entry) {
+ return entry.getInfo().getPoolName();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
new file mode 100644
index 0000000..0141215
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/EncryptionZoneIterator.java
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.BatchedRemoteIterator;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+/**
+ * EncryptionZoneIterator is a remote iterator that iterates over encryption
+ * zones. It supports retrying in case of namenode failover.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class EncryptionZoneIterator
+ extends BatchedRemoteIterator<Long, EncryptionZone> {
+
+ private final ClientProtocol namenode;
+ private final Sampler<?> traceSampler;
+
+ public EncryptionZoneIterator(ClientProtocol namenode,
+ Sampler<?> traceSampler) {
+ super(Long.valueOf(0));
+ this.namenode = namenode;
+ this.traceSampler = traceSampler;
+ }
+
+ @Override
+ public BatchedEntries<EncryptionZone> makeRequest(Long prevId)
+ throws IOException {
+ TraceScope scope = Trace.startSpan("listEncryptionZones", traceSampler);
+ try {
+ return namenode.listEncryptionZones(prevId);
+ } finally {
+ scope.close();
+ }
+ }
+
+ @Override
+ public Long elementToPrevKey(EncryptionZone entry) {
+ return entry.getId();
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java
new file mode 100644
index 0000000..25084c7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/QuotaByStorageTypeExceededException.java
@@ -0,0 +1,56 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StorageType;
+
+import static org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix.long2String;
+
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class QuotaByStorageTypeExceededException extends QuotaExceededException {
+ protected static final long serialVersionUID = 1L;
+ protected StorageType type;
+
+ public QuotaByStorageTypeExceededException() {}
+
+ public QuotaByStorageTypeExceededException(String msg) {
+ super(msg);
+ }
+
+ public QuotaByStorageTypeExceededException(long quota, long count, StorageType type) {
+ super(quota, count);
+ this.type = type;
+ }
+
+ @Override
+ public String getMessage() {
+ String msg = super.getMessage();
+ if (msg == null) {
+ return "Quota by storage type : " + type.toString() +
+ " on path : " + (pathName==null ? "": pathName) +
+ " is exceeded. quota = " + long2String(quota, "B", 2) +
+ " but space consumed = " + long2String(count, "B", 2);
+ } else {
+ return msg;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
new file mode 100644
index 0000000..03fb704
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/UnresolvedPathException.java
@@ -0,0 +1,87 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.UnresolvedLinkException;
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Thrown when a symbolic link is encountered in a path.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public final class UnresolvedPathException extends UnresolvedLinkException {
+ private static final long serialVersionUID = 1L;
+ private String path; // The path containing the link
+ private String preceding; // The path part preceding the link
+ private String remainder; // The path part following the link
+ private String linkTarget; // The link's target
+
+ /**
+ * Used by RemoteException to instantiate an UnresolvedPathException.
+ */
+ public UnresolvedPathException(String msg) {
+ super(msg);
+ }
+
+ public UnresolvedPathException(String path, String preceding,
+ String remainder, String linkTarget) {
+ this.path = path;
+ this.preceding = preceding;
+ this.remainder = remainder;
+ this.linkTarget = linkTarget;
+ }
+
+ /**
+ * Return a path with the link resolved with the target.
+ */
+ public Path getResolvedPath() throws IOException {
+ // If the path is absolute we cam throw out the preceding part and
+ // just append the remainder to the target, otherwise append each
+ // piece to resolve the link in path.
+ boolean noRemainder = (remainder == null || "".equals(remainder));
+ Path target = new Path(linkTarget);
+ if (target.isUriPathAbsolute()) {
+ return noRemainder ? target : new Path(target, remainder);
+ } else {
+ return noRemainder
+ ? new Path(preceding, target)
+ : new Path(new Path(preceding, linkTarget), remainder);
+ }
+ }
+
+ @Override
+ public String getMessage() {
+ String msg = super.getMessage();
+ if (msg != null) {
+ return msg;
+ }
+ String myMsg = "Unresolved path " + path;
+ try {
+ return getResolvedPath().toString();
+ } catch (IOException e) {
+ // Ignore
+ }
+ return myMsg;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
new file mode 100644
index 0000000..c69986a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/datatransfer/ReplaceDatanodeOnFailure.java
@@ -0,0 +1,200 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.protocol.datatransfer;
+
+import org.apache.hadoop.HadoopIllegalArgumentException;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+
+/**
+ * The setting of replace-datanode-on-failure feature.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class ReplaceDatanodeOnFailure {
+ /** The replacement policies */
+ public enum Policy {
+ /** The feature is disabled in the entire site. */
+ DISABLE(Condition.FALSE),
+ /** Never add a new datanode. */
+ NEVER(Condition.FALSE),
+ /** @see ReplaceDatanodeOnFailure.Condition#DEFAULT */
+ DEFAULT(Condition.DEFAULT),
+ /** Always add a new datanode when an existing datanode is removed. */
+ ALWAYS(Condition.TRUE);
+
+ private final Condition condition;
+
+ private Policy(Condition condition) {
+ this.condition = condition;
+ }
+
+ Condition getCondition() {
+ return condition;
+ }
+ }
+
+ /** Datanode replacement condition */
+ private static interface Condition {
+ /** Return true unconditionally. */
+ static final Condition TRUE = new Condition() {
+ @Override
+ public boolean satisfy(short replication, DatanodeInfo[] existings,
+ int nExistings, boolean isAppend, boolean isHflushed) {
+ return true;
+ }
+ };
+
+ /** Return false unconditionally. */
+ static final Condition FALSE = new Condition() {
+ @Override
+ public boolean satisfy(short replication, DatanodeInfo[] existings,
+ int nExistings, boolean isAppend, boolean isHflushed) {
+ return false;
+ }
+ };
+
+ /**
+ * DEFAULT condition:
+ * Let r be the replication number.
+ * Let n be the number of existing datanodes.
+ * Add a new datanode only if r >= 3 and either
+ * (1) floor(r/2) >= n; or
+ * (2) r > n and the block is hflushed/appended.
+ */
+ static final Condition DEFAULT = new Condition() {
+ @Override
+ public boolean satisfy(final short replication,
+ final DatanodeInfo[] existings, final int n, final boolean isAppend,
+ final boolean isHflushed) {
+ if (replication < 3) {
+ return false;
+ } else {
+ if (n <= (replication/2)) {
+ return true;
+ } else {
+ return isAppend || isHflushed;
+ }
+ }
+ }
+ };
+
+ /** Is the condition satisfied? */
+ public boolean satisfy(short replication, DatanodeInfo[] existings,
+ int nExistings, boolean isAppend, boolean isHflushed);
+ }
+
+ private final Policy policy;
+ private final boolean bestEffort;
+
+ public ReplaceDatanodeOnFailure(Policy policy, boolean bestEffort) {
+ this.policy = policy;
+ this.bestEffort = bestEffort;
+ }
+
+ /** Check if the feature is enabled. */
+ public void checkEnabled() {
+ if (policy == Policy.DISABLE) {
+ throw new UnsupportedOperationException(
+ "This feature is disabled. Please refer to "
+ + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY
+ + " configuration property.");
+ }
+ }
+
+ /**
+ * Best effort means that the client will try to replace the failed datanode
+ * (provided that the policy is satisfied), however, it will continue the
+ * write operation in case that the datanode replacement also fails.
+ *
+ * @return Suppose the datanode replacement fails.
+ * false: An exception should be thrown so that the write will fail.
+ * true : The write should be resumed with the remaining datandoes.
+ */
+ public boolean isBestEffort() {
+ return bestEffort;
+ }
+
+ /** Does it need a replacement according to the policy? */
+ public boolean satisfy(
+ final short replication, final DatanodeInfo[] existings,
+ final boolean isAppend, final boolean isHflushed) {
+ final int n = existings == null? 0: existings.length;
+ if (n == 0 || n >= replication) {
+ //don't need to add datanode for any policy.
+ return false;
+ } else {
+ return policy.getCondition().satisfy(
+ replication, existings, n, isAppend, isHflushed);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return policy.toString();
+ }
+
+ /** Get the setting from configuration. */
+ public static ReplaceDatanodeOnFailure get(final Configuration conf) {
+ final Policy policy = getPolicy(conf);
+ final boolean bestEffort = conf.getBoolean(
+ HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
+ HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_DEFAULT);
+
+ return new ReplaceDatanodeOnFailure(policy, bestEffort);
+ }
+
+ private static Policy getPolicy(final Configuration conf) {
+ final boolean enabled = conf.getBoolean(
+ HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY,
+ HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_DEFAULT);
+ if (!enabled) {
+ return Policy.DISABLE;
+ }
+
+ final String policy = conf.get(
+ HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY,
+ HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_DEFAULT);
+ for(int i = 1; i < Policy.values().length; i++) {
+ final Policy p = Policy.values()[i];
+ if (p.name().equalsIgnoreCase(policy)) {
+ return p;
+ }
+ }
+ throw new HadoopIllegalArgumentException("Illegal configuration value for "
+ + HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY
+ + ": " + policy);
+ }
+
+ /** Write the setting to configuration. */
+ public static void write(final Policy policy,
+ final boolean bestEffort, final Configuration conf) {
+ conf.setBoolean(
+ HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.ENABLE_KEY,
+ policy != Policy.DISABLE);
+ conf.set(
+ HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.POLICY_KEY,
+ policy.name());
+ conf.setBoolean(
+ HdfsClientConfigKeys.BlockWrite.ReplaceDatanodeOnFailure.BEST_EFFORT_KEY,
+ bestEffort);
+ }
+}
\ No newline at end of file
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
new file mode 100644
index 0000000..b159d3a
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/datanode/ReplicaNotFoundException.java
@@ -0,0 +1,53 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.apache.hadoop.hdfs.server.datanode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+
+/**
+ * Exception indicating that DataNode does not have a replica
+ * that matches the target block.
+ */
+public class ReplicaNotFoundException extends IOException {
+ private static final long serialVersionUID = 1L;
+ public final static String NON_RBW_REPLICA = "Cannot recover a non-RBW replica ";
+ public final static String UNFINALIZED_REPLICA =
+ "Cannot append to an unfinalized replica ";
+ public final static String UNFINALIZED_AND_NONRBW_REPLICA =
+ "Cannot recover append/close to a replica that's not FINALIZED and not RBW ";
+ public final static String NON_EXISTENT_REPLICA =
+ "Cannot append to a non-existent replica ";
+ public final static String UNEXPECTED_GS_REPLICA =
+ "Cannot append to a replica with unexpected generation stamp ";
+
+ public ReplicaNotFoundException() {
+ super();
+ }
+
+ public ReplicaNotFoundException(ExtendedBlock b) {
+ super("Replica not found for " + b);
+ }
+
+ public ReplicaNotFoundException(String msg) {
+ super(msg);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
new file mode 100644
index 0000000..0bdd2a5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/server/namenode/RetryStartFileException.java
@@ -0,0 +1,36 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+@InterfaceAudience.Private
+public class RetryStartFileException extends IOException {
+ private static final long serialVersionUID = 1L;
+
+ public RetryStartFileException() {
+ super("Preconditions for creating a file failed because of a " +
+ "transient error, retry create later.");
+ }
+
+ public RetryStartFileException(String s) {
+ super(s);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index c3a1668..38af55b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -629,6 +629,9 @@ Release 2.8.0 - UNRELEASED
HDFS-8873. Allow the directoryScanner to be rate-limited (Daniel Templeton
via Colin P. McCabe)
+ HDFS-8053. Move DFSIn/OutputStream and related classes to
+ hadoop-hdfs-client. (Mingliang Liu via wheat9)
+
OPTIMIZATIONS
HDFS-8026. Trace FSOutputSummer#writeChecksumChunks rather than
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
index 60029e0..c88c4c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/dev-support/findbugsExcludeFile.xml
@@ -74,15 +74,6 @@
</Match>
<!--
- ResponseProccessor is thread that is designed to catch RuntimeException.
- -->
- <Match>
- <Class name="org.apache.hadoop.hdfs.DataStreamer$ResponseProcessor" />
- <Method name="run" />
- <Bug pattern="REC_CATCH_EXCEPTION" />
- </Match>
-
- <!--
lastAppliedTxid is carefully unsynchronized in the BackupNode in a couple spots.
See the comments in BackupImage for justification.
-->
@@ -196,14 +187,4 @@
<Method name="assertAllResultsEqual" />
<Bug pattern="NP_LOAD_OF_KNOWN_NULL_VALUE" />
</Match>
-
- <!--
- We use a separate lock to guard cachingStrategy in order to separate
- locks for p-reads from seek + read invocations.
- -->
- <Match>
- <Class name="org.apache.hadoop.hdfs.DFSInputStream" />
- <Field name="cachingStrategy" />
- <Bug pattern="IS2_INCONSISTENT_SYNC" />
- </Match>
</FindBugsFilter>
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java
deleted file mode 100644
index 2200994..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.fs;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Wrapper for {@link BlockLocation} that also adds {@link VolumeId} volume
- * location information for each replica.
- */
-@InterfaceStability.Unstable
-@InterfaceAudience.Public
-@Deprecated
-public class BlockStorageLocation extends BlockLocation {
-
- private final VolumeId[] volumeIds;
-
- public BlockStorageLocation(BlockLocation loc, VolumeId[] volumeIds)
- throws IOException {
- // Initialize with data from passed in BlockLocation
- super(loc.getNames(), loc.getHosts(), loc.getTopologyPaths(), loc
- .getOffset(), loc.getLength(), loc.isCorrupt());
- this.volumeIds = volumeIds;
- }
-
- /**
- * Gets the list of {@link VolumeId} corresponding to the block's replicas.
- *
- * @return volumeIds list of VolumeId for the block's replicas
- */
- public VolumeId[] getVolumeIds() {
- return volumeIds;
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
deleted file mode 100644
index 0ccacda..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.fs;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-
-/**
- * Wrapper for {@link BlockLocation} that also includes a {@link LocatedBlock},
- * allowing more detailed queries to the datanode about a block.
- *
- */
-@InterfaceAudience.Private
-@InterfaceStability.Unstable
-public class HdfsBlockLocation extends BlockLocation {
-
- private final LocatedBlock block;
-
- public HdfsBlockLocation(BlockLocation loc, LocatedBlock block)
- throws IOException {
- // Initialize with data from passed in BlockLocation
- super(loc);
- this.block = block;
- }
-
- public LocatedBlock getLocatedBlock() {
- return block;
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
deleted file mode 100644
index 6e9d3d7..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.fs;
-
-import org.apache.commons.lang.builder.EqualsBuilder;
-import org.apache.commons.lang.builder.HashCodeBuilder;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.util.StringUtils;
-
-import com.google.common.base.Preconditions;
-
-/**
- * HDFS-specific volume identifier which implements {@link VolumeId}. Can be
- * used to differentiate between the data directories on a single datanode. This
- * identifier is only unique on a per-datanode basis.
- */
-@InterfaceStability.Unstable
-@InterfaceAudience.Public
-public class HdfsVolumeId implements VolumeId {
-
- private final byte[] id;
-
- public HdfsVolumeId(byte[] id) {
- Preconditions.checkNotNull(id, "id cannot be null");
- this.id = id;
- }
-
- @Override
- public int compareTo(VolumeId arg0) {
- if (arg0 == null) {
- return 1;
- }
- return hashCode() - arg0.hashCode();
- }
-
- @Override
- public int hashCode() {
- return new HashCodeBuilder().append(id).toHashCode();
- }
-
- @Override
- public boolean equals(Object obj) {
- if (obj == null || obj.getClass() != getClass()) {
- return false;
- }
- if (obj == this) {
- return true;
- }
- HdfsVolumeId that = (HdfsVolumeId) obj;
- return new EqualsBuilder().append(this.id, that.id).isEquals();
- }
-
- @Override
- public String toString() {
- return StringUtils.byteToHexString(id);
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/VolumeId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/VolumeId.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/VolumeId.java
deleted file mode 100644
index e56e304..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/fs/VolumeId.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.fs;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * Opaque interface that identifies a disk location. Subclasses
- * should implement {@link Comparable} and override both equals and hashCode.
- */
-@InterfaceStability.Unstable
-@InterfaceAudience.Public
-public interface VolumeId extends Comparable<VolumeId> {
-
- @Override
- abstract public int compareTo(VolumeId arg0);
-
- @Override
- abstract public int hashCode();
-
- @Override
- abstract public boolean equals(Object obj);
-
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
deleted file mode 100644
index 7bba8a4..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs;
-
-import java.io.IOException;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-
-/**
- * This exception is thrown when a read encounters a block that has no locations
- * associated with it.
- */
-@InterfaceAudience.Private
-@InterfaceStability.Evolving
-public class BlockMissingException extends IOException {
-
- private static final long serialVersionUID = 1L;
-
- private final String filename;
- private final long offset;
-
- /**
- * An exception that indicates that file was corrupted.
- * @param filename name of corrupted file
- * @param description a description of the corruption details
- */
- public BlockMissingException(String filename, String description, long offset) {
- super(description);
- this.filename = filename;
- this.offset = offset;
- }
-
- /**
- * Returns the name of the corrupted file.
- * @return name of corrupted file
- */
- public String getFile() {
- return filename;
- }
-
- /**
- * Returns the offset at which this file is corrupted
- * @return offset of corrupted file
- */
- public long getOffset() {
- return offset;
- }
-}
[04/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and
related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Posted by wh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
deleted file mode 100644
index 5392c66..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClientFaultInjector.java
+++ /dev/null
@@ -1,57 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-/**
- * Used for injecting faults in DFSClient and DFSOutputStream tests.
- * Calls into this are a no-op in production code.
- */
-@VisibleForTesting
-@InterfaceAudience.Private
-public class DFSClientFaultInjector {
- public static DFSClientFaultInjector instance = new DFSClientFaultInjector();
- public static AtomicLong exceptionNum = new AtomicLong(0);
-
- public static DFSClientFaultInjector get() {
- return instance;
- }
-
- public boolean corruptPacket() {
- return false;
- }
-
- public boolean uncorruptPacket() {
- return false;
- }
-
- public boolean failPacket() {
- return false;
- }
-
- public void startFetchFromDatanode() {}
-
- public void fetchFromDatanodeException() {}
-
- public void readFromDatanodeDelay() {}
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index c9f23f9..0ba174f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -18,8 +18,6 @@
package org.apache.hadoop.hdfs;
-import java.util.concurrent.TimeUnit;
-
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
deleted file mode 100644
index 2a228e8..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSHedgedReadMetrics.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import org.apache.hadoop.classification.InterfaceAudience;
-
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- * The client-side metrics for hedged read feature.
- * This class has a number of metrics variables that are publicly accessible,
- * we can grab them from client side, like HBase.
- */
-@InterfaceAudience.Private
-public class DFSHedgedReadMetrics {
- public final AtomicLong hedgedReadOps = new AtomicLong();
- public final AtomicLong hedgedReadOpsWin = new AtomicLong();
- public final AtomicLong hedgedReadOpsInCurThread = new AtomicLong();
-
- public void incHedgedReadOps() {
- hedgedReadOps.incrementAndGet();
- }
-
- public void incHedgedReadOpsInCurThread() {
- hedgedReadOpsInCurThread.incrementAndGet();
- }
-
- public void incHedgedReadWins() {
- hedgedReadOpsWin.incrementAndGet();
- }
-
- public long getHedgedReadOps() {
- return hedgedReadOps.longValue();
- }
-
- public long getHedgedReadOpsInCurThread() {
- return hedgedReadOpsInCurThread.longValue();
- }
-
- public long getHedgedReadWins() {
- return hedgedReadOpsWin.longValue();
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
deleted file mode 100644
index 1f9e3e9..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInotifyEventInputStream.java
+++ /dev/null
@@ -1,239 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs;
-
-import com.google.common.collect.Iterators;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.hdfs.inotify.EventBatch;
-import org.apache.hadoop.hdfs.inotify.EventBatchList;
-import org.apache.hadoop.hdfs.inotify.MissingEventsException;
-import org.apache.hadoop.hdfs.protocol.ClientProtocol;
-import org.apache.hadoop.util.Time;
-import org.apache.htrace.Sampler;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.Iterator;
-import java.util.Random;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Stream for reading inotify events. DFSInotifyEventInputStreams should not
- * be shared among multiple threads.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Unstable
-public class DFSInotifyEventInputStream {
- public static Logger LOG = LoggerFactory.getLogger(DFSInotifyEventInputStream
- .class);
-
- /**
- * The trace sampler to use when making RPCs to the NameNode.
- */
- private final Sampler<?> traceSampler;
-
- private final ClientProtocol namenode;
- private Iterator<EventBatch> it;
- private long lastReadTxid;
- /**
- * The most recent txid the NameNode told us it has sync'ed -- helps us
- * determine how far behind we are in the edit stream.
- */
- private long syncTxid;
- /**
- * Used to generate wait times in {@link DFSInotifyEventInputStream#take()}.
- */
- private Random rng = new Random();
-
- private static final int INITIAL_WAIT_MS = 10;
-
- DFSInotifyEventInputStream(Sampler<?> traceSampler, ClientProtocol namenode)
- throws IOException {
- // Only consider new transaction IDs.
- this(traceSampler, namenode, namenode.getCurrentEditLogTxid());
- }
-
- DFSInotifyEventInputStream(Sampler traceSampler, ClientProtocol namenode,
- long lastReadTxid) throws IOException {
- this.traceSampler = traceSampler;
- this.namenode = namenode;
- this.it = Iterators.emptyIterator();
- this.lastReadTxid = lastReadTxid;
- }
-
- /**
- * Returns the next batch of events in the stream or null if no new
- * batches are currently available.
- *
- * @throws IOException because of network error or edit log
- * corruption. Also possible if JournalNodes are unresponsive in the
- * QJM setting (even one unresponsive JournalNode is enough in rare cases),
- * so catching this exception and retrying at least a few times is
- * recommended.
- * @throws MissingEventsException if we cannot return the next batch in the
- * stream because the data for the events (and possibly some subsequent
- * events) has been deleted (generally because this stream is a very large
- * number of transactions behind the current state of the NameNode). It is
- * safe to continue reading from the stream after this exception is thrown
- * The next available batch of events will be returned.
- */
- public EventBatch poll() throws IOException, MissingEventsException {
- TraceScope scope =
- Trace.startSpan("inotifyPoll", traceSampler);
- try {
- // need to keep retrying until the NN sends us the latest committed txid
- if (lastReadTxid == -1) {
- LOG.debug("poll(): lastReadTxid is -1, reading current txid from NN");
- lastReadTxid = namenode.getCurrentEditLogTxid();
- return null;
- }
- if (!it.hasNext()) {
- EventBatchList el = namenode.getEditsFromTxid(lastReadTxid + 1);
- if (el.getLastTxid() != -1) {
- // we only want to set syncTxid when we were actually able to read some
- // edits on the NN -- otherwise it will seem like edits are being
- // generated faster than we can read them when the problem is really
- // that we are temporarily unable to read edits
- syncTxid = el.getSyncTxid();
- it = el.getBatches().iterator();
- long formerLastReadTxid = lastReadTxid;
- lastReadTxid = el.getLastTxid();
- if (el.getFirstTxid() != formerLastReadTxid + 1) {
- throw new MissingEventsException(formerLastReadTxid + 1,
- el.getFirstTxid());
- }
- } else {
- LOG.debug("poll(): read no edits from the NN when requesting edits " +
- "after txid {}", lastReadTxid);
- return null;
- }
- }
-
- if (it.hasNext()) { // can be empty if el.getLastTxid != -1 but none of the
- // newly seen edit log ops actually got converted to events
- return it.next();
- } else {
- return null;
- }
- } finally {
- scope.close();
- }
- }
-
- /**
- * Return a estimate of how many transaction IDs behind the NameNode's
- * current state this stream is. Clients should periodically call this method
- * and check if its result is steadily increasing, which indicates that they
- * are falling behind (i.e. transaction are being generated faster than the
- * client is reading them). If a client falls too far behind events may be
- * deleted before the client can read them.
- * <p/>
- * A return value of -1 indicates that an estimate could not be produced, and
- * should be ignored. The value returned by this method is really only useful
- * when compared to previous or subsequent returned values.
- */
- public long getTxidsBehindEstimate() {
- if (syncTxid == 0) {
- return -1;
- } else {
- assert syncTxid >= lastReadTxid;
- // this gives the difference between the last txid we have fetched to the
- // client and syncTxid at the time we last fetched events from the
- // NameNode
- return syncTxid - lastReadTxid;
- }
- }
-
- /**
- * Returns the next event batch in the stream, waiting up to the specified
- * amount of time for a new batch. Returns null if one is not available at the
- * end of the specified amount of time. The time before the method returns may
- * exceed the specified amount of time by up to the time required for an RPC
- * to the NameNode.
- *
- * @param time number of units of the given TimeUnit to wait
- * @param tu the desired TimeUnit
- * @throws IOException see {@link DFSInotifyEventInputStream#poll()}
- * @throws MissingEventsException
- * see {@link DFSInotifyEventInputStream#poll()}
- * @throws InterruptedException if the calling thread is interrupted
- */
- public EventBatch poll(long time, TimeUnit tu) throws IOException,
- InterruptedException, MissingEventsException {
- TraceScope scope = Trace.startSpan("inotifyPollWithTimeout", traceSampler);
- EventBatch next = null;
- try {
- long initialTime = Time.monotonicNow();
- long totalWait = TimeUnit.MILLISECONDS.convert(time, tu);
- long nextWait = INITIAL_WAIT_MS;
- while ((next = poll()) == null) {
- long timeLeft = totalWait - (Time.monotonicNow() - initialTime);
- if (timeLeft <= 0) {
- LOG.debug("timed poll(): timed out");
- break;
- } else if (timeLeft < nextWait * 2) {
- nextWait = timeLeft;
- } else {
- nextWait *= 2;
- }
- LOG.debug("timed poll(): poll() returned null, sleeping for {} ms",
- nextWait);
- Thread.sleep(nextWait);
- }
- } finally {
- scope.close();
- }
- return next;
- }
-
- /**
- * Returns the next batch of events in the stream, waiting indefinitely if
- * a new batch is not immediately available.
- *
- * @throws IOException see {@link DFSInotifyEventInputStream#poll()}
- * @throws MissingEventsException see
- * {@link DFSInotifyEventInputStream#poll()}
- * @throws InterruptedException if the calling thread is interrupted
- */
- public EventBatch take() throws IOException, InterruptedException,
- MissingEventsException {
- TraceScope scope = Trace.startSpan("inotifyTake", traceSampler);
- EventBatch next = null;
- try {
- int nextWaitMin = INITIAL_WAIT_MS;
- while ((next = poll()) == null) {
- // sleep for a random period between nextWaitMin and nextWaitMin * 2
- // to avoid stampedes at the NN if there are multiple clients
- int sleepTime = nextWaitMin + rng.nextInt(nextWaitMin);
- LOG.debug("take(): poll() returned null, sleeping for {} ms", sleepTime);
- Thread.sleep(sleepTime);
- // the maximum sleep is 2 minutes
- nextWaitMin = Math.min(60000, nextWaitMin * 2);
- }
- } finally {
- scope.close();
- }
-
- return next;
- }
-}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
deleted file mode 100644
index 139a27c..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ /dev/null
@@ -1,1915 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.hadoop.hdfs;
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.nio.ByteBuffer;
-import java.util.AbstractMap;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CancellationException;
-import java.util.concurrent.CompletionService;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorCompletionService;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.io.IOUtils;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.fs.ByteBufferReadable;
-import org.apache.hadoop.fs.ByteBufferUtil;
-import org.apache.hadoop.fs.CanSetDropBehind;
-import org.apache.hadoop.fs.CanSetReadahead;
-import org.apache.hadoop.fs.CanUnbuffer;
-import org.apache.hadoop.fs.ChecksumException;
-import org.apache.hadoop.fs.FSInputStream;
-import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.HasEnhancedByteBufferAccess;
-import org.apache.hadoop.fs.ReadOption;
-import org.apache.hadoop.fs.StorageType;
-import org.apache.hadoop.fs.UnresolvedLinkException;
-import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
-import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlock;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
-import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
-import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
-import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
-import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
-import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
-import org.apache.hadoop.io.ByteBufferPool;
-import org.apache.hadoop.ipc.RPC;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.token.SecretManager.InvalidToken;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.IdentityHashStore;
-import org.apache.htrace.Span;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceScope;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/****************************************************************
- * DFSInputStream provides bytes from a named file. It handles
- * negotiation of the namenode and various datanodes as necessary.
- ****************************************************************/
-@InterfaceAudience.Private
-public class DFSInputStream extends FSInputStream
-implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
- HasEnhancedByteBufferAccess, CanUnbuffer {
- @VisibleForTesting
- public static boolean tcpReadsDisabledForTesting = false;
- private long hedgedReadOpsLoopNumForTesting = 0;
- protected final DFSClient dfsClient;
- protected AtomicBoolean closed = new AtomicBoolean(false);
- protected final String src;
- protected final boolean verifyChecksum;
-
- // state by stateful read only:
- // (protected by lock on this)
- /////
- private DatanodeInfo currentNode = null;
- protected LocatedBlock currentLocatedBlock = null;
- protected long pos = 0;
- protected long blockEnd = -1;
- private BlockReader blockReader = null;
- ////
-
- // state shared by stateful and positional read:
- // (protected by lock on infoLock)
- ////
- protected LocatedBlocks locatedBlocks = null;
- private long lastBlockBeingWrittenLength = 0;
- private FileEncryptionInfo fileEncryptionInfo = null;
- protected CachingStrategy cachingStrategy;
- ////
-
- protected final ReadStatistics readStatistics = new ReadStatistics();
- // lock for state shared between read and pread
- // Note: Never acquire a lock on <this> with this lock held to avoid deadlocks
- // (it's OK to acquire this lock when the lock on <this> is held)
- protected final Object infoLock = new Object();
-
- /**
- * Track the ByteBuffers that we have handed out to readers.
- *
- * The value type can be either ByteBufferPool or ClientMmap, depending on
- * whether we this is a memory-mapped buffer or not.
- */
- private IdentityHashStore<ByteBuffer, Object> extendedReadBuffers;
-
- private synchronized IdentityHashStore<ByteBuffer, Object>
- getExtendedReadBuffers() {
- if (extendedReadBuffers == null) {
- extendedReadBuffers = new IdentityHashStore<ByteBuffer, Object>(0);
- }
- return extendedReadBuffers;
- }
-
- public static class ReadStatistics {
- public ReadStatistics() {
- clear();
- }
-
- public ReadStatistics(ReadStatistics rhs) {
- this.totalBytesRead = rhs.getTotalBytesRead();
- this.totalLocalBytesRead = rhs.getTotalLocalBytesRead();
- this.totalShortCircuitBytesRead = rhs.getTotalShortCircuitBytesRead();
- this.totalZeroCopyBytesRead = rhs.getTotalZeroCopyBytesRead();
- }
-
- /**
- * @return The total bytes read. This will always be at least as
- * high as the other numbers, since it includes all of them.
- */
- public long getTotalBytesRead() {
- return totalBytesRead;
- }
-
- /**
- * @return The total local bytes read. This will always be at least
- * as high as totalShortCircuitBytesRead, since all short-circuit
- * reads are also local.
- */
- public long getTotalLocalBytesRead() {
- return totalLocalBytesRead;
- }
-
- /**
- * @return The total short-circuit local bytes read.
- */
- public long getTotalShortCircuitBytesRead() {
- return totalShortCircuitBytesRead;
- }
-
- /**
- * @return The total number of zero-copy bytes read.
- */
- public long getTotalZeroCopyBytesRead() {
- return totalZeroCopyBytesRead;
- }
-
- /**
- * @return The total number of bytes read which were not local.
- */
- public long getRemoteBytesRead() {
- return totalBytesRead - totalLocalBytesRead;
- }
-
- void addRemoteBytes(long amt) {
- this.totalBytesRead += amt;
- }
-
- void addLocalBytes(long amt) {
- this.totalBytesRead += amt;
- this.totalLocalBytesRead += amt;
- }
-
- void addShortCircuitBytes(long amt) {
- this.totalBytesRead += amt;
- this.totalLocalBytesRead += amt;
- this.totalShortCircuitBytesRead += amt;
- }
-
- void addZeroCopyBytes(long amt) {
- this.totalBytesRead += amt;
- this.totalLocalBytesRead += amt;
- this.totalShortCircuitBytesRead += amt;
- this.totalZeroCopyBytesRead += amt;
- }
-
- void clear() {
- this.totalBytesRead = 0;
- this.totalLocalBytesRead = 0;
- this.totalShortCircuitBytesRead = 0;
- this.totalZeroCopyBytesRead = 0;
- }
-
- private long totalBytesRead;
-
- private long totalLocalBytesRead;
-
- private long totalShortCircuitBytesRead;
-
- private long totalZeroCopyBytesRead;
- }
-
- /**
- * This variable tracks the number of failures since the start of the
- * most recent user-facing operation. That is to say, it should be reset
- * whenever the user makes a call on this stream, and if at any point
- * during the retry logic, the failure count exceeds a threshold,
- * the errors will be thrown back to the operation.
- *
- * Specifically this counts the number of times the client has gone
- * back to the namenode to get a new list of block locations, and is
- * capped at maxBlockAcquireFailures
- */
- protected int failures = 0;
-
- /* XXX Use of CocurrentHashMap is temp fix. Need to fix
- * parallel accesses to DFSInputStream (through ptreads) properly */
- private final ConcurrentHashMap<DatanodeInfo, DatanodeInfo> deadNodes =
- new ConcurrentHashMap<DatanodeInfo, DatanodeInfo>();
-
- private byte[] oneByteBuf; // used for 'int read()'
-
- void addToDeadNodes(DatanodeInfo dnInfo) {
- deadNodes.put(dnInfo, dnInfo);
- }
-
- DFSInputStream(DFSClient dfsClient, String src, boolean verifyChecksum,
- LocatedBlocks locatedBlocks) throws IOException, UnresolvedLinkException {
- this.dfsClient = dfsClient;
- this.verifyChecksum = verifyChecksum;
- this.src = src;
- synchronized (infoLock) {
- this.cachingStrategy = dfsClient.getDefaultReadCachingStrategy();
- }
- this.locatedBlocks = locatedBlocks;
- openInfo(false);
- }
-
- /**
- * Grab the open-file info from namenode
- * @param refreshLocatedBlocks whether to re-fetch locatedblocks
- */
- void openInfo(boolean refreshLocatedBlocks) throws IOException,
- UnresolvedLinkException {
- final DfsClientConf conf = dfsClient.getConf();
- synchronized(infoLock) {
- lastBlockBeingWrittenLength =
- fetchLocatedBlocksAndGetLastBlockLength(refreshLocatedBlocks);
- int retriesForLastBlockLength = conf.getRetryTimesForGetLastBlockLength();
- while (retriesForLastBlockLength > 0) {
- // Getting last block length as -1 is a special case. When cluster
- // restarts, DNs may not report immediately. At this time partial block
- // locations will not be available with NN for getting the length. Lets
- // retry for 3 times to get the length.
- if (lastBlockBeingWrittenLength == -1) {
- DFSClient.LOG.warn("Last block locations not available. "
- + "Datanodes might not have reported blocks completely."
- + " Will retry for " + retriesForLastBlockLength + " times");
- waitFor(conf.getRetryIntervalForGetLastBlockLength());
- lastBlockBeingWrittenLength =
- fetchLocatedBlocksAndGetLastBlockLength(true);
- } else {
- break;
- }
- retriesForLastBlockLength--;
- }
- if (retriesForLastBlockLength == 0) {
- throw new IOException("Could not obtain the last block locations.");
- }
- }
- }
-
- private void waitFor(int waitTime) throws IOException {
- try {
- Thread.sleep(waitTime);
- } catch (InterruptedException e) {
- throw new IOException(
- "Interrupted while getting the last block length.");
- }
- }
-
- private long fetchLocatedBlocksAndGetLastBlockLength(boolean refresh)
- throws IOException {
- LocatedBlocks newInfo = locatedBlocks;
- if (locatedBlocks == null || refresh) {
- newInfo = dfsClient.getLocatedBlocks(src, 0);
- }
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("newInfo = " + newInfo);
- }
- if (newInfo == null) {
- throw new IOException("Cannot open filename " + src);
- }
-
- if (locatedBlocks != null) {
- Iterator<LocatedBlock> oldIter = locatedBlocks.getLocatedBlocks().iterator();
- Iterator<LocatedBlock> newIter = newInfo.getLocatedBlocks().iterator();
- while (oldIter.hasNext() && newIter.hasNext()) {
- if (! oldIter.next().getBlock().equals(newIter.next().getBlock())) {
- throw new IOException("Blocklist for " + src + " has changed!");
- }
- }
- }
- locatedBlocks = newInfo;
- long lastBlockBeingWrittenLength = 0;
- if (!locatedBlocks.isLastBlockComplete()) {
- final LocatedBlock last = locatedBlocks.getLastLocatedBlock();
- if (last != null) {
- if (last.getLocations().length == 0) {
- if (last.getBlockSize() == 0) {
- // if the length is zero, then no data has been written to
- // datanode. So no need to wait for the locations.
- return 0;
- }
- return -1;
- }
- final long len = readBlockLength(last);
- last.getBlock().setNumBytes(len);
- lastBlockBeingWrittenLength = len;
- }
- }
-
- fileEncryptionInfo = locatedBlocks.getFileEncryptionInfo();
-
- return lastBlockBeingWrittenLength;
- }
-
- /** Read the block length from one of the datanodes. */
- private long readBlockLength(LocatedBlock locatedblock) throws IOException {
- assert locatedblock != null : "LocatedBlock cannot be null";
- int replicaNotFoundCount = locatedblock.getLocations().length;
-
- final DfsClientConf conf = dfsClient.getConf();
- for(DatanodeInfo datanode : locatedblock.getLocations()) {
- ClientDatanodeProtocol cdp = null;
-
- try {
- cdp = DFSUtilClient.createClientDatanodeProtocolProxy(datanode,
- dfsClient.getConfiguration(), conf.getSocketTimeout(),
- conf.isConnectToDnViaHostname(), locatedblock);
-
- final long n = cdp.getReplicaVisibleLength(locatedblock.getBlock());
-
- if (n >= 0) {
- return n;
- }
- }
- catch(IOException ioe) {
- if (ioe instanceof RemoteException &&
- (((RemoteException) ioe).unwrapRemoteException() instanceof
- ReplicaNotFoundException)) {
- // special case : replica might not be on the DN, treat as 0 length
- replicaNotFoundCount--;
- }
-
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("Failed to getReplicaVisibleLength from datanode "
- + datanode + " for block " + locatedblock.getBlock(), ioe);
- }
- } finally {
- if (cdp != null) {
- RPC.stopProxy(cdp);
- }
- }
- }
-
- // Namenode told us about these locations, but none know about the replica
- // means that we hit the race between pipeline creation start and end.
- // we require all 3 because some other exception could have happened
- // on a DN that has it. we want to report that error
- if (replicaNotFoundCount == 0) {
- return 0;
- }
-
- throw new IOException("Cannot obtain block length for " + locatedblock);
- }
-
- public long getFileLength() {
- synchronized(infoLock) {
- return locatedBlocks == null? 0:
- locatedBlocks.getFileLength() + lastBlockBeingWrittenLength;
- }
- }
-
- // Short circuit local reads are forbidden for files that are
- // under construction. See HDFS-2757.
- boolean shortCircuitForbidden() {
- synchronized(infoLock) {
- return locatedBlocks.isUnderConstruction();
- }
- }
-
- /**
- * Returns the datanode from which the stream is currently reading.
- */
- public synchronized DatanodeInfo getCurrentDatanode() {
- return currentNode;
- }
-
- /**
- * Returns the block containing the target position.
- */
- synchronized public ExtendedBlock getCurrentBlock() {
- if (currentLocatedBlock == null){
- return null;
- }
- return currentLocatedBlock.getBlock();
- }
-
- /**
- * Return collection of blocks that has already been located.
- */
- public List<LocatedBlock> getAllBlocks() throws IOException {
- return getBlockRange(0, getFileLength());
- }
-
- /**
- * Get block at the specified position.
- * Fetch it from the namenode if not cached.
- *
- * @param offset block corresponding to this offset in file is returned
- * @return located block
- * @throws IOException
- */
- protected LocatedBlock getBlockAt(long offset) throws IOException {
- synchronized(infoLock) {
- assert (locatedBlocks != null) : "locatedBlocks is null";
-
- final LocatedBlock blk;
-
- //check offset
- if (offset < 0 || offset >= getFileLength()) {
- throw new IOException("offset < 0 || offset >= getFileLength(), offset="
- + offset
- + ", locatedBlocks=" + locatedBlocks);
- }
- else if (offset >= locatedBlocks.getFileLength()) {
- // offset to the portion of the last block,
- // which is not known to the name-node yet;
- // getting the last block
- blk = locatedBlocks.getLastLocatedBlock();
- }
- else {
- // search cached blocks first
- int targetBlockIdx = locatedBlocks.findBlock(offset);
- if (targetBlockIdx < 0) { // block is not cached
- targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
- // fetch more blocks
- final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
- assert (newBlocks != null) : "Could not find target position " + offset;
- locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
- }
- blk = locatedBlocks.get(targetBlockIdx);
- }
- return blk;
- }
- }
-
- /** Fetch a block from namenode and cache it */
- protected void fetchBlockAt(long offset) throws IOException {
- synchronized(infoLock) {
- int targetBlockIdx = locatedBlocks.findBlock(offset);
- if (targetBlockIdx < 0) { // block is not cached
- targetBlockIdx = LocatedBlocks.getInsertIndex(targetBlockIdx);
- }
- // fetch blocks
- final LocatedBlocks newBlocks = dfsClient.getLocatedBlocks(src, offset);
- if (newBlocks == null) {
- throw new IOException("Could not find target position " + offset);
- }
- locatedBlocks.insertRange(targetBlockIdx, newBlocks.getLocatedBlocks());
- }
- }
-
- /**
- * Get blocks in the specified range.
- * Fetch them from the namenode if not cached. This function
- * will not get a read request beyond the EOF.
- * @param offset starting offset in file
- * @param length length of data
- * @return consequent segment of located blocks
- * @throws IOException
- */
- private List<LocatedBlock> getBlockRange(long offset,
- long length) throws IOException {
- // getFileLength(): returns total file length
- // locatedBlocks.getFileLength(): returns length of completed blocks
- if (offset >= getFileLength()) {
- throw new IOException("Offset: " + offset +
- " exceeds file length: " + getFileLength());
- }
- synchronized(infoLock) {
- final List<LocatedBlock> blocks;
- final long lengthOfCompleteBlk = locatedBlocks.getFileLength();
- final boolean readOffsetWithinCompleteBlk = offset < lengthOfCompleteBlk;
- final boolean readLengthPastCompleteBlk = offset + length > lengthOfCompleteBlk;
-
- if (readOffsetWithinCompleteBlk) {
- //get the blocks of finalized (completed) block range
- blocks = getFinalizedBlockRange(offset,
- Math.min(length, lengthOfCompleteBlk - offset));
- } else {
- blocks = new ArrayList<LocatedBlock>(1);
- }
-
- // get the blocks from incomplete block range
- if (readLengthPastCompleteBlk) {
- blocks.add(locatedBlocks.getLastLocatedBlock());
- }
-
- return blocks;
- }
- }
-
- /**
- * Get blocks in the specified range.
- * Includes only the complete blocks.
- * Fetch them from the namenode if not cached.
- */
- private List<LocatedBlock> getFinalizedBlockRange(
- long offset, long length) throws IOException {
- synchronized(infoLock) {
- assert (locatedBlocks != null) : "locatedBlocks is null";
- List<LocatedBlock> blockRange = new ArrayList<LocatedBlock>();
- // search cached blocks first
- int blockIdx = locatedBlocks.findBlock(offset);
- if (blockIdx < 0) { // block is not cached
- blockIdx = LocatedBlocks.getInsertIndex(blockIdx);
- }
- long remaining = length;
- long curOff = offset;
- while(remaining > 0) {
- LocatedBlock blk = null;
- if(blockIdx < locatedBlocks.locatedBlockCount())
- blk = locatedBlocks.get(blockIdx);
- if (blk == null || curOff < blk.getStartOffset()) {
- LocatedBlocks newBlocks;
- newBlocks = dfsClient.getLocatedBlocks(src, curOff, remaining);
- locatedBlocks.insertRange(blockIdx, newBlocks.getLocatedBlocks());
- continue;
- }
- assert curOff >= blk.getStartOffset() : "Block not found";
- blockRange.add(blk);
- long bytesRead = blk.getStartOffset() + blk.getBlockSize() - curOff;
- remaining -= bytesRead;
- curOff += bytesRead;
- blockIdx++;
- }
- return blockRange;
- }
- }
-
- /**
- * Open a DataInputStream to a DataNode so that it can be read from.
- * We get block ID and the IDs of the destinations at startup, from the namenode.
- */
- private synchronized DatanodeInfo blockSeekTo(long target) throws IOException {
- if (target >= getFileLength()) {
- throw new IOException("Attempted to read past end of file");
- }
-
- // Will be getting a new BlockReader.
- closeCurrentBlockReaders();
-
- //
- // Connect to best DataNode for desired Block, with potential offset
- //
- DatanodeInfo chosenNode = null;
- int refetchToken = 1; // only need to get a new access token once
- int refetchEncryptionKey = 1; // only need to get a new encryption key once
-
- boolean connectFailedOnce = false;
-
- while (true) {
- //
- // Compute desired block
- //
- LocatedBlock targetBlock = getBlockAt(target);
-
- // update current position
- this.pos = target;
- this.blockEnd = targetBlock.getStartOffset() +
- targetBlock.getBlockSize() - 1;
- this.currentLocatedBlock = targetBlock;
-
- long offsetIntoBlock = target - targetBlock.getStartOffset();
-
- DNAddrPair retval = chooseDataNode(targetBlock, null);
- chosenNode = retval.info;
- InetSocketAddress targetAddr = retval.addr;
- StorageType storageType = retval.storageType;
-
- try {
- blockReader = getBlockReader(targetBlock, offsetIntoBlock,
- targetBlock.getBlockSize() - offsetIntoBlock, targetAddr,
- storageType, chosenNode);
- if(connectFailedOnce) {
- DFSClient.LOG.info("Successfully connected to " + targetAddr +
- " for " + targetBlock.getBlock());
- }
- return chosenNode;
- } catch (IOException ex) {
- if (ex instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
- DFSClient.LOG.info("Will fetch a new encryption key and retry, "
- + "encryption key was invalid when connecting to " + targetAddr
- + " : " + ex);
- // The encryption key used is invalid.
- refetchEncryptionKey--;
- dfsClient.clearDataEncryptionKey();
- } else if (refetchToken > 0 && tokenRefetchNeeded(ex, targetAddr)) {
- refetchToken--;
- fetchBlockAt(target);
- } else {
- connectFailedOnce = true;
- DFSClient.LOG.warn("Failed to connect to " + targetAddr + " for block"
- + ", add to deadNodes and continue. " + ex, ex);
- // Put chosen node into dead list, continue
- addToDeadNodes(chosenNode);
- }
- }
- }
- }
-
- protected BlockReader getBlockReader(LocatedBlock targetBlock,
- long offsetInBlock, long length, InetSocketAddress targetAddr,
- StorageType storageType, DatanodeInfo datanode) throws IOException {
- ExtendedBlock blk = targetBlock.getBlock();
- Token<BlockTokenIdentifier> accessToken = targetBlock.getBlockToken();
- CachingStrategy curCachingStrategy;
- boolean shortCircuitForbidden;
- synchronized (infoLock) {
- curCachingStrategy = cachingStrategy;
- shortCircuitForbidden = shortCircuitForbidden();
- }
- return new BlockReaderFactory(dfsClient.getConf()).
- setInetSocketAddress(targetAddr).
- setRemotePeerFactory(dfsClient).
- setDatanodeInfo(datanode).
- setStorageType(storageType).
- setFileName(src).
- setBlock(blk).
- setBlockToken(accessToken).
- setStartOffset(offsetInBlock).
- setVerifyChecksum(verifyChecksum).
- setClientName(dfsClient.clientName).
- setLength(length).
- setCachingStrategy(curCachingStrategy).
- setAllowShortCircuitLocalReads(!shortCircuitForbidden).
- setClientCacheContext(dfsClient.getClientContext()).
- setUserGroupInformation(dfsClient.ugi).
- setConfiguration(dfsClient.getConfiguration()).
- build();
- }
-
- /**
- * Close it down!
- */
- @Override
- public synchronized void close() throws IOException {
- if (!closed.compareAndSet(false, true)) {
- DFSClient.LOG.debug("DFSInputStream has been closed already");
- return;
- }
- dfsClient.checkOpen();
-
- if ((extendedReadBuffers != null) && (!extendedReadBuffers.isEmpty())) {
- final StringBuilder builder = new StringBuilder();
- extendedReadBuffers.visitAll(new IdentityHashStore.Visitor<ByteBuffer, Object>() {
- private String prefix = "";
- @Override
- public void accept(ByteBuffer k, Object v) {
- builder.append(prefix).append(k);
- prefix = ", ";
- }
- });
- DFSClient.LOG.warn("closing file " + src + ", but there are still " +
- "unreleased ByteBuffers allocated by read(). " +
- "Please release " + builder.toString() + ".");
- }
- closeCurrentBlockReaders();
- super.close();
- }
-
- @Override
- public synchronized int read() throws IOException {
- if (oneByteBuf == null) {
- oneByteBuf = new byte[1];
- }
- int ret = read( oneByteBuf, 0, 1 );
- return ( ret <= 0 ) ? -1 : (oneByteBuf[0] & 0xff);
- }
-
- /**
- * Wraps different possible read implementations so that readBuffer can be
- * strategy-agnostic.
- */
- interface ReaderStrategy {
- public int doRead(BlockReader blockReader, int off, int len)
- throws ChecksumException, IOException;
-
- /**
- * Copy data from the src ByteBuffer into the read buffer.
- * @param src The src buffer where the data is copied from
- * @param offset Useful only when the ReadStrategy is based on a byte array.
- * Indicate the offset of the byte array for copy.
- * @param length Useful only when the ReadStrategy is based on a byte array.
- * Indicate the length of the data to copy.
- */
- public int copyFrom(ByteBuffer src, int offset, int length);
- }
-
- protected void updateReadStatistics(ReadStatistics readStatistics,
- int nRead, BlockReader blockReader) {
- if (nRead <= 0) return;
- synchronized(infoLock) {
- if (blockReader.isShortCircuit()) {
- readStatistics.addShortCircuitBytes(nRead);
- } else if (blockReader.isLocal()) {
- readStatistics.addLocalBytes(nRead);
- } else {
- readStatistics.addRemoteBytes(nRead);
- }
- }
- }
-
- /**
- * Used to read bytes into a byte[]
- */
- private class ByteArrayStrategy implements ReaderStrategy {
- final byte[] buf;
-
- public ByteArrayStrategy(byte[] buf) {
- this.buf = buf;
- }
-
- @Override
- public int doRead(BlockReader blockReader, int off, int len)
- throws ChecksumException, IOException {
- int nRead = blockReader.read(buf, off, len);
- updateReadStatistics(readStatistics, nRead, blockReader);
- return nRead;
- }
-
- @Override
- public int copyFrom(ByteBuffer src, int offset, int length) {
- ByteBuffer writeSlice = src.duplicate();
- writeSlice.get(buf, offset, length);
- return length;
- }
- }
-
- /**
- * Used to read bytes into a user-supplied ByteBuffer
- */
- protected class ByteBufferStrategy implements ReaderStrategy {
- final ByteBuffer buf;
- ByteBufferStrategy(ByteBuffer buf) {
- this.buf = buf;
- }
-
- @Override
- public int doRead(BlockReader blockReader, int off, int len)
- throws ChecksumException, IOException {
- int oldpos = buf.position();
- int oldlimit = buf.limit();
- boolean success = false;
- try {
- int ret = blockReader.read(buf);
- success = true;
- updateReadStatistics(readStatistics, ret, blockReader);
- if (ret == 0) {
- DFSClient.LOG.warn("zero");
- }
- return ret;
- } finally {
- if (!success) {
- // Reset to original state so that retries work correctly.
- buf.position(oldpos);
- buf.limit(oldlimit);
- }
- }
- }
-
- @Override
- public int copyFrom(ByteBuffer src, int offset, int length) {
- ByteBuffer writeSlice = src.duplicate();
- int remaining = Math.min(buf.remaining(), writeSlice.remaining());
- writeSlice.limit(writeSlice.position() + remaining);
- buf.put(writeSlice);
- return remaining;
- }
- }
-
- /* This is a used by regular read() and handles ChecksumExceptions.
- * name readBuffer() is chosen to imply similarity to readBuffer() in
- * ChecksumFileSystem
- */
- private synchronized int readBuffer(ReaderStrategy reader, int off, int len,
- Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
- throws IOException {
- IOException ioe;
-
- /* we retry current node only once. So this is set to true only here.
- * Intention is to handle one common case of an error that is not a
- * failure on datanode or client : when DataNode closes the connection
- * since client is idle. If there are other cases of "non-errors" then
- * then a datanode might be retried by setting this to true again.
- */
- boolean retryCurrentNode = true;
-
- while (true) {
- // retry as many times as seekToNewSource allows.
- try {
- return reader.doRead(blockReader, off, len);
- } catch ( ChecksumException ce ) {
- DFSClient.LOG.warn("Found Checksum error for "
- + getCurrentBlock() + " from " + currentNode
- + " at " + ce.getPos());
- ioe = ce;
- retryCurrentNode = false;
- // we want to remember which block replicas we have tried
- addIntoCorruptedBlockMap(getCurrentBlock(), currentNode,
- corruptedBlockMap);
- } catch ( IOException e ) {
- if (!retryCurrentNode) {
- DFSClient.LOG.warn("Exception while reading from "
- + getCurrentBlock() + " of " + src + " from "
- + currentNode, e);
- }
- ioe = e;
- }
- boolean sourceFound = false;
- if (retryCurrentNode) {
- /* possibly retry the same node so that transient errors don't
- * result in application level failures (e.g. Datanode could have
- * closed the connection because the client is idle for too long).
- */
- sourceFound = seekToBlockSource(pos);
- } else {
- addToDeadNodes(currentNode);
- sourceFound = seekToNewSource(pos);
- }
- if (!sourceFound) {
- throw ioe;
- }
- retryCurrentNode = false;
- }
- }
-
- protected synchronized int readWithStrategy(ReaderStrategy strategy, int off, int len) throws IOException {
- dfsClient.checkOpen();
- if (closed.get()) {
- throw new IOException("Stream closed");
- }
- Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
- = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
- failures = 0;
- if (pos < getFileLength()) {
- int retries = 2;
- while (retries > 0) {
- try {
- // currentNode can be left as null if previous read had a checksum
- // error on the same block. See HDFS-3067
- if (pos > blockEnd || currentNode == null) {
- currentNode = blockSeekTo(pos);
- }
- int realLen = (int) Math.min(len, (blockEnd - pos + 1L));
- synchronized(infoLock) {
- if (locatedBlocks.isLastBlockComplete()) {
- realLen = (int) Math.min(realLen,
- locatedBlocks.getFileLength() - pos);
- }
- }
- int result = readBuffer(strategy, off, realLen, corruptedBlockMap);
-
- if (result >= 0) {
- pos += result;
- } else {
- // got a EOS from reader though we expect more data on it.
- throw new IOException("Unexpected EOS from the reader");
- }
- if (dfsClient.stats != null) {
- dfsClient.stats.incrementBytesRead(result);
- }
- return result;
- } catch (ChecksumException ce) {
- throw ce;
- } catch (IOException e) {
- if (retries == 1) {
- DFSClient.LOG.warn("DFS Read", e);
- }
- blockEnd = -1;
- if (currentNode != null) { addToDeadNodes(currentNode); }
- if (--retries == 0) {
- throw e;
- }
- } finally {
- // Check if need to report block replicas corruption either read
- // was successful or ChecksumException occured.
- reportCheckSumFailure(corruptedBlockMap,
- currentLocatedBlock.getLocations().length);
- }
- }
- }
- return -1;
- }
-
- /**
- * Read the entire buffer.
- */
- @Override
- public synchronized int read(final byte buf[], int off, int len) throws IOException {
- ReaderStrategy byteArrayReader = new ByteArrayStrategy(buf);
- TraceScope scope =
- dfsClient.getPathTraceScope("DFSInputStream#byteArrayRead", src);
- try {
- return readWithStrategy(byteArrayReader, off, len);
- } finally {
- scope.close();
- }
- }
-
- @Override
- public synchronized int read(final ByteBuffer buf) throws IOException {
- ReaderStrategy byteBufferReader = new ByteBufferStrategy(buf);
- TraceScope scope =
- dfsClient.getPathTraceScope("DFSInputStream#byteBufferRead", src);
- try {
- return readWithStrategy(byteBufferReader, 0, buf.remaining());
- } finally {
- scope.close();
- }
- }
-
-
- /**
- * Add corrupted block replica into map.
- */
- protected void addIntoCorruptedBlockMap(ExtendedBlock blk, DatanodeInfo node,
- Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap) {
- Set<DatanodeInfo> dnSet = null;
- if((corruptedBlockMap.containsKey(blk))) {
- dnSet = corruptedBlockMap.get(blk);
- }else {
- dnSet = new HashSet<DatanodeInfo>();
- }
- if (!dnSet.contains(node)) {
- dnSet.add(node);
- corruptedBlockMap.put(blk, dnSet);
- }
- }
-
- private DNAddrPair chooseDataNode(LocatedBlock block,
- Collection<DatanodeInfo> ignoredNodes) throws IOException {
- while (true) {
- DNAddrPair result = getBestNodeDNAddrPair(block, ignoredNodes);
- if (result != null) {
- return result;
- } else {
- String errMsg = getBestNodeDNAddrPairErrorString(block.getLocations(),
- deadNodes, ignoredNodes);
- String blockInfo = block.getBlock() + " file=" + src;
- if (failures >= dfsClient.getConf().getMaxBlockAcquireFailures()) {
- String description = "Could not obtain block: " + blockInfo;
- DFSClient.LOG.warn(description + errMsg
- + ". Throwing a BlockMissingException");
- throw new BlockMissingException(src, description,
- block.getStartOffset());
- }
-
- DatanodeInfo[] nodes = block.getLocations();
- if (nodes == null || nodes.length == 0) {
- DFSClient.LOG.info("No node available for " + blockInfo);
- }
- DFSClient.LOG.info("Could not obtain " + block.getBlock()
- + " from any node: " + errMsg
- + ". Will get new block locations from namenode and retry...");
- try {
- // Introducing a random factor to the wait time before another retry.
- // The wait time is dependent on # of failures and a random factor.
- // At the first time of getting a BlockMissingException, the wait time
- // is a random number between 0..3000 ms. If the first retry
- // still fails, we will wait 3000 ms grace period before the 2nd retry.
- // Also at the second retry, the waiting window is expanded to 6000 ms
- // alleviating the request rate from the server. Similarly the 3rd retry
- // will wait 6000ms grace period before retry and the waiting window is
- // expanded to 9000ms.
- final int timeWindow = dfsClient.getConf().getTimeWindow();
- double waitTime = timeWindow * failures + // grace period for the last round of attempt
- // expanding time window for each failure
- timeWindow * (failures + 1) *
- ThreadLocalRandom.current().nextDouble();
- DFSClient.LOG.warn("DFS chooseDataNode: got # " + (failures + 1) + " IOException, will wait for " + waitTime + " msec.");
- Thread.sleep((long)waitTime);
- } catch (InterruptedException iex) {
- }
- deadNodes.clear(); //2nd option is to remove only nodes[blockId]
- openInfo(true);
- block = refreshLocatedBlock(block);
- failures++;
- }
- }
- }
-
- /**
- * Get the best node from which to stream the data.
- * @param block LocatedBlock, containing nodes in priority order.
- * @param ignoredNodes Do not choose nodes in this array (may be null)
- * @return The DNAddrPair of the best node. Null if no node can be chosen.
- */
- protected DNAddrPair getBestNodeDNAddrPair(LocatedBlock block,
- Collection<DatanodeInfo> ignoredNodes) {
- DatanodeInfo[] nodes = block.getLocations();
- StorageType[] storageTypes = block.getStorageTypes();
- DatanodeInfo chosenNode = null;
- StorageType storageType = null;
- if (nodes != null) {
- for (int i = 0; i < nodes.length; i++) {
- if (!deadNodes.containsKey(nodes[i])
- && (ignoredNodes == null || !ignoredNodes.contains(nodes[i]))) {
- chosenNode = nodes[i];
- // Storage types are ordered to correspond with nodes, so use the same
- // index to get storage type.
- if (storageTypes != null && i < storageTypes.length) {
- storageType = storageTypes[i];
- }
- break;
- }
- }
- }
- if (chosenNode == null) {
- DFSClient.LOG.warn("No live nodes contain block " + block.getBlock() +
- " after checking nodes = " + Arrays.toString(nodes) +
- ", ignoredNodes = " + ignoredNodes);
- return null;
- }
- final String dnAddr =
- chosenNode.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname());
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
- }
- InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
- return new DNAddrPair(chosenNode, targetAddr, storageType);
- }
-
- private static String getBestNodeDNAddrPairErrorString(
- DatanodeInfo nodes[], AbstractMap<DatanodeInfo,
- DatanodeInfo> deadNodes, Collection<DatanodeInfo> ignoredNodes) {
- StringBuilder errMsgr = new StringBuilder(
- " No live nodes contain current block ");
- errMsgr.append("Block locations:");
- for (DatanodeInfo datanode : nodes) {
- errMsgr.append(" ");
- errMsgr.append(datanode.toString());
- }
- errMsgr.append(" Dead nodes: ");
- for (DatanodeInfo datanode : deadNodes.keySet()) {
- errMsgr.append(" ");
- errMsgr.append(datanode.toString());
- }
- if (ignoredNodes != null) {
- errMsgr.append(" Ignored nodes: ");
- for (DatanodeInfo datanode : ignoredNodes) {
- errMsgr.append(" ");
- errMsgr.append(datanode.toString());
- }
- }
- return errMsgr.toString();
- }
-
- protected void fetchBlockByteRange(LocatedBlock block, long start, long end,
- byte[] buf, int offset,
- Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
- throws IOException {
- block = refreshLocatedBlock(block);
- while (true) {
- DNAddrPair addressPair = chooseDataNode(block, null);
- try {
- actualGetFromOneDataNode(addressPair, block, start, end,
- buf, offset, corruptedBlockMap);
- return;
- } catch (IOException e) {
- // Ignore. Already processed inside the function.
- // Loop through to try the next node.
- }
- }
- }
-
- private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
- final LocatedBlock block, final long start, final long end,
- final ByteBuffer bb,
- final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
- final int hedgedReadId) {
- final Span parentSpan = Trace.currentSpan();
- return new Callable<ByteBuffer>() {
- @Override
- public ByteBuffer call() throws Exception {
- byte[] buf = bb.array();
- int offset = bb.position();
- TraceScope scope =
- Trace.startSpan("hedgedRead" + hedgedReadId, parentSpan);
- try {
- actualGetFromOneDataNode(datanode, block, start, end, buf,
- offset, corruptedBlockMap);
- return bb;
- } finally {
- scope.close();
- }
- }
- };
- }
-
- /**
- * Used when reading contiguous blocks
- */
- private void actualGetFromOneDataNode(final DNAddrPair datanode,
- LocatedBlock block, final long start, final long end, byte[] buf,
- int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
- throws IOException {
- final int length = (int) (end - start + 1);
- actualGetFromOneDataNode(datanode, block, start, end, buf,
- new int[]{offset}, new int[]{length}, corruptedBlockMap);
- }
-
- /**
- * Read data from one DataNode.
- * @param datanode the datanode from which to read data
- * @param block the located block containing the requested data
- * @param startInBlk the startInBlk offset of the block
- * @param endInBlk the endInBlk offset of the block
- * @param buf the given byte array into which the data is read
- * @param offsets the data may be read into multiple segments of the buf
- * (when reading a striped block). this array indicates the
- * offset of each buf segment.
- * @param lengths the length of each buf segment
- * @param corruptedBlockMap map recording list of datanodes with corrupted
- * block replica
- */
- void actualGetFromOneDataNode(final DNAddrPair datanode,
- LocatedBlock block, final long startInBlk, final long endInBlk,
- byte[] buf, int[] offsets, int[] lengths,
- Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
- throws IOException {
- DFSClientFaultInjector.get().startFetchFromDatanode();
- int refetchToken = 1; // only need to get a new access token once
- int refetchEncryptionKey = 1; // only need to get a new encryption key once
- final int len = (int) (endInBlk - startInBlk + 1);
- checkReadPortions(offsets, lengths, len);
-
- while (true) {
- // cached block locations may have been updated by chooseDataNode()
- // or fetchBlockAt(). Always get the latest list of locations at the
- // start of the loop.
- block = refreshLocatedBlock(block);
- BlockReader reader = null;
- try {
- DFSClientFaultInjector.get().fetchFromDatanodeException();
- reader = getBlockReader(block, startInBlk, len, datanode.addr,
- datanode.storageType, datanode.info);
- for (int i = 0; i < offsets.length; i++) {
- int nread = reader.readAll(buf, offsets[i], lengths[i]);
- updateReadStatistics(readStatistics, nread, reader);
- if (nread != lengths[i]) {
- throw new IOException("truncated return from reader.read(): " +
- "excpected " + lengths[i] + ", got " + nread);
- }
- }
- DFSClientFaultInjector.get().readFromDatanodeDelay();
- return;
- } catch (ChecksumException e) {
- String msg = "fetchBlockByteRange(). Got a checksum exception for "
- + src + " at " + block.getBlock() + ":" + e.getPos() + " from "
- + datanode.info;
- DFSClient.LOG.warn(msg);
- // we want to remember what we have tried
- addIntoCorruptedBlockMap(block.getBlock(), datanode.info,
- corruptedBlockMap);
- addToDeadNodes(datanode.info);
- throw new IOException(msg);
- } catch (IOException e) {
- if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
- DFSClient.LOG.info("Will fetch a new encryption key and retry, "
- + "encryption key was invalid when connecting to " + datanode.addr
- + " : " + e);
- // The encryption key used is invalid.
- refetchEncryptionKey--;
- dfsClient.clearDataEncryptionKey();
- } else if (refetchToken > 0 && tokenRefetchNeeded(e, datanode.addr)) {
- refetchToken--;
- try {
- fetchBlockAt(block.getStartOffset());
- } catch (IOException fbae) {
- // ignore IOE, since we can retry it later in a loop
- }
- } else {
- String msg = "Failed to connect to " + datanode.addr + " for file "
- + src + " for block " + block.getBlock() + ":" + e;
- DFSClient.LOG.warn("Connection failure: " + msg, e);
- addToDeadNodes(datanode.info);
- throw new IOException(msg);
- }
- } finally {
- if (reader != null) {
- reader.close();
- }
- }
- }
- }
-
- /**
- * Refresh cached block locations.
- * @param block The currently cached block locations
- * @return Refreshed block locations
- * @throws IOException
- */
- protected LocatedBlock refreshLocatedBlock(LocatedBlock block)
- throws IOException {
- return getBlockAt(block.getStartOffset());
- }
-
- /**
- * This method verifies that the read portions are valid and do not overlap
- * with each other.
- */
- private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) {
- Preconditions.checkArgument(offsets.length == lengths.length && offsets.length > 0);
- int sum = 0;
- for (int i = 0; i < lengths.length; i++) {
- if (i > 0) {
- int gap = offsets[i] - offsets[i - 1];
- // make sure read portions do not overlap with each other
- Preconditions.checkArgument(gap >= lengths[i - 1]);
- }
- sum += lengths[i];
- }
- Preconditions.checkArgument(sum == totalLen);
- }
-
- /**
- * Like {@link #fetchBlockByteRange}except we start up a second, parallel,
- * 'hedged' read if the first read is taking longer than configured amount of
- * time. We then wait on which ever read returns first.
- */
- private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
- long end, byte[] buf, int offset,
- Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
- throws IOException {
- final DfsClientConf conf = dfsClient.getConf();
- ArrayList<Future<ByteBuffer>> futures = new ArrayList<Future<ByteBuffer>>();
- CompletionService<ByteBuffer> hedgedService =
- new ExecutorCompletionService<ByteBuffer>(
- dfsClient.getHedgedReadsThreadPool());
- ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
- ByteBuffer bb = null;
- int len = (int) (end - start + 1);
- int hedgedReadId = 0;
- block = refreshLocatedBlock(block);
- while (true) {
- // see HDFS-6591, this metric is used to verify/catch unnecessary loops
- hedgedReadOpsLoopNumForTesting++;
- DNAddrPair chosenNode = null;
- // there is no request already executing.
- if (futures.isEmpty()) {
- // chooseDataNode is a commitment. If no node, we go to
- // the NN to reget block locations. Only go here on first read.
- chosenNode = chooseDataNode(block, ignored);
- bb = ByteBuffer.wrap(buf, offset, len);
- Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
- chosenNode, block, start, end, bb,
- corruptedBlockMap, hedgedReadId++);
- Future<ByteBuffer> firstRequest = hedgedService
- .submit(getFromDataNodeCallable);
- futures.add(firstRequest);
- try {
- Future<ByteBuffer> future = hedgedService.poll(
- conf.getHedgedReadThresholdMillis(), TimeUnit.MILLISECONDS);
- if (future != null) {
- future.get();
- return;
- }
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("Waited " + conf.getHedgedReadThresholdMillis()
- + "ms to read from " + chosenNode.info
- + "; spawning hedged read");
- }
- // Ignore this node on next go around.
- ignored.add(chosenNode.info);
- dfsClient.getHedgedReadMetrics().incHedgedReadOps();
- continue; // no need to refresh block locations
- } catch (InterruptedException e) {
- // Ignore
- } catch (ExecutionException e) {
- // Ignore already logged in the call.
- }
- } else {
- // We are starting up a 'hedged' read. We have a read already
- // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
- // If no nodes to do hedged reads against, pass.
- try {
- chosenNode = getBestNodeDNAddrPair(block, ignored);
- if (chosenNode == null) {
- chosenNode = chooseDataNode(block, ignored);
- }
- bb = ByteBuffer.allocate(len);
- Callable<ByteBuffer> getFromDataNodeCallable = getFromOneDataNode(
- chosenNode, block, start, end, bb,
- corruptedBlockMap, hedgedReadId++);
- Future<ByteBuffer> oneMoreRequest = hedgedService
- .submit(getFromDataNodeCallable);
- futures.add(oneMoreRequest);
- } catch (IOException ioe) {
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("Failed getting node for hedged read: "
- + ioe.getMessage());
- }
- }
- // if not succeeded. Submit callables for each datanode in a loop, wait
- // for a fixed interval and get the result from the fastest one.
- try {
- ByteBuffer result = getFirstToComplete(hedgedService, futures);
- // cancel the rest.
- cancelAll(futures);
- if (result.array() != buf) { // compare the array pointers
- dfsClient.getHedgedReadMetrics().incHedgedReadWins();
- System.arraycopy(result.array(), result.position(), buf, offset,
- len);
- } else {
- dfsClient.getHedgedReadMetrics().incHedgedReadOps();
- }
- return;
- } catch (InterruptedException ie) {
- // Ignore and retry
- }
- // We got here if exception. Ignore this node on next go around IFF
- // we found a chosenNode to hedge read against.
- if (chosenNode != null && chosenNode.info != null) {
- ignored.add(chosenNode.info);
- }
- }
- }
- }
-
- @VisibleForTesting
- public long getHedgedReadOpsLoopNumForTesting() {
- return hedgedReadOpsLoopNumForTesting;
- }
-
- private ByteBuffer getFirstToComplete(
- CompletionService<ByteBuffer> hedgedService,
- ArrayList<Future<ByteBuffer>> futures) throws InterruptedException {
- if (futures.isEmpty()) {
- throw new InterruptedException("let's retry");
- }
- Future<ByteBuffer> future = null;
- try {
- future = hedgedService.take();
- ByteBuffer bb = future.get();
- futures.remove(future);
- return bb;
- } catch (ExecutionException e) {
- // already logged in the Callable
- futures.remove(future);
- } catch (CancellationException ce) {
- // already logged in the Callable
- futures.remove(future);
- }
-
- throw new InterruptedException("let's retry");
- }
-
- private void cancelAll(List<Future<ByteBuffer>> futures) {
- for (Future<ByteBuffer> future : futures) {
- // Unfortunately, hdfs reads do not take kindly to interruption.
- // Threads return a variety of interrupted-type exceptions but
- // also complaints about invalid pbs -- likely because read
- // is interrupted before gets whole pb. Also verbose WARN
- // logging. So, for now, do not interrupt running read.
- future.cancel(false);
- }
- }
-
- /**
- * Should the block access token be refetched on an exception
- *
- * @param ex Exception received
- * @param targetAddr Target datanode address from where exception was received
- * @return true if block access token has expired or invalid and it should be
- * refetched
- */
- protected static boolean tokenRefetchNeeded(IOException ex,
- InetSocketAddress targetAddr) {
- /*
- * Get a new access token and retry. Retry is needed in 2 cases. 1)
- * When both NN and DN re-started while DFSClient holding a cached
- * access token. 2) In the case that NN fails to update its
- * access key at pre-set interval (by a wide margin) and
- * subsequently restarts. In this case, DN re-registers itself with
- * NN and receives a new access key, but DN will delete the old
- * access key from its memory since it's considered expired based on
- * the estimated expiration date.
- */
- if (ex instanceof InvalidBlockTokenException || ex instanceof InvalidToken) {
- DFSClient.LOG.info("Access token was invalid when connecting to "
- + targetAddr + " : " + ex);
- return true;
- }
- return false;
- }
-
- /**
- * Read bytes starting from the specified position.
- *
- * @param position start read from this position
- * @param buffer read buffer
- * @param offset offset into buffer
- * @param length number of bytes to read
- *
- * @return actual number of bytes read
- */
- @Override
- public int read(long position, byte[] buffer, int offset, int length)
- throws IOException {
- TraceScope scope =
- dfsClient.getPathTraceScope("DFSInputStream#byteArrayPread", src);
- try {
- return pread(position, buffer, offset, length);
- } finally {
- scope.close();
- }
- }
-
- private int pread(long position, byte[] buffer, int offset, int length)
- throws IOException {
- // sanity checks
- dfsClient.checkOpen();
- if (closed.get()) {
- throw new IOException("Stream closed");
- }
- failures = 0;
- long filelen = getFileLength();
- if ((position < 0) || (position >= filelen)) {
- return -1;
- }
- int realLen = length;
- if ((position + length) > filelen) {
- realLen = (int)(filelen - position);
- }
-
- // determine the block and byte range within the block
- // corresponding to position and realLen
- List<LocatedBlock> blockRange = getBlockRange(position, realLen);
- int remaining = realLen;
- Map<ExtendedBlock,Set<DatanodeInfo>> corruptedBlockMap
- = new HashMap<ExtendedBlock, Set<DatanodeInfo>>();
- for (LocatedBlock blk : blockRange) {
- long targetStart = position - blk.getStartOffset();
- long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
- try {
- if (dfsClient.isHedgedReadsEnabled()) {
- hedgedFetchBlockByteRange(blk, targetStart,
- targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
- } else {
- fetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1,
- buffer, offset, corruptedBlockMap);
- }
- } finally {
- // Check and report if any block replicas are corrupted.
- // BlockMissingException may be caught if all block replicas are
- // corrupted.
- reportCheckSumFailure(corruptedBlockMap, blk.getLocations().length);
- }
-
- remaining -= bytesToRead;
- position += bytesToRead;
- offset += bytesToRead;
- }
- assert remaining == 0 : "Wrong number of bytes read.";
- if (dfsClient.stats != null) {
- dfsClient.stats.incrementBytesRead(realLen);
- }
- return realLen;
- }
-
- /**
- * DFSInputStream reports checksum failure.
- * Case I : client has tried multiple data nodes and at least one of the
- * attempts has succeeded. We report the other failures as corrupted block to
- * namenode.
- * Case II: client has tried out all data nodes, but all failed. We
- * only report if the total number of replica is 1. We do not
- * report otherwise since this maybe due to the client is a handicapped client
- * (who can not read).
- * @param corruptedBlockMap map of corrupted blocks
- * @param dataNodeCount number of data nodes who contains the block replicas
- */
- protected void reportCheckSumFailure(
- Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
- int dataNodeCount) {
- if (corruptedBlockMap.isEmpty()) {
- return;
- }
- Iterator<Entry<ExtendedBlock, Set<DatanodeInfo>>> it = corruptedBlockMap
- .entrySet().iterator();
- Entry<ExtendedBlock, Set<DatanodeInfo>> entry = it.next();
- ExtendedBlock blk = entry.getKey();
- Set<DatanodeInfo> dnSet = entry.getValue();
- if (((dnSet.size() < dataNodeCount) && (dnSet.size() > 0))
- || ((dataNodeCount == 1) && (dnSet.size() == dataNodeCount))) {
- DatanodeInfo[] locs = new DatanodeInfo[dnSet.size()];
- int i = 0;
- for (DatanodeInfo dn:dnSet) {
- locs[i++] = dn;
- }
- LocatedBlock [] lblocks = { new LocatedBlock(blk, locs) };
- dfsClient.reportChecksumFailure(src, lblocks);
- }
- corruptedBlockMap.clear();
- }
-
- @Override
- public long skip(long n) throws IOException {
- if ( n > 0 ) {
- long curPos = getPos();
- long fileLen = getFileLength();
- if( n+curPos > fileLen ) {
- n = fileLen - curPos;
- }
- seek(curPos+n);
- return n;
- }
- return n < 0 ? -1 : 0;
- }
-
- /**
- * Seek to a new arbitrary location
- */
- @Override
- public synchronized void seek(long targetPos) throws IOException {
- if (targetPos > getFileLength()) {
- throw new EOFException("Cannot seek after EOF");
- }
- if (targetPos < 0) {
- throw new EOFException("Cannot seek to negative offset");
- }
- if (closed.get()) {
- throw new IOException("Stream is closed!");
- }
- boolean done = false;
- if (pos <= targetPos && targetPos <= blockEnd) {
- //
- // If this seek is to a positive position in the current
- // block, and this piece of data might already be lying in
- // the TCP buffer, then just eat up the intervening data.
- //
- int diff = (int)(targetPos - pos);
- if (diff <= blockReader.available()) {
- try {
- pos += blockReader.skip(diff);
- if (pos == targetPos) {
- done = true;
- } else {
- // The range was already checked. If the block reader returns
- // something unexpected instead of throwing an exception, it is
- // most likely a bug.
- String errMsg = "BlockReader failed to seek to " +
- targetPos + ". Instead, it seeked to " + pos + ".";
- DFSClient.LOG.warn(errMsg);
- throw new IOException(errMsg);
- }
- } catch (IOException e) {//make following read to retry
- if(DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("Exception while seek to " + targetPos
- + " from " + getCurrentBlock() + " of " + src + " from "
- + currentNode, e);
- }
- }
- }
- }
- if (!done) {
- pos = targetPos;
- blockEnd = -1;
- }
- }
-
- /**
- * Same as {@link #seekToNewSource(long)} except that it does not exclude
- * the current datanode and might connect to the same node.
- */
- private boolean seekToBlockSource(long targetPos)
- throws IOException {
- currentNode = blockSeekTo(targetPos);
- return true;
- }
-
- /**
- * Seek to given position on a node other than the current node. If
- * a node other than the current node is found, then returns true.
- * If another node could not be found, then returns false.
- */
- @Override
- public synchronized boolean seekToNewSource(long targetPos) throws IOException {
- if (currentNode == null) {
- return seekToBlockSource(targetPos);
- }
- boolean markedDead = deadNodes.containsKey(currentNode);
- addToDeadNodes(currentNode);
- DatanodeInfo oldNode = currentNode;
- DatanodeInfo newNode = blockSeekTo(targetPos);
- if (!markedDead) {
- /* remove it from deadNodes. blockSeekTo could have cleared
- * deadNodes and added currentNode again. Thats ok. */
- deadNodes.remove(oldNode);
- }
- if (!oldNode.getDatanodeUuid().equals(newNode.getDatanodeUuid())) {
- currentNode = newNode;
- return true;
- } else {
- return false;
- }
- }
-
- /**
- */
- @Override
- public synchronized long getPos() {
- return pos;
- }
-
- /** Return the size of the remaining available bytes
- * if the size is less than or equal to {@link Integer#MAX_VALUE},
- * otherwise, return {@link Integer#MAX_VALUE}.
- */
- @Override
- public synchronized int available() throws IOException {
- if (closed.get()) {
- throw new IOException("Stream closed");
- }
-
- final long remaining = getFileLength() - pos;
- return remaining <= Integer.MAX_VALUE? (int)remaining: Integer.MAX_VALUE;
- }
-
- /**
- * We definitely don't support marks
- */
- @Override
- public boolean markSupported() {
- return false;
- }
- @Override
- public void mark(int readLimit) {
- }
- @Override
- public void reset() throws IOException {
- throw new IOException("Mark/reset not supported");
- }
-
- /** Utility class to encapsulate data node info and its address. */
- static final class DNAddrPair {
- final DatanodeInfo info;
- final InetSocketAddress addr;
- final StorageType storageType;
-
- DNAddrPair(DatanodeInfo info, InetSocketAddress addr,
- StorageType storageType) {
- this.info = info;
- this.addr = addr;
- this.storageType = storageType;
- }
- }
-
- /**
- * Get statistics about the reads which this DFSInputStream has done.
- */
- public ReadStatistics getReadStatistics() {
- synchronized(infoLock) {
- return new ReadStatistics(readStatistics);
- }
- }
-
- /**
- * Clear statistics about the reads which this DFSInputStream has done.
- */
- public void clearReadStatistics() {
- synchronized(infoLock) {
- readStatistics.clear();
- }
- }
-
- public FileEncryptionInfo getFileEncryptionInfo() {
- synchronized(infoLock) {
- return fileEncryptionInfo;
- }
- }
-
- protected void closeCurrentBlockReaders() {
- if (blockReader == null) return;
- // Close the current block reader so that the new caching settings can
- // take effect immediately.
- try {
- blockReader.close();
- } catch (IOException e) {
- DFSClient.LOG.error("error closing blockReader", e);
- }
- blockReader = null;
- blockEnd = -1;
- }
-
- @Override
- public synchronized void setReadahead(Long readahead)
- throws IOException {
- synchronized (infoLock) {
- this.cachingStrategy =
- new CachingStrategy.Builder(this.cachingStrategy).setReadahead(readahead).build();
- }
- closeCurrentBlockReaders();
- }
-
- @Override
- public synchronized void setDropBehind(Boolean dropBehind)
- throws IOException {
- synchronized (infoLock) {
- this.cachingStrategy =
- new CachingStrategy.Builder(this.cachingStrategy).setDropBehind(dropBehind).build();
- }
- closeCurrentBlockReaders();
- }
-
- /**
- * The immutable empty buffer we return when we reach EOF when doing a
- * zero-copy read.
- */
- private static final ByteBuffer EMPTY_BUFFER =
- ByteBuffer.allocateDirect(0).asReadOnlyBuffer();
-
- @Override
- public synchronized ByteBuffer read(ByteBufferPool bufferPool,
- int maxLength, EnumSet<ReadOption> opts)
- throws IOException, UnsupportedOperationException {
- if (maxLength == 0) {
- return EMPTY_BUFFER;
- } else if (maxLength < 0) {
- throw new IllegalArgumentException("can't read a negative " +
- "number of bytes.");
- }
- if ((blockReader == null) || (blockEnd == -1)) {
- if (pos >= getFileLength()) {
- return null;
- }
- /*
- * If we don't have a blockReader, or the one we have has no more bytes
- * left to read, we call seekToBlockSource to get a new blockReader and
- * recalculate blockEnd. Note that we assume we're not at EOF here
- * (we check this above).
- */
- if ((!seekToBlockSource(pos)) || (blockReader == null)) {
- throw new IOException("failed to allocate new BlockReader " +
- "at position " + pos);
- }
- }
- ByteBuffer buffer = null;
- if (dfsClient.getConf().getShortCircuitConf().isShortCircuitMmapEnabled()) {
- buffer = tryReadZeroCopy(maxLength, opts);
- }
- if (buffer != null) {
- return buffer;
- }
- buffer = ByteBufferUtil.fallbackRead(this, bufferPool, maxLength);
- if (buffer != null) {
- getExtendedReadBuffers().put(buffer, bufferPool);
- }
- return buffer;
- }
-
- private synchronized ByteBuffer tryReadZeroCopy(int maxLength,
- EnumSet<ReadOption> opts) throws IOException {
- // Copy 'pos' and 'blockEnd' to local variables to make it easier for the
- // JVM to optimize this function.
- final long curPos = pos;
- final long curEnd = blockEnd;
- final long blockStartInFile = currentLocatedBlock.getStartOffset();
- final long blockPos = curPos - blockStartInFile;
-
- // Shorten this read if the end of the block is nearby.
- long length63;
- if ((curPos + maxLength) <= (curEnd + 1)) {
- length63 = maxLength;
- } else {
- length63 = 1 + curEnd - curPos;
- if (length63 <= 0) {
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
- curPos + " of " + src + "; " + length63 + " bytes left in block. " +
- "blockPos=" + blockPos + "; curPos=" + curPos +
- "; curEnd=" + curEnd);
- }
- return null;
- }
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("Reducing read length from " + maxLength +
- " to " + length63 + " to avoid going more than one byte " +
- "past the end of the block. blockPos=" + blockPos +
- "; curPos=" + curPos + "; curEnd=" + curEnd);
- }
- }
- // Make sure that don't go beyond 31-bit offsets in the MappedByteBuffer.
- int length;
- if (blockPos + length63 <= Integer.MAX_VALUE) {
- length = (int)length63;
- } else {
- long length31 = Integer.MAX_VALUE - blockPos;
- if (length31 <= 0) {
- // Java ByteBuffers can't be longer than 2 GB, because they use
- // 4-byte signed integers to represent capacity, etc.
- // So we can't mmap the parts of the block higher than the 2 GB offset.
- // FIXME: we could work around this with multiple memory maps.
- // See HDFS-5101.
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("Unable to perform a zero-copy read from offset " +
- curPos + " of " + src + "; 31-bit MappedByteBuffer limit " +
- "exceeded. blockPos=" + blockPos + ", curEnd=" + curEnd);
- }
- return null;
- }
- length = (int)length31;
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("Reducing read length from " + maxLength +
- " to " + length + " to avoid 31-bit limit. " +
- "blockPos=" + blockPos + "; curPos=" + curPos +
- "; curEnd=" + curEnd);
- }
- }
- final ClientMmap clientMmap = blockReader.getClientMmap(opts);
- if (clientMmap == null) {
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("unable to perform a zero-copy read from offset " +
- curPos + " of " + src + "; BlockReader#getClientMmap returned " +
- "null.");
- }
- return null;
- }
- boolean success = false;
- ByteBuffer buffer;
- try {
- seek(curPos + length);
- buffer = clientMmap.getMappedByteBuffer().asReadOnlyBuffer();
- buffer.position((int)blockPos);
- buffer.limit((int)(blockPos + length));
- getExtendedReadBuffers().put(buffer, clientMmap);
- synchronized (infoLock) {
- readStatistics.addZeroCopyBytes(length);
- }
- if (DFSClient.LOG.isDebugEnabled()) {
- DFSClient.LOG.debug("readZeroCopy read " + length +
- " bytes from offset " + curPos + " via the zero-copy read " +
- "path. blockEnd = " + blockEnd);
- }
- success = true;
- } finally {
- if (!success) {
- IOUtils.closeQuietly(clientMmap);
- }
- }
- return buffer;
- }
-
- @Override
- public synchronized void releaseBuffer(ByteBuffer buffer) {
- if (buffer == EMPTY_BUFFER) return;
- Object val = getExtendedReadBuffers().remove(buffer);
- if (val == null) {
- throw new IllegalArgumentException("tried to release a buffer " +
- "that was not created by this stream, " + buffer);
- }
- if (val instanceof ClientMmap) {
- IOUtils.closeQuietly((ClientMmap)val);
- } else if (val instanceof ByteBufferPool) {
- ((ByteBufferPool)val).putBuffer(buffer);
- }
- }
-
- @Override
- public synchronized void unbuffer() {
- closeCurrentBlockReaders();
- }
-}
[12/12] hadoop git commit: HDFS-8053. Move DFSIn/OutputStream and
related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Posted by wh...@apache.org.
HDFS-8053. Move DFSIn/OutputStream and related classes to hadoop-hdfs-client. Contributed by Mingliang Liu.
Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/94cbb6d1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/94cbb6d1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/94cbb6d1
Branch: refs/heads/branch-2
Commit: 94cbb6d16483ba011b7104565b4084bf2a3eb6e6
Parents: b46e4ce
Author: Haohui Mai <wh...@apache.org>
Authored: Sat Sep 26 11:08:25 2015 -0700
Committer: Haohui Mai <wh...@apache.org>
Committed: Sat Sep 26 11:16:50 2015 -0700
----------------------------------------------------------------------
.../dev-support/findbugsExcludeFile.xml | 24 +
.../apache/hadoop/fs/BlockStorageLocation.java | 52 +
.../org/apache/hadoop/fs/HdfsBlockLocation.java | 47 +
.../java/org/apache/hadoop/fs/HdfsVolumeId.java | 73 +
.../java/org/apache/hadoop/fs/VolumeId.java | 40 +
.../hadoop/hdfs/BlockMissingException.java | 65 +
.../apache/hadoop/hdfs/BlockReaderFactory.java | 893 +++++
.../hadoop/hdfs/BlockStorageLocationUtil.java | 369 ++
.../java/org/apache/hadoop/hdfs/DFSClient.java | 3203 ++++++++++++++++++
.../hadoop/hdfs/DFSClientFaultInjector.java | 60 +
.../hadoop/hdfs/DFSHedgedReadMetrics.java | 58 +
.../hadoop/hdfs/DFSInotifyEventInputStream.java | 239 ++
.../org/apache/hadoop/hdfs/DFSInputStream.java | 1915 +++++++++++
.../org/apache/hadoop/hdfs/DFSOutputStream.java | 923 +++++
.../java/org/apache/hadoop/hdfs/DFSPacket.java | 345 ++
.../org/apache/hadoop/hdfs/DFSUtilClient.java | 24 +
.../org/apache/hadoop/hdfs/DataStreamer.java | 1904 +++++++++++
.../hadoop/hdfs/HdfsConfigurationLoader.java | 44 +
.../apache/hadoop/hdfs/RemotePeerFactory.java | 43 +
.../hdfs/UnknownCipherSuiteException.java | 35 +
.../UnknownCryptoProtocolVersionException.java | 38 +
.../org/apache/hadoop/hdfs/XAttrHelper.java | 174 +
.../hadoop/hdfs/client/HdfsDataInputStream.java | 113 +
.../hdfs/client/HdfsDataOutputStream.java | 112 +
.../hadoop/hdfs/client/impl/LeaseRenewer.java | 524 +++
.../hdfs/inotify/MissingEventsException.java | 54 +
.../hadoop/hdfs/protocol/AclException.java | 39 +
.../hdfs/protocol/CacheDirectiveIterator.java | 130 +
.../hadoop/hdfs/protocol/CachePoolIterator.java | 63 +
.../hdfs/protocol/EncryptionZoneIterator.java | 64 +
.../QuotaByStorageTypeExceededException.java | 56 +
.../hdfs/protocol/UnresolvedPathException.java | 87 +
.../datatransfer/ReplaceDatanodeOnFailure.java | 200 ++
.../datanode/ReplicaNotFoundException.java | 53 +
.../namenode/RetryStartFileException.java | 36 +
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt | 3 +
.../dev-support/findbugsExcludeFile.xml | 19 -
.../apache/hadoop/fs/BlockStorageLocation.java | 52 -
.../org/apache/hadoop/fs/HdfsBlockLocation.java | 47 -
.../java/org/apache/hadoop/fs/HdfsVolumeId.java | 73 -
.../java/org/apache/hadoop/fs/VolumeId.java | 40 -
.../hadoop/hdfs/BlockMissingException.java | 65 -
.../apache/hadoop/hdfs/BlockReaderFactory.java | 892 -----
.../hadoop/hdfs/BlockStorageLocationUtil.java | 369 --
.../java/org/apache/hadoop/hdfs/DFSClient.java | 3200 -----------------
.../hadoop/hdfs/DFSClientFaultInjector.java | 57 -
.../org/apache/hadoop/hdfs/DFSConfigKeys.java | 2 -
.../hadoop/hdfs/DFSHedgedReadMetrics.java | 58 -
.../hadoop/hdfs/DFSInotifyEventInputStream.java | 239 --
.../org/apache/hadoop/hdfs/DFSInputStream.java | 1915 -----------
.../org/apache/hadoop/hdfs/DFSOutputStream.java | 923 -----
.../java/org/apache/hadoop/hdfs/DFSPacket.java | 345 --
.../java/org/apache/hadoop/hdfs/DFSUtil.java | 23 -
.../org/apache/hadoop/hdfs/DataStreamer.java | 1903 -----------
.../apache/hadoop/hdfs/HdfsConfiguration.java | 11 +-
.../apache/hadoop/hdfs/RemotePeerFactory.java | 43 -
.../hdfs/UnknownCipherSuiteException.java | 35 -
.../UnknownCryptoProtocolVersionException.java | 38 -
.../org/apache/hadoop/hdfs/XAttrHelper.java | 174 -
.../hadoop/hdfs/client/HdfsDataInputStream.java | 113 -
.../hdfs/client/HdfsDataOutputStream.java | 112 -
.../hadoop/hdfs/client/impl/LeaseRenewer.java | 524 ---
.../hdfs/inotify/MissingEventsException.java | 54 -
.../hadoop/hdfs/protocol/AclException.java | 39 -
.../hdfs/protocol/CacheDirectiveIterator.java | 130 -
.../hadoop/hdfs/protocol/CachePoolIterator.java | 63 -
.../hdfs/protocol/EncryptionZoneIterator.java | 64 -
.../QuotaByStorageTypeExceededException.java | 56 -
.../hdfs/protocol/UnresolvedPathException.java | 87 -
.../datatransfer/ReplaceDatanodeOnFailure.java | 200 --
.../hadoop/hdfs/server/balancer/Dispatcher.java | 3 +-
.../hdfs/server/datanode/BlockReceiver.java | 4 +-
.../hdfs/server/datanode/BlockSender.java | 4 +-
.../hadoop/hdfs/server/datanode/DataNode.java | 3 +-
.../hdfs/server/datanode/DataXceiver.java | 6 +-
.../datanode/ReplicaNotFoundException.java | 53 -
.../datanode/fsdataset/impl/BlockPoolSlice.java | 4 +-
.../datanode/fsdataset/impl/FsDatasetImpl.java | 8 +-
.../impl/RamDiskAsyncLazyPersistService.java | 4 +-
.../namenode/RetryStartFileException.java | 36 -
.../hdfs/server/namenode/TransferFsImage.java | 4 +-
.../datanode/TestFiDataTransferProtocol2.java | 1 -
.../org/apache/hadoop/hdfs/DFSTestUtil.java | 2 +-
.../hdfs/MiniDFSClusterWithNodeGroup.java | 2 +-
.../hadoop/hdfs/TestBlockStoragePolicy.java | 1 -
.../TestClientProtocolForPipelineRecovery.java | 6 +-
.../apache/hadoop/hdfs/TestCrcCorruption.java | 2 +-
.../org/apache/hadoop/hdfs/TestDFSUtil.java | 8 +-
.../org/apache/hadoop/hdfs/TestFileStatus.java | 2 +-
.../java/org/apache/hadoop/hdfs/TestPread.java | 10 +-
90 files changed, 12143 insertions(+), 12087 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
index 515da24..41a8564 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/dev-support/findbugsExcludeFile.xml
@@ -32,4 +32,28 @@
<Method name="allocSlot" />
<Bug pattern="UL_UNRELEASED_LOCK" />
</Match>
+ <Match>
+ <Class name="org.apache.hadoop.hdfs.DFSInputStream"/>
+ <Field name="tcpReadsDisabledForTesting"/>
+ <Bug pattern="MS_SHOULD_BE_FINAL"/>
+ </Match>
+
+ <!--
+ ResponseProccessor is thread that is designed to catch RuntimeException.
+ -->
+ <Match>
+ <Class name="org.apache.hadoop.hdfs.DataStreamer$ResponseProcessor" />
+ <Method name="run" />
+ <Bug pattern="REC_CATCH_EXCEPTION" />
+ </Match>
+
+ <!--
+ We use a separate lock to guard cachingStrategy in order to separate
+ locks for p-reads from seek + read invocations.
+ -->
+ <Match>
+ <Class name="org.apache.hadoop.hdfs.DFSInputStream" />
+ <Field name="cachingStrategy" />
+ <Bug pattern="IS2_INCONSISTENT_SYNC" />
+ </Match>
</FindBugsFilter>
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java
new file mode 100644
index 0000000..2200994
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/BlockStorageLocation.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Wrapper for {@link BlockLocation} that also adds {@link VolumeId} volume
+ * location information for each replica.
+ */
+@InterfaceStability.Unstable
+@InterfaceAudience.Public
+@Deprecated
+public class BlockStorageLocation extends BlockLocation {
+
+ private final VolumeId[] volumeIds;
+
+ public BlockStorageLocation(BlockLocation loc, VolumeId[] volumeIds)
+ throws IOException {
+ // Initialize with data from passed in BlockLocation
+ super(loc.getNames(), loc.getHosts(), loc.getTopologyPaths(), loc
+ .getOffset(), loc.getLength(), loc.isCorrupt());
+ this.volumeIds = volumeIds;
+ }
+
+ /**
+ * Gets the list of {@link VolumeId} corresponding to the block's replicas.
+ *
+ * @return volumeIds list of VolumeId for the block's replicas
+ */
+ public VolumeId[] getVolumeIds() {
+ return volumeIds;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
new file mode 100644
index 0000000..0ccacda
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsBlockLocation.java
@@ -0,0 +1,47 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+
+/**
+ * Wrapper for {@link BlockLocation} that also includes a {@link LocatedBlock},
+ * allowing more detailed queries to the datanode about a block.
+ *
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+public class HdfsBlockLocation extends BlockLocation {
+
+ private final LocatedBlock block;
+
+ public HdfsBlockLocation(BlockLocation loc, LocatedBlock block)
+ throws IOException {
+ // Initialize with data from passed in BlockLocation
+ super(loc);
+ this.block = block;
+ }
+
+ public LocatedBlock getLocatedBlock() {
+ return block;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
new file mode 100644
index 0000000..6e9d3d7
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/HdfsVolumeId.java
@@ -0,0 +1,73 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.util.StringUtils;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * HDFS-specific volume identifier which implements {@link VolumeId}. Can be
+ * used to differentiate between the data directories on a single datanode. This
+ * identifier is only unique on a per-datanode basis.
+ */
+@InterfaceStability.Unstable
+@InterfaceAudience.Public
+public class HdfsVolumeId implements VolumeId {
+
+ private final byte[] id;
+
+ public HdfsVolumeId(byte[] id) {
+ Preconditions.checkNotNull(id, "id cannot be null");
+ this.id = id;
+ }
+
+ @Override
+ public int compareTo(VolumeId arg0) {
+ if (arg0 == null) {
+ return 1;
+ }
+ return hashCode() - arg0.hashCode();
+ }
+
+ @Override
+ public int hashCode() {
+ return new HashCodeBuilder().append(id).toHashCode();
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (obj == null || obj.getClass() != getClass()) {
+ return false;
+ }
+ if (obj == this) {
+ return true;
+ }
+ HdfsVolumeId that = (HdfsVolumeId) obj;
+ return new EqualsBuilder().append(this.id, that.id).isEquals();
+ }
+
+ @Override
+ public String toString() {
+ return StringUtils.byteToHexString(id);
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/VolumeId.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/VolumeId.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/VolumeId.java
new file mode 100644
index 0000000..e56e304
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/fs/VolumeId.java
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Opaque interface that identifies a disk location. Subclasses
+ * should implement {@link Comparable} and override both equals and hashCode.
+ */
+@InterfaceStability.Unstable
+@InterfaceAudience.Public
+public interface VolumeId extends Comparable<VolumeId> {
+
+ @Override
+ abstract public int compareTo(VolumeId arg0);
+
+ @Override
+ abstract public int hashCode();
+
+ @Override
+ abstract public boolean equals(Object obj);
+
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
new file mode 100644
index 0000000..7bba8a4
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockMissingException.java
@@ -0,0 +1,65 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * This exception is thrown when a read encounters a block that has no locations
+ * associated with it.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlockMissingException extends IOException {
+
+ private static final long serialVersionUID = 1L;
+
+ private final String filename;
+ private final long offset;
+
+ /**
+ * An exception that indicates that file was corrupted.
+ * @param filename name of corrupted file
+ * @param description a description of the corruption details
+ */
+ public BlockMissingException(String filename, String description, long offset) {
+ super(description);
+ this.filename = filename;
+ this.offset = offset;
+ }
+
+ /**
+ * Returns the name of the corrupted file.
+ * @return name of corrupted file
+ */
+ public String getFile() {
+ return filename;
+ }
+
+ /**
+ * Returns the offset at which this file is corrupted
+ * @return offset of corrupted file
+ */
+ public long getOffset() {
+ return offset;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
new file mode 100644
index 0000000..69e9da2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderFactory.java
@@ -0,0 +1,893 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import static org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitFdResponse.USE_RECEIPT_VERIFICATION;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.net.InetSocketAddress;
+import java.util.List;
+
+import com.google.common.io.ByteArrayDataOutput;
+import com.google.common.io.ByteStreams;
+import org.apache.commons.lang.mutable.MutableBoolean;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf.ShortCircuitConf;
+import org.apache.hadoop.hdfs.net.DomainPeer;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
+import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
+import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
+import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.shortcircuit.DomainSocketFactory;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitCache.ShortCircuitReplicaCreator;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplicaInfo;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.Slot;
+import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
+import org.apache.hadoop.hdfs.util.IOUtilsClient;
+import org.apache.hadoop.ipc.RemoteException;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.PerformanceAdvisory;
+import org.apache.hadoop.util.Time;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Utility class to create BlockReader implementations.
+ */
+@InterfaceAudience.Private
+public class BlockReaderFactory implements ShortCircuitReplicaCreator {
+ static final Logger LOG = LoggerFactory.getLogger(BlockReaderFactory.class);
+
+ public static class FailureInjector {
+ public void injectRequestFileDescriptorsFailure() throws IOException {
+ // do nothing
+ }
+ public boolean getSupportsReceiptVerification() {
+ return true;
+ }
+ }
+
+ @VisibleForTesting
+ static ShortCircuitReplicaCreator
+ createShortCircuitReplicaInfoCallback = null;
+
+ private final DfsClientConf conf;
+
+ /**
+ * Injects failures into specific operations during unit tests.
+ */
+ private static FailureInjector failureInjector = new FailureInjector();
+
+ /**
+ * The file name, for logging and debugging purposes.
+ */
+ private String fileName;
+
+ /**
+ * The block ID and block pool ID to use.
+ */
+ private ExtendedBlock block;
+
+ /**
+ * The block token to use for security purposes.
+ */
+ private Token<BlockTokenIdentifier> token;
+
+ /**
+ * The offset within the block to start reading at.
+ */
+ private long startOffset;
+
+ /**
+ * If false, we won't try to verify the block checksum.
+ */
+ private boolean verifyChecksum;
+
+ /**
+ * The name of this client.
+ */
+ private String clientName;
+
+ /**
+ * The DataNode we're talking to.
+ */
+ private DatanodeInfo datanode;
+
+ /**
+ * StorageType of replica on DataNode.
+ */
+ private StorageType storageType;
+
+ /**
+ * If false, we won't try short-circuit local reads.
+ */
+ private boolean allowShortCircuitLocalReads;
+
+ /**
+ * The ClientContext to use for things like the PeerCache.
+ */
+ private ClientContext clientContext;
+
+ /**
+ * Number of bytes to read. -1 indicates no limit.
+ */
+ private long length = -1;
+
+ /**
+ * Caching strategy to use when reading the block.
+ */
+ private CachingStrategy cachingStrategy;
+
+ /**
+ * Socket address to use to connect to peer.
+ */
+ private InetSocketAddress inetSocketAddress;
+
+ /**
+ * Remote peer factory to use to create a peer, if needed.
+ */
+ private RemotePeerFactory remotePeerFactory;
+
+ /**
+ * UserGroupInformation to use for legacy block reader local objects, if needed.
+ */
+ private UserGroupInformation userGroupInformation;
+
+ /**
+ * Configuration to use for legacy block reader local objects, if needed.
+ */
+ private Configuration configuration;
+
+ /**
+ * Information about the domain socket path we should use to connect to the
+ * local peer-- or null if we haven't examined the local domain socket.
+ */
+ private DomainSocketFactory.PathInfo pathInfo;
+
+ /**
+ * The remaining number of times that we'll try to pull a socket out of the
+ * cache.
+ */
+ private int remainingCacheTries;
+
+ public BlockReaderFactory(DfsClientConf conf) {
+ this.conf = conf;
+ this.remainingCacheTries = conf.getNumCachedConnRetry();
+ }
+
+ public BlockReaderFactory setFileName(String fileName) {
+ this.fileName = fileName;
+ return this;
+ }
+
+ public BlockReaderFactory setBlock(ExtendedBlock block) {
+ this.block = block;
+ return this;
+ }
+
+ public BlockReaderFactory setBlockToken(Token<BlockTokenIdentifier> token) {
+ this.token = token;
+ return this;
+ }
+
+ public BlockReaderFactory setStartOffset(long startOffset) {
+ this.startOffset = startOffset;
+ return this;
+ }
+
+ public BlockReaderFactory setVerifyChecksum(boolean verifyChecksum) {
+ this.verifyChecksum = verifyChecksum;
+ return this;
+ }
+
+ public BlockReaderFactory setClientName(String clientName) {
+ this.clientName = clientName;
+ return this;
+ }
+
+ public BlockReaderFactory setDatanodeInfo(DatanodeInfo datanode) {
+ this.datanode = datanode;
+ return this;
+ }
+
+ public BlockReaderFactory setStorageType(StorageType storageType) {
+ this.storageType = storageType;
+ return this;
+ }
+
+ public BlockReaderFactory setAllowShortCircuitLocalReads(
+ boolean allowShortCircuitLocalReads) {
+ this.allowShortCircuitLocalReads = allowShortCircuitLocalReads;
+ return this;
+ }
+
+ public BlockReaderFactory setClientCacheContext(
+ ClientContext clientContext) {
+ this.clientContext = clientContext;
+ return this;
+ }
+
+ public BlockReaderFactory setLength(long length) {
+ this.length = length;
+ return this;
+ }
+
+ public BlockReaderFactory setCachingStrategy(
+ CachingStrategy cachingStrategy) {
+ this.cachingStrategy = cachingStrategy;
+ return this;
+ }
+
+ public BlockReaderFactory setInetSocketAddress (
+ InetSocketAddress inetSocketAddress) {
+ this.inetSocketAddress = inetSocketAddress;
+ return this;
+ }
+
+ public BlockReaderFactory setUserGroupInformation(
+ UserGroupInformation userGroupInformation) {
+ this.userGroupInformation = userGroupInformation;
+ return this;
+ }
+
+ public BlockReaderFactory setRemotePeerFactory(
+ RemotePeerFactory remotePeerFactory) {
+ this.remotePeerFactory = remotePeerFactory;
+ return this;
+ }
+
+ public BlockReaderFactory setConfiguration(
+ Configuration configuration) {
+ this.configuration = configuration;
+ return this;
+ }
+
+ @VisibleForTesting
+ public static void setFailureInjectorForTesting(FailureInjector injector) {
+ failureInjector = injector;
+ }
+
+ /**
+ * Build a BlockReader with the given options.
+ *
+ * This function will do the best it can to create a block reader that meets
+ * all of our requirements. We prefer short-circuit block readers
+ * (BlockReaderLocal and BlockReaderLocalLegacy) over remote ones, since the
+ * former avoid the overhead of socket communication. If short-circuit is
+ * unavailable, our next fallback is data transfer over UNIX domain sockets,
+ * if dfs.client.domain.socket.data.traffic has been enabled. If that doesn't
+ * work, we will try to create a remote block reader that operates over TCP
+ * sockets.
+ *
+ * There are a few caches that are important here.
+ *
+ * The ShortCircuitCache stores file descriptor objects which have been passed
+ * from the DataNode.
+ *
+ * The DomainSocketFactory stores information about UNIX domain socket paths
+ * that we not been able to use in the past, so that we don't waste time
+ * retrying them over and over. (Like all the caches, it does have a timeout,
+ * though.)
+ *
+ * The PeerCache stores peers that we have used in the past. If we can reuse
+ * one of these peers, we avoid the overhead of re-opening a socket. However,
+ * if the socket has been timed out on the remote end, our attempt to reuse
+ * the socket may end with an IOException. For that reason, we limit our
+ * attempts at socket reuse to dfs.client.cached.conn.retry times. After
+ * that, we create new sockets. This avoids the problem where a thread tries
+ * to talk to a peer that it hasn't talked to in a while, and has to clean out
+ * every entry in a socket cache full of stale entries.
+ *
+ * @return The new BlockReader. We will not return null.
+ *
+ * @throws InvalidToken
+ * If the block token was invalid.
+ * InvalidEncryptionKeyException
+ * If the encryption key was invalid.
+ * Other IOException
+ * If there was another problem.
+ */
+ public BlockReader build() throws IOException {
+ Preconditions.checkNotNull(configuration);
+ BlockReader reader = tryToCreateExternalBlockReader();
+ if (reader != null) {
+ return reader;
+ }
+ final ShortCircuitConf scConf = conf.getShortCircuitConf();
+ if (scConf.isShortCircuitLocalReads() && allowShortCircuitLocalReads) {
+ if (clientContext.getUseLegacyBlockReaderLocal()) {
+ reader = getLegacyBlockReaderLocal();
+ if (reader != null) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": returning new legacy block reader local.");
+ }
+ return reader;
+ }
+ } else {
+ reader = getBlockReaderLocal();
+ if (reader != null) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": returning new block reader local.");
+ }
+ return reader;
+ }
+ }
+ }
+ if (scConf.isDomainSocketDataTraffic()) {
+ reader = getRemoteBlockReaderFromDomain();
+ if (reader != null) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": returning new remote block reader using " +
+ "UNIX domain socket on " + pathInfo.getPath());
+ }
+ return reader;
+ }
+ }
+ Preconditions.checkState(!DFSInputStream.tcpReadsDisabledForTesting,
+ "TCP reads were disabled for testing, but we failed to " +
+ "do a non-TCP read.");
+ return getRemoteBlockReaderFromTcp();
+ }
+
+ private BlockReader tryToCreateExternalBlockReader() {
+ List<Class<? extends ReplicaAccessorBuilder>> clses =
+ conf.getReplicaAccessorBuilderClasses();
+ for (Class<? extends ReplicaAccessorBuilder> cls : clses) {
+ try {
+ ByteArrayDataOutput bado = ByteStreams.newDataOutput();
+ token.write(bado);
+ byte tokenBytes[] = bado.toByteArray();
+
+ Constructor<? extends ReplicaAccessorBuilder> ctor =
+ cls.getConstructor();
+ ReplicaAccessorBuilder builder = ctor.newInstance();
+ ReplicaAccessor accessor = builder.
+ setAllowShortCircuitReads(allowShortCircuitLocalReads).
+ setBlock(block.getBlockId(), block.getBlockPoolId()).
+ setGenerationStamp(block.getGenerationStamp()).
+ setBlockAccessToken(tokenBytes).
+ setClientName(clientName).
+ setConfiguration(configuration).
+ setFileName(fileName).
+ setVerifyChecksum(verifyChecksum).
+ setVisibleLength(length).
+ build();
+ if (accessor == null) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": No ReplicaAccessor created by " +
+ cls.getName());
+ }
+ } else {
+ return new ExternalBlockReader(accessor, length, startOffset);
+ }
+ } catch (Throwable t) {
+ LOG.warn("Failed to construct new object of type " +
+ cls.getName(), t);
+ }
+ }
+ return null;
+ }
+
+
+ /**
+ * Get {@link BlockReaderLocalLegacy} for short circuited local reads.
+ * This block reader implements the path-based style of local reads
+ * first introduced in HDFS-2246.
+ */
+ private BlockReader getLegacyBlockReaderLocal() throws IOException {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": trying to construct BlockReaderLocalLegacy");
+ }
+ if (!DFSUtilClient.isLocalAddress(inetSocketAddress)) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": can't construct BlockReaderLocalLegacy because " +
+ "the address " + inetSocketAddress + " is not local");
+ }
+ return null;
+ }
+ if (clientContext.getDisableLegacyBlockReaderLocal()) {
+ PerformanceAdvisory.LOG.debug("{}: can't construct " +
+ "BlockReaderLocalLegacy because " +
+ "disableLegacyBlockReaderLocal is set.", this);
+ return null;
+ }
+ IOException ioe;
+ try {
+ return BlockReaderLocalLegacy.newBlockReader(conf,
+ userGroupInformation, configuration, fileName, block, token,
+ datanode, startOffset, length, storageType);
+ } catch (RemoteException remoteException) {
+ ioe = remoteException.unwrapRemoteException(
+ InvalidToken.class, AccessControlException.class);
+ } catch (IOException e) {
+ ioe = e;
+ }
+ if ((!(ioe instanceof AccessControlException)) &&
+ isSecurityException(ioe)) {
+ // Handle security exceptions.
+ // We do not handle AccessControlException here, since
+ // BlockReaderLocalLegacy#newBlockReader uses that exception to indicate
+ // that the user is not in dfs.block.local-path-access.user, a condition
+ // which requires us to disable legacy SCR.
+ throw ioe;
+ }
+ LOG.warn(this + ": error creating legacy BlockReaderLocal. " +
+ "Disabling legacy local reads.", ioe);
+ clientContext.setDisableLegacyBlockReaderLocal();
+ return null;
+ }
+
+ private BlockReader getBlockReaderLocal() throws InvalidToken {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": trying to construct a BlockReaderLocal " +
+ "for short-circuit reads.");
+ }
+ if (pathInfo == null) {
+ pathInfo = clientContext.getDomainSocketFactory()
+ .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
+ }
+ if (!pathInfo.getPathState().getUsableForShortCircuit()) {
+ PerformanceAdvisory.LOG.debug("{}: {} is not usable for short circuit; " +
+ "giving up on BlockReaderLocal.", this, pathInfo);
+ return null;
+ }
+ ShortCircuitCache cache = clientContext.getShortCircuitCache();
+ ExtendedBlockId key = new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
+ ShortCircuitReplicaInfo info = cache.fetchOrCreate(key, this);
+ InvalidToken exc = info.getInvalidTokenException();
+ if (exc != null) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": got InvalidToken exception while trying to " +
+ "construct BlockReaderLocal via " + pathInfo.getPath());
+ }
+ throw exc;
+ }
+ if (info.getReplica() == null) {
+ PerformanceAdvisory.LOG.debug("{}: failed to get " +
+ "ShortCircuitReplica. Cannot construct " +
+ "BlockReaderLocal via {}", this, pathInfo.getPath());
+ return null;
+ }
+ return new BlockReaderLocal.Builder(conf.getShortCircuitConf()).
+ setFilename(fileName).
+ setBlock(block).
+ setStartOffset(startOffset).
+ setShortCircuitReplica(info.getReplica()).
+ setVerifyChecksum(verifyChecksum).
+ setCachingStrategy(cachingStrategy).
+ setStorageType(storageType).
+ build();
+ }
+
+ /**
+ * Fetch a pair of short-circuit block descriptors from a local DataNode.
+ *
+ * @return Null if we could not communicate with the datanode,
+ * a new ShortCircuitReplicaInfo object otherwise.
+ * ShortCircuitReplicaInfo objects may contain either an InvalidToken
+ * exception, or a ShortCircuitReplica object ready to use.
+ */
+ @Override
+ public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
+ if (createShortCircuitReplicaInfoCallback != null) {
+ ShortCircuitReplicaInfo info =
+ createShortCircuitReplicaInfoCallback.createShortCircuitReplicaInfo();
+ if (info != null) return info;
+ }
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": trying to create ShortCircuitReplicaInfo.");
+ }
+ BlockReaderPeer curPeer;
+ while (true) {
+ curPeer = nextDomainPeer();
+ if (curPeer == null) break;
+ if (curPeer.fromCache) remainingCacheTries--;
+ DomainPeer peer = (DomainPeer)curPeer.peer;
+ Slot slot = null;
+ ShortCircuitCache cache = clientContext.getShortCircuitCache();
+ try {
+ MutableBoolean usedPeer = new MutableBoolean(false);
+ slot = cache.allocShmSlot(datanode, peer, usedPeer,
+ new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId()),
+ clientName);
+ if (usedPeer.booleanValue()) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": allocShmSlot used up our previous socket " +
+ peer.getDomainSocket() + ". Allocating a new one...");
+ }
+ curPeer = nextDomainPeer();
+ if (curPeer == null) break;
+ peer = (DomainPeer)curPeer.peer;
+ }
+ ShortCircuitReplicaInfo info = requestFileDescriptors(peer, slot);
+ clientContext.getPeerCache().put(datanode, peer);
+ return info;
+ } catch (IOException e) {
+ if (slot != null) {
+ cache.freeSlot(slot);
+ }
+ if (curPeer.fromCache) {
+ // Handle an I/O error we got when using a cached socket.
+ // These are considered less serious, because the socket may be stale.
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(this + ": closing stale domain peer " + peer, e);
+ }
+ IOUtilsClient.cleanup(LOG, peer);
+ } else {
+ // Handle an I/O error we got when using a newly created socket.
+ // We temporarily disable the domain socket path for a few minutes in
+ // this case, to prevent wasting more time on it.
+ LOG.warn(this + ": I/O error requesting file descriptors. " +
+ "Disabling domain socket " + peer.getDomainSocket(), e);
+ IOUtilsClient.cleanup(LOG, peer);
+ clientContext.getDomainSocketFactory()
+ .disableDomainSocketPath(pathInfo.getPath());
+ return null;
+ }
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Request file descriptors from a DomainPeer.
+ *
+ * @param peer The peer to use for communication.
+ * @param slot If non-null, the shared memory slot to associate with the
+ * new ShortCircuitReplica.
+ *
+ * @return A ShortCircuitReplica object if we could communicate with the
+ * datanode; null, otherwise.
+ * @throws IOException If we encountered an I/O exception while communicating
+ * with the datanode.
+ */
+ private ShortCircuitReplicaInfo requestFileDescriptors(DomainPeer peer,
+ Slot slot) throws IOException {
+ ShortCircuitCache cache = clientContext.getShortCircuitCache();
+ final DataOutputStream out =
+ new DataOutputStream(new BufferedOutputStream(peer.getOutputStream()));
+ SlotId slotId = slot == null ? null : slot.getSlotId();
+ new Sender(out).requestShortCircuitFds(block, token, slotId, 1,
+ failureInjector.getSupportsReceiptVerification());
+ DataInputStream in = new DataInputStream(peer.getInputStream());
+ BlockOpResponseProto resp = BlockOpResponseProto.parseFrom(
+ PBHelperClient.vintPrefixed(in));
+ DomainSocket sock = peer.getDomainSocket();
+ failureInjector.injectRequestFileDescriptorsFailure();
+ switch (resp.getStatus()) {
+ case SUCCESS:
+ byte buf[] = new byte[1];
+ FileInputStream fis[] = new FileInputStream[2];
+ sock.recvFileInputStreams(fis, buf, 0, buf.length);
+ ShortCircuitReplica replica = null;
+ try {
+ ExtendedBlockId key =
+ new ExtendedBlockId(block.getBlockId(), block.getBlockPoolId());
+ if (buf[0] == USE_RECEIPT_VERIFICATION.getNumber()) {
+ LOG.trace("Sending receipt verification byte for slot " + slot);
+ sock.getOutputStream().write(0);
+ }
+ replica = new ShortCircuitReplica(key, fis[0], fis[1], cache,
+ Time.monotonicNow(), slot);
+ return new ShortCircuitReplicaInfo(replica);
+ } catch (IOException e) {
+ // This indicates an error reading from disk, or a format error. Since
+ // it's not a socket communication problem, we return null rather than
+ // throwing an exception.
+ LOG.warn(this + ": error creating ShortCircuitReplica.", e);
+ return null;
+ } finally {
+ if (replica == null) {
+ IOUtilsClient.cleanup(DFSClient.LOG, fis[0], fis[1]);
+ }
+ }
+ case ERROR_UNSUPPORTED:
+ if (!resp.hasShortCircuitAccessVersion()) {
+ LOG.warn("short-circuit read access is disabled for " +
+ "DataNode " + datanode + ". reason: " + resp.getMessage());
+ clientContext.getDomainSocketFactory()
+ .disableShortCircuitForPath(pathInfo.getPath());
+ } else {
+ LOG.warn("short-circuit read access for the file " +
+ fileName + " is disabled for DataNode " + datanode +
+ ". reason: " + resp.getMessage());
+ }
+ return null;
+ case ERROR_ACCESS_TOKEN:
+ String msg = "access control error while " +
+ "attempting to set up short-circuit access to " +
+ fileName + resp.getMessage();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(this + ":" + msg);
+ }
+ return new ShortCircuitReplicaInfo(new InvalidToken(msg));
+ default:
+ LOG.warn(this + ": unknown response code " + resp.getStatus() +
+ " while attempting to set up short-circuit access. " +
+ resp.getMessage());
+ clientContext.getDomainSocketFactory()
+ .disableShortCircuitForPath(pathInfo.getPath());
+ return null;
+ }
+ }
+
+ /**
+ * Get a RemoteBlockReader that communicates over a UNIX domain socket.
+ *
+ * @return The new BlockReader, or null if we failed to create the block
+ * reader.
+ *
+ * @throws InvalidToken If the block token was invalid.
+ * Potentially other security-related execptions.
+ */
+ private BlockReader getRemoteBlockReaderFromDomain() throws IOException {
+ if (pathInfo == null) {
+ pathInfo = clientContext.getDomainSocketFactory()
+ .getPathInfo(inetSocketAddress, conf.getShortCircuitConf());
+ }
+ if (!pathInfo.getPathState().getUsableForDataTransfer()) {
+ PerformanceAdvisory.LOG.debug("{}: not trying to create a " +
+ "remote block reader because the UNIX domain socket at {}" +
+ " is not usable.", this, pathInfo);
+ return null;
+ }
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": trying to create a remote block reader from the " +
+ "UNIX domain socket at " + pathInfo.getPath());
+ }
+
+ while (true) {
+ BlockReaderPeer curPeer = nextDomainPeer();
+ if (curPeer == null) break;
+ if (curPeer.fromCache) remainingCacheTries--;
+ DomainPeer peer = (DomainPeer)curPeer.peer;
+ BlockReader blockReader = null;
+ try {
+ blockReader = getRemoteBlockReader(peer);
+ return blockReader;
+ } catch (IOException ioe) {
+ IOUtilsClient.cleanup(LOG, peer);
+ if (isSecurityException(ioe)) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": got security exception while constructing " +
+ "a remote block reader from the unix domain socket at " +
+ pathInfo.getPath(), ioe);
+ }
+ throw ioe;
+ }
+ if (curPeer.fromCache) {
+ // Handle an I/O error we got when using a cached peer. These are
+ // considered less serious, because the underlying socket may be stale.
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Closed potentially stale domain peer " + peer, ioe);
+ }
+ } else {
+ // Handle an I/O error we got when using a newly created domain peer.
+ // We temporarily disable the domain socket path for a few minutes in
+ // this case, to prevent wasting more time on it.
+ LOG.warn("I/O error constructing remote block reader. Disabling " +
+ "domain socket " + peer.getDomainSocket(), ioe);
+ clientContext.getDomainSocketFactory()
+ .disableDomainSocketPath(pathInfo.getPath());
+ return null;
+ }
+ } finally {
+ if (blockReader == null) {
+ IOUtilsClient.cleanup(LOG, peer);
+ }
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Get a RemoteBlockReader that communicates over a TCP socket.
+ *
+ * @return The new BlockReader. We will not return null, but instead throw
+ * an exception if this fails.
+ *
+ * @throws InvalidToken
+ * If the block token was invalid.
+ * InvalidEncryptionKeyException
+ * If the encryption key was invalid.
+ * Other IOException
+ * If there was another problem.
+ */
+ private BlockReader getRemoteBlockReaderFromTcp() throws IOException {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": trying to create a remote block reader from a " +
+ "TCP socket");
+ }
+ BlockReader blockReader = null;
+ while (true) {
+ BlockReaderPeer curPeer = null;
+ Peer peer = null;
+ try {
+ curPeer = nextTcpPeer();
+ if (curPeer.fromCache) remainingCacheTries--;
+ peer = curPeer.peer;
+ blockReader = getRemoteBlockReader(peer);
+ return blockReader;
+ } catch (IOException ioe) {
+ if (isSecurityException(ioe)) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace(this + ": got security exception while constructing " +
+ "a remote block reader from " + peer, ioe);
+ }
+ throw ioe;
+ }
+ if ((curPeer != null) && curPeer.fromCache) {
+ // Handle an I/O error we got when using a cached peer. These are
+ // considered less serious, because the underlying socket may be
+ // stale.
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Closed potentially stale remote peer " + peer, ioe);
+ }
+ } else {
+ // Handle an I/O error we got when using a newly created peer.
+ LOG.warn("I/O error constructing remote block reader.", ioe);
+ throw ioe;
+ }
+ } finally {
+ if (blockReader == null) {
+ IOUtilsClient.cleanup(LOG, peer);
+ }
+ }
+ }
+ }
+
+ public static class BlockReaderPeer {
+ final Peer peer;
+ final boolean fromCache;
+
+ BlockReaderPeer(Peer peer, boolean fromCache) {
+ this.peer = peer;
+ this.fromCache = fromCache;
+ }
+ }
+
+ /**
+ * Get the next DomainPeer-- either from the cache or by creating it.
+ *
+ * @return the next DomainPeer, or null if we could not construct one.
+ */
+ private BlockReaderPeer nextDomainPeer() {
+ if (remainingCacheTries > 0) {
+ Peer peer = clientContext.getPeerCache().get(datanode, true);
+ if (peer != null) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("nextDomainPeer: reusing existing peer " + peer);
+ }
+ return new BlockReaderPeer(peer, true);
+ }
+ }
+ DomainSocket sock = clientContext.getDomainSocketFactory().
+ createSocket(pathInfo, conf.getSocketTimeout());
+ if (sock == null) return null;
+ return new BlockReaderPeer(new DomainPeer(sock), false);
+ }
+
+ /**
+ * Get the next TCP-based peer-- either from the cache or by creating it.
+ *
+ * @return the next Peer, or null if we could not construct one.
+ *
+ * @throws IOException If there was an error while constructing the peer
+ * (such as an InvalidEncryptionKeyException)
+ */
+ private BlockReaderPeer nextTcpPeer() throws IOException {
+ if (remainingCacheTries > 0) {
+ Peer peer = clientContext.getPeerCache().get(datanode, false);
+ if (peer != null) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("nextTcpPeer: reusing existing peer " + peer);
+ }
+ return new BlockReaderPeer(peer, true);
+ }
+ }
+ try {
+ Peer peer = remotePeerFactory.newConnectedPeer(inetSocketAddress, token,
+ datanode);
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("nextTcpPeer: created newConnectedPeer " + peer);
+ }
+ return new BlockReaderPeer(peer, false);
+ } catch (IOException e) {
+ if (LOG.isTraceEnabled()) {
+ LOG.trace("nextTcpPeer: failed to create newConnectedPeer " +
+ "connected to " + datanode);
+ }
+ throw e;
+ }
+ }
+
+ /**
+ * Determine if an exception is security-related.
+ *
+ * We need to handle these exceptions differently than other IOExceptions.
+ * They don't indicate a communication problem. Instead, they mean that there
+ * is some action the client needs to take, such as refetching block tokens,
+ * renewing encryption keys, etc.
+ *
+ * @param ioe The exception
+ * @return True only if the exception is security-related.
+ */
+ private static boolean isSecurityException(IOException ioe) {
+ return (ioe instanceof InvalidToken) ||
+ (ioe instanceof InvalidEncryptionKeyException) ||
+ (ioe instanceof InvalidBlockTokenException) ||
+ (ioe instanceof AccessControlException);
+ }
+
+ @SuppressWarnings("deprecation")
+ private BlockReader getRemoteBlockReader(Peer peer) throws IOException {
+ if (conf.getShortCircuitConf().isUseLegacyBlockReader()) {
+ return RemoteBlockReader.newBlockReader(fileName,
+ block, token, startOffset, length, conf.getIoBufferSize(),
+ verifyChecksum, clientName, peer, datanode,
+ clientContext.getPeerCache(), cachingStrategy);
+ } else {
+ return RemoteBlockReader2.newBlockReader(
+ fileName, block, token, startOffset, length,
+ verifyChecksum, clientName, peer, datanode,
+ clientContext.getPeerCache(), cachingStrategy);
+ }
+ }
+
+ @Override
+ public String toString() {
+ return "BlockReaderFactory(fileName=" + fileName + ", block=" + block + ")";
+ }
+
+ /**
+ * File name to print when accessing a block directly (from servlets)
+ * @param s Address of the block location
+ * @param poolId Block pool ID of the block
+ * @param blockId Block ID of the block
+ * @return string that has a file name for debug purposes
+ */
+ public static String getFileName(final InetSocketAddress s,
+ final String poolId, final long blockId) {
+ return s.toString() + ":" + poolId + ":" + blockId;
+ }
+}
http://git-wip-us.apache.org/repos/asf/hadoop/blob/94cbb6d1/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
new file mode 100644
index 0000000..807ede8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockStorageLocationUtil.java
@@ -0,0 +1,369 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdfs;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.BlockLocation;
+import org.apache.hadoop.fs.BlockStorageLocation;
+import org.apache.hadoop.fs.HdfsVolumeId;
+import org.apache.hadoop.fs.VolumeId;
+import org.apache.hadoop.hdfs.protocol.ClientDatanodeProtocol;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.HdfsBlocksMetadata;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
+import org.apache.hadoop.ipc.RPC;
+import org.apache.hadoop.security.token.Token;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@InterfaceAudience.Private
+@InterfaceStability.Unstable
+class BlockStorageLocationUtil {
+
+ static final Logger LOG = LoggerFactory
+ .getLogger(BlockStorageLocationUtil.class);
+
+ /**
+ * Create a list of {@link VolumeBlockLocationCallable} corresponding to a set
+ * of datanodes and blocks. The blocks must all correspond to the same
+ * block pool.
+ *
+ * @param datanodeBlocks
+ * Map of datanodes to block replicas at each datanode
+ * @return callables Used to query each datanode for location information on
+ * the block replicas at the datanode
+ */
+ private static List<VolumeBlockLocationCallable> createVolumeBlockLocationCallables(
+ Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
+ int timeout, boolean connectToDnViaHostname, Span parent) {
+
+ if (datanodeBlocks.isEmpty()) {
+ return Lists.newArrayList();
+ }
+
+ // Construct the callables, one per datanode
+ List<VolumeBlockLocationCallable> callables =
+ new ArrayList<VolumeBlockLocationCallable>();
+ for (Map.Entry<DatanodeInfo, List<LocatedBlock>> entry : datanodeBlocks
+ .entrySet()) {
+ // Construct RPC parameters
+ DatanodeInfo datanode = entry.getKey();
+ List<LocatedBlock> locatedBlocks = entry.getValue();
+ if (locatedBlocks.isEmpty()) {
+ continue;
+ }
+
+ // Ensure that the blocks all are from the same block pool.
+ String poolId = locatedBlocks.get(0).getBlock().getBlockPoolId();
+ for (LocatedBlock lb : locatedBlocks) {
+ if (!poolId.equals(lb.getBlock().getBlockPoolId())) {
+ throw new IllegalArgumentException(
+ "All blocks to be queried must be in the same block pool: " +
+ locatedBlocks.get(0).getBlock() + " and " + lb +
+ " are from different pools.");
+ }
+ }
+
+ long[] blockIds = new long[locatedBlocks.size()];
+ int i = 0;
+ List<Token<BlockTokenIdentifier>> dnTokens =
+ new ArrayList<Token<BlockTokenIdentifier>>(
+ locatedBlocks.size());
+ for (LocatedBlock b : locatedBlocks) {
+ blockIds[i++] = b.getBlock().getBlockId();
+ dnTokens.add(b.getBlockToken());
+ }
+ VolumeBlockLocationCallable callable = new VolumeBlockLocationCallable(
+ conf, datanode, poolId, blockIds, dnTokens, timeout,
+ connectToDnViaHostname, parent);
+ callables.add(callable);
+ }
+ return callables;
+ }
+
+ /**
+ * Queries datanodes for the blocks specified in <code>datanodeBlocks</code>,
+ * making one RPC to each datanode. These RPCs are made in parallel using a
+ * threadpool.
+ *
+ * @param datanodeBlocks
+ * Map of datanodes to the blocks present on the DN
+ * @return metadatas Map of datanodes to block metadata of the DN
+ * @throws InvalidBlockTokenException
+ * if client does not have read access on a requested block
+ */
+ static Map<DatanodeInfo, HdfsBlocksMetadata> queryDatanodesForHdfsBlocksMetadata(
+ Configuration conf, Map<DatanodeInfo, List<LocatedBlock>> datanodeBlocks,
+ int poolsize, int timeoutMs, boolean connectToDnViaHostname)
+ throws InvalidBlockTokenException {
+
+ List<VolumeBlockLocationCallable> callables =
+ createVolumeBlockLocationCallables(conf, datanodeBlocks, timeoutMs,
+ connectToDnViaHostname, Trace.currentSpan());
+
+ // Use a thread pool to execute the Callables in parallel
+ List<Future<HdfsBlocksMetadata>> futures =
+ new ArrayList<Future<HdfsBlocksMetadata>>();
+ ExecutorService executor = new ScheduledThreadPoolExecutor(poolsize);
+ try {
+ futures = executor.invokeAll(callables, timeoutMs,
+ TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ // Swallow the exception here, because we can return partial results
+ }
+ executor.shutdown();
+
+ Map<DatanodeInfo, HdfsBlocksMetadata> metadatas =
+ Maps.newHashMapWithExpectedSize(datanodeBlocks.size());
+ // Fill in metadatas with results from DN RPCs, where possible
+ for (int i = 0; i < futures.size(); i++) {
+ VolumeBlockLocationCallable callable = callables.get(i);
+ DatanodeInfo datanode = callable.getDatanodeInfo();
+ Future<HdfsBlocksMetadata> future = futures.get(i);
+ try {
+ HdfsBlocksMetadata metadata = future.get();
+ metadatas.put(callable.getDatanodeInfo(), metadata);
+ } catch (CancellationException e) {
+ LOG.info("Cancelled while waiting for datanode "
+ + datanode.getIpcAddr(false) + ": " + e.toString());
+ } catch (ExecutionException e) {
+ Throwable t = e.getCause();
+ if (t instanceof InvalidBlockTokenException) {
+ LOG.warn("Invalid access token when trying to retrieve "
+ + "information from datanode " + datanode.getIpcAddr(false));
+ throw (InvalidBlockTokenException) t;
+ }
+ else if (t instanceof UnsupportedOperationException) {
+ LOG.info("Datanode " + datanode.getIpcAddr(false) + " does not support"
+ + " required #getHdfsBlocksMetadata() API");
+ throw (UnsupportedOperationException) t;
+ } else {
+ LOG.info("Failed to query block locations on datanode " +
+ datanode.getIpcAddr(false) + ": " + t);
+ }
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Could not fetch information from datanode", t);
+ }
+ } catch (InterruptedException e) {
+ // Shouldn't happen, because invokeAll waits for all Futures to be ready
+ LOG.info("Interrupted while fetching HdfsBlocksMetadata");
+ }
+ }
+
+ return metadatas;
+ }
+
+ /**
+ * Group the per-replica {@link VolumeId} info returned from
+ * {@link DFSClient#queryDatanodesForHdfsBlocksMetadata(Map)} to be
+ * associated
+ * with the corresponding {@link LocatedBlock}.
+ *
+ * @param blocks
+ * Original LocatedBlock array
+ * @param metadatas
+ * VolumeId information for the replicas on each datanode
+ * @return blockVolumeIds per-replica VolumeId information associated with the
+ * parent LocatedBlock
+ */
+ static Map<LocatedBlock, List<VolumeId>> associateVolumeIdsWithBlocks(
+ List<LocatedBlock> blocks,
+ Map<DatanodeInfo, HdfsBlocksMetadata> metadatas) {
+
+ // Initialize mapping of ExtendedBlock to LocatedBlock.
+ // Used to associate results from DN RPCs to the parent LocatedBlock
+ Map<Long, LocatedBlock> blockIdToLocBlock =
+ new HashMap<Long, LocatedBlock>();
+ for (LocatedBlock b : blocks) {
+ blockIdToLocBlock.put(b.getBlock().getBlockId(), b);
+ }
+
+ // Initialize the mapping of blocks -> list of VolumeIds, one per replica
+ // This is filled out with real values from the DN RPCs
+ Map<LocatedBlock, List<VolumeId>> blockVolumeIds =
+ new HashMap<LocatedBlock, List<VolumeId>>();
+ for (LocatedBlock b : blocks) {
+ ArrayList<VolumeId> l = new ArrayList<VolumeId>(b.getLocations().length);
+ for (int i = 0; i < b.getLocations().length; i++) {
+ l.add(null);
+ }
+ blockVolumeIds.put(b, l);
+ }
+
+ // Iterate through the list of metadatas (one per datanode).
+ // For each metadata, if it's valid, insert its volume location information
+ // into the Map returned to the caller
+ for (Map.Entry<DatanodeInfo, HdfsBlocksMetadata> entry : metadatas.entrySet()) {
+ DatanodeInfo datanode = entry.getKey();
+ HdfsBlocksMetadata metadata = entry.getValue();
+ // Check if metadata is valid
+ if (metadata == null) {
+ continue;
+ }
+ long[] metaBlockIds = metadata.getBlockIds();
+ List<byte[]> metaVolumeIds = metadata.getVolumeIds();
+ List<Integer> metaVolumeIndexes = metadata.getVolumeIndexes();
+ // Add VolumeId for each replica in the HdfsBlocksMetadata
+ for (int j = 0; j < metaBlockIds.length; j++) {
+ int volumeIndex = metaVolumeIndexes.get(j);
+ long blockId = metaBlockIds[j];
+ // Skip if block wasn't found, or not a valid index into metaVolumeIds
+ // Also skip if the DN responded with a block we didn't ask for
+ if (volumeIndex == Integer.MAX_VALUE
+ || volumeIndex >= metaVolumeIds.size()
+ || !blockIdToLocBlock.containsKey(blockId)) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("No data for block " + blockId);
+ }
+ continue;
+ }
+ // Get the VolumeId by indexing into the list of VolumeIds
+ // provided by the datanode
+ byte[] volumeId = metaVolumeIds.get(volumeIndex);
+ HdfsVolumeId id = new HdfsVolumeId(volumeId);
+ // Find out which index we are in the LocatedBlock's replicas
+ LocatedBlock locBlock = blockIdToLocBlock.get(blockId);
+ DatanodeInfo[] dnInfos = locBlock.getLocations();
+ int index = -1;
+ for (int k = 0; k < dnInfos.length; k++) {
+ if (dnInfos[k].equals(datanode)) {
+ index = k;
+ break;
+ }
+ }
+ if (index < 0) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Datanode responded with a block volume id we did" +
+ " not request, omitting.");
+ }
+ continue;
+ }
+ // Place VolumeId at the same index as the DN's index in the list of
+ // replicas
+ List<VolumeId> volumeIds = blockVolumeIds.get(locBlock);
+ volumeIds.set(index, id);
+ }
+ }
+ return blockVolumeIds;
+ }
+
+ /**
+ * Helper method to combine a list of {@link LocatedBlock} with associated
+ * {@link VolumeId} information to form a list of {@link BlockStorageLocation}
+ * .
+ */
+ static BlockStorageLocation[] convertToVolumeBlockLocations(
+ List<LocatedBlock> blocks,
+ Map<LocatedBlock, List<VolumeId>> blockVolumeIds) throws IOException {
+ // Construct the final return value of VolumeBlockLocation[]
+ BlockLocation[] locations = DFSUtilClient.locatedBlocks2Locations(blocks);
+ List<BlockStorageLocation> volumeBlockLocs =
+ new ArrayList<BlockStorageLocation>(locations.length);
+ for (int i = 0; i < locations.length; i++) {
+ LocatedBlock locBlock = blocks.get(i);
+ List<VolumeId> volumeIds = blockVolumeIds.get(locBlock);
+ BlockStorageLocation bsLoc = new BlockStorageLocation(locations[i],
+ volumeIds.toArray(new VolumeId[0]));
+ volumeBlockLocs.add(bsLoc);
+ }
+ return volumeBlockLocs.toArray(new BlockStorageLocation[] {});
+ }
+
+ /**
+ * Callable that sets up an RPC proxy to a datanode and queries it for
+ * volume location information for a list of ExtendedBlocks.
+ */
+ private static class VolumeBlockLocationCallable implements
+ Callable<HdfsBlocksMetadata> {
+
+ private final Configuration configuration;
+ private final int timeout;
+ private final DatanodeInfo datanode;
+ private final String poolId;
+ private final long[] blockIds;
+ private final List<Token<BlockTokenIdentifier>> dnTokens;
+ private final boolean connectToDnViaHostname;
+ private final Span parentSpan;
+
+ VolumeBlockLocationCallable(Configuration configuration,
+ DatanodeInfo datanode, String poolId, long []blockIds,
+ List<Token<BlockTokenIdentifier>> dnTokens, int timeout,
+ boolean connectToDnViaHostname, Span parentSpan) {
+ this.configuration = configuration;
+ this.timeout = timeout;
+ this.datanode = datanode;
+ this.poolId = poolId;
+ this.blockIds = blockIds;
+ this.dnTokens = dnTokens;
+ this.connectToDnViaHostname = connectToDnViaHostname;
+ this.parentSpan = parentSpan;
+ }
+
+ public DatanodeInfo getDatanodeInfo() {
+ return datanode;
+ }
+
+ @Override
+ public HdfsBlocksMetadata call() throws Exception {
+ HdfsBlocksMetadata metadata = null;
+ // Create the RPC proxy and make the RPC
+ ClientDatanodeProtocol cdp = null;
+ TraceScope scope =
+ Trace.startSpan("getHdfsBlocksMetadata", parentSpan);
+ try {
+ cdp = DFSUtilClient.createClientDatanodeProtocolProxy(
+ datanode, configuration,
+ timeout, connectToDnViaHostname);
+ metadata = cdp.getHdfsBlocksMetadata(poolId, blockIds, dnTokens);
+ } catch (IOException e) {
+ // Bubble this up to the caller, handle with the Future
+ throw e;
+ } finally {
+ scope.close();
+ if (cdp != null) {
+ RPC.stopProxy(cdp);
+ }
+ }
+ return metadata;
+ }
+ }
+}