http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
deleted file mode 100644
index aa7c552..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/PrivateAzureDataLakeFileSystem.java
+++ /dev/null
@@ -1,1116 +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.web;
-
-import com.google.common.annotations.VisibleForTesting;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.BlockLocation;
-import org.apache.hadoop.fs.CreateFlag;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FSInputStream;
-import org.apache.hadoop.fs.FileStatus;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.web.resources.ADLFlush;
-import org.apache.hadoop.hdfs.web.resources.ADLGetOpParam;
-import org.apache.hadoop.hdfs.web.resources.ADLPostOpParam;
-import org.apache.hadoop.hdfs.web.resources.ADLPutOpParam;
-import org.apache.hadoop.hdfs.web.resources.ADLVersionInfo;
-import org.apache.hadoop.hdfs.web.resources.AppendADLNoRedirectParam;
-import org.apache.hadoop.hdfs.web.resources.BlockSizeParam;
-import org.apache.hadoop.hdfs.web.resources.BufferSizeParam;
-import org.apache.hadoop.hdfs.web.resources.CreateADLNoRedirectParam;
-import org.apache.hadoop.hdfs.web.resources.CreateFlagParam;
-import org.apache.hadoop.hdfs.web.resources.CreateParentParam;
-import org.apache.hadoop.hdfs.web.resources.GetOpParam;
-import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
-import org.apache.hadoop.hdfs.web.resources.LeaseParam;
-import org.apache.hadoop.hdfs.web.resources.LengthParam;
-import org.apache.hadoop.hdfs.web.resources.OffsetParam;
-import org.apache.hadoop.hdfs.web.resources.OverwriteParam;
-import org.apache.hadoop.hdfs.web.resources.Param;
-import org.apache.hadoop.hdfs.web.resources.PermissionParam;
-import org.apache.hadoop.hdfs.web.resources.PutOpParam;
-import org.apache.hadoop.hdfs.web.resources.ReadADLNoRedirectParam;
-import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.VersionInfo;
-import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_WEBHDFS_OAUTH_ENABLED_KEY;
-import static 
org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_WEBHDFS_OAUTH_ENABLED_DEFAULT;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.net.HttpURLConnection;
-import java.net.SocketException;
-import java.net.URI;
-import java.net.URL;
-import java.util.EnumSet;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Extended @see SWebHdfsFileSystem API. This class contains Azure data lake
- * specific stability, Reliability and performance improvement.
- * <p>
- * Motivation behind PrivateAzureDataLakeFileSystem to encapsulate dependent
- * implementation on org.apache.hadoop.hdfs.web package to configure query
- * parameters, configuration over HTTP request send to backend .. etc. This
- * class should be refactored and moved under package org.apache.hadoop.fs
- * .adl once the required dependent changes are made into ASF code.
- */
-public class PrivateAzureDataLakeFileSystem extends SWebHdfsFileSystem {
-
-  public static final String SCHEME = "adl";
-
-  // Feature configuration
-  private boolean featureGetBlockLocationLocallyBundled = true;
-  private boolean featureConcurrentReadWithReadAhead = true;
-  private boolean featureRedirectOff = true;
-  private boolean featureFlushWhenEOF = true;
-  private boolean overrideOwner = false;
-  private int maxConcurrentConnection;
-  private int maxBufferSize;
-  private String userName;
-
-  /**
-   * Constructor.
-   */
-  public PrivateAzureDataLakeFileSystem() {
-    try {
-      userName = UserGroupInformation.getCurrentUser().getShortUserName();
-    } catch (IOException e) {
-      userName = "hadoop";
-    }
-  }
-
-  @Override
-  public synchronized void initialize(URI uri, Configuration conf)
-      throws IOException {
-    if (!conf.getBoolean(DFS_WEBHDFS_OAUTH_ENABLED_KEY,
-                         DFS_WEBHDFS_OAUTH_ENABLED_DEFAULT)) {
-      // clone configuration, enable OAuth2
-      conf = new Configuration(conf);
-      conf.setBoolean(DFS_WEBHDFS_OAUTH_ENABLED_KEY, true);
-    }
-    super.initialize(uri, conf);
-    overrideOwner = getConf()
-        .getBoolean(ADLConfKeys.ADL_DEBUG_OVERRIDE_LOCAL_USER_AS_OWNER,
-            ADLConfKeys.ADL_DEBUG_SET_LOCAL_USER_AS_OWNER_DEFAULT);
-
-    featureRedirectOff = getConf()
-        .getBoolean(ADLConfKeys.ADL_FEATURE_REDIRECT_OFF,
-            ADLConfKeys.ADL_FEATURE_REDIRECT_OFF_DEFAULT);
-
-    featureGetBlockLocationLocallyBundled = getConf()
-        .getBoolean(ADLConfKeys.ADL_FEATURE_GET_BLOCK_LOCATION_LOCALLY_BUNDLED,
-            
ADLConfKeys.ADL_FEATURE_GET_BLOCK_LOCATION_LOCALLY_BUNDLED_DEFAULT);
-
-    featureConcurrentReadWithReadAhead = getConf().
-        getBoolean(ADLConfKeys.ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD,
-            ADLConfKeys.ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_DEFAULT);
-
-    maxBufferSize = getConf().getInt(
-        ADLConfKeys.ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_BUFFER_SIZE,
-        ADLConfKeys
-            .ADL_FEATURE_CONCURRENT_READ_WITH_READ_AHEAD_BUFFER_SIZE_DEFAULT);
-
-    maxConcurrentConnection = getConf().getInt(
-        ADLConfKeys.ADL_FEATURE_CONCURRENT_READ_AHEAD_MAX_CONCURRENT_CONN,
-        ADLConfKeys
-            .ADL_FEATURE_CONCURRENT_READ_AHEAD_MAX_CONCURRENT_CONN_DEFAULT);
-  }
-
-  @VisibleForTesting
-  protected boolean isFeatureGetBlockLocationLocallyBundled() {
-    return featureGetBlockLocationLocallyBundled;
-  }
-
-  @VisibleForTesting
-  protected boolean isFeatureConcurrentReadWithReadAhead() {
-    return featureConcurrentReadWithReadAhead;
-  }
-
-  @VisibleForTesting
-  protected boolean isFeatureRedirectOff() {
-    return featureRedirectOff;
-  }
-
-  @VisibleForTesting
-  protected boolean isOverrideOwnerFeatureOn() {
-    return overrideOwner;
-  }
-
-  @VisibleForTesting
-  protected int getMaxBufferSize() {
-    return maxBufferSize;
-  }
-
-  @VisibleForTesting
-  protected int getMaxConcurrentConnection() {
-    return maxConcurrentConnection;
-  }
-
-  @Override
-  public String getScheme() {
-    return SCHEME;
-  }
-
-  /**
-   * Constructing home directory locally is fine as long as Hadoop
-   * local user name and ADL user name relationship story is not fully baked
-   * yet.
-   *
-   * @return Hadoop local user home directory.
-   */
-  @Override
-  public final Path getHomeDirectory() {
-    try {
-      return makeQualified(new Path(
-          "/user/" + 
UserGroupInformation.getCurrentUser().getShortUserName()));
-    } catch (IOException e) {
-    }
-
-    return new Path("/user/" + userName);
-  }
-
-  /**
-   * Azure data lake does not support user configuration for data replication
-   * hence not leaving system to query on
-   * azure data lake.
-   *
-   * Stub implementation
-   *
-   * @param p           Not honoured
-   * @param replication Not honoured
-   * @return True hard coded since ADL file system does not support
-   * replication configuration
-   * @throws IOException No exception would not thrown in this case however
-   *                     aligning with parent api definition.
-   */
-  @Override
-  public final boolean setReplication(final Path p, final short replication)
-      throws IOException {
-    return true;
-  }
-
-  /**
-   * @param f File/Folder path
-   * @return FileStatus instance containing metadata information of f
-   * @throws IOException For any system error
-   */
-  @Override
-  public FileStatus getFileStatus(Path f) throws IOException {
-    statistics.incrementReadOps(1);
-    FileStatus status = super.getFileStatus(f);
-
-    if (overrideOwner) {
-      FileStatus proxiedStatus = new FileStatus(status.getLen(),
-          status.isDirectory(), status.getReplication(), status.getBlockSize(),
-          status.getModificationTime(), status.getAccessTime(),
-          status.getPermission(), userName, "hdfs", status.getPath());
-      return proxiedStatus;
-    } else {
-      return status;
-    }
-  }
-
-  /**
-   * Create call semantic is handled differently in case of ADL. Create
-   * semantics is translated to Create/Append
-   * semantics.
-   * 1. No dedicated connection to server.
-   * 2. Buffering is locally done, Once buffer is full or flush is invoked on
-   * the by the caller. All the pending
-   * data is pushed to ADL as APPEND operation code.
-   * 3. On close - Additional call is send to server to close the stream, and
-   * release lock from the stream.
-   *
-   * Necessity of Create/Append semantics is
-   * 1. ADL backend server does not allow idle connection for longer duration
-   * . In case of slow writer scenario,
-   * observed connection timeout/Connection reset causing occasional job
-   * failures.
-   * 2. Performance boost to jobs which are slow writer, avoided network 
latency
-   * 3. ADL equally better performing with multiple of 4MB chunk as append
-   * calls.
-   *
-   * @param f           File path
-   * @param permission  Access permission for the newly created file
-   * @param overwrite   Remove existing file and recreate new one if true
-   *                    otherwise throw error if file exist
-   * @param bufferSize  Buffer size, ADL backend does not honour
-   * @param replication Replication count, ADL backend does not honour
-   * @param blockSize   Block size, ADL backend does not honour
-   * @param progress    Progress indicator
-   * @return FSDataOutputStream OutputStream on which application can push
-   * stream of bytes
-   * @throws IOException when system error, internal server error or user error
-   */
-  @Override
-  public FSDataOutputStream create(final Path f, final FsPermission permission,
-      final boolean overwrite, final int bufferSize, final short replication,
-      final long blockSize, final Progressable progress) throws IOException {
-    statistics.incrementWriteOps(1);
-
-    return new FSDataOutputStream(new BatchAppendOutputStream(f, bufferSize,
-        new PermissionParam(applyUMask(permission)),
-        new OverwriteParam(overwrite), new BufferSizeParam(bufferSize),
-        new ReplicationParam(replication), new BlockSizeParam(blockSize),
-        new ADLVersionInfo(VersionInfo.getVersion())), statistics) {
-    };
-  }
-
-  @Override
-  public FSDataOutputStream createNonRecursive(final Path f,
-      final FsPermission permission, final EnumSet<CreateFlag> flag,
-      final int bufferSize, final short replication, final long blockSize,
-      final Progressable progress) throws IOException {
-    statistics.incrementWriteOps(1);
-
-    String leaseId = java.util.UUID.randomUUID().toString();
-    return new FSDataOutputStream(new BatchAppendOutputStream(f, bufferSize,
-        new PermissionParam(applyUMask(permission)), new CreateFlagParam(flag),
-        new CreateParentParam(false), new BufferSizeParam(bufferSize),
-        new ReplicationParam(replication), new LeaseParam(leaseId),
-        new BlockSizeParam(blockSize),
-        new ADLVersionInfo(VersionInfo.getVersion())), statistics) {
-    };
-  }
-
-  /**
-   * Since defined as private in parent class, redefined to pass through
-   * Create api implementation.
-   *
-   * @param permission
-   * @return FsPermission list
-   */
-  private FsPermission applyUMask(FsPermission permission) {
-    FsPermission fsPermission = permission;
-    if (fsPermission == null) {
-      fsPermission = FsPermission.getDefault();
-    }
-    return fsPermission.applyUMask(FsPermission.getUMask(getConf()));
-  }
-
-  /**
-   * Open call semantic is handled differently in case of ADL. Instead of
-   * network stream is returned to the user,
-   * Overridden FsInputStream is returned.
-   *
-   * 1. No dedicated connection to server.
-   * 2. Process level concurrent read ahead Buffering is done, This allows
-   * data to be available for caller quickly.
-   * 3. Number of byte to read ahead is configurable.
-   *
-   * Advantage of Process level concurrent read ahead Buffering semantics is
-   * 1. ADL backend server does not allow idle connection for longer duration
-   * . In case of slow reader scenario,
-   * observed connection timeout/Connection reset causing occasional job
-   * failures.
-   * 2. Performance boost to jobs which are slow reader, avoided network 
latency
-   * 3. Compressed format support like ORC, and large data files gains the
-   * most out of this implementation.
-   *
-   * Read ahead feature is configurable.
-   *
-   * @param f          File path
-   * @param buffersize Buffer size
-   * @return FSDataInputStream InputStream on which application can read
-   * stream of bytes
-   * @throws IOException when system error, internal server error or user error
-   */
-  @Override
-  public FSDataInputStream open(final Path f, final int buffersize)
-      throws IOException {
-    statistics.incrementReadOps(1);
-
-    final HttpOpParam.Op op = GetOpParam.Op.OPEN;
-    // use a runner so the open can recover from an invalid token
-    FsPathConnectionRunner runner = null;
-
-    if (featureConcurrentReadWithReadAhead) {
-      URL url = this.toUrl(op, f, new BufferSizeParam(buffersize),
-          new ReadADLNoRedirectParam(true),
-          new ADLVersionInfo(VersionInfo.getVersion()));
-
-      BatchByteArrayInputStream bb = new BatchByteArrayInputStream(url, f,
-          maxBufferSize, maxConcurrentConnection);
-
-      FSDataInputStream fin = new FSDataInputStream(bb);
-      return fin;
-    } else {
-      if (featureRedirectOff) {
-        runner = new FsPathConnectionRunner(ADLGetOpParam.Op.OPEN, f,
-            new BufferSizeParam(buffersize), new ReadADLNoRedirectParam(true),
-            new ADLVersionInfo(VersionInfo.getVersion()));
-      } else {
-        runner = new FsPathConnectionRunner(op, f,
-            new BufferSizeParam(buffersize));
-      }
-
-      return new FSDataInputStream(
-          new OffsetUrlInputStream(new UnresolvedUrlOpener(runner),
-              new OffsetUrlOpener(null)));
-    }
-  }
-
-  /**
-   * @param f File/Folder path
-   * @return FileStatus array list
-   * @throws IOException For system error
-   */
-  @Override
-  public FileStatus[] listStatus(final Path f) throws IOException {
-    FileStatus[] fileStatuses = super.listStatus(f);
-    for (int i = 0; i < fileStatuses.length; i++) {
-      if (overrideOwner) {
-        fileStatuses[i] = new FileStatus(fileStatuses[i].getLen(),
-            fileStatuses[i].isDirectory(), fileStatuses[i].getReplication(),
-            fileStatuses[i].getBlockSize(),
-            fileStatuses[i].getModificationTime(),
-            fileStatuses[i].getAccessTime(), fileStatuses[i].getPermission(),
-            userName, "hdfs", fileStatuses[i].getPath());
-      }
-    }
-    return fileStatuses;
-  }
-
-  @Override
-  public BlockLocation[] getFileBlockLocations(final FileStatus status,
-      final long offset, final long length) throws IOException {
-    if (status == null) {
-      return null;
-    }
-
-    if (featureGetBlockLocationLocallyBundled) {
-      if ((offset < 0) || (length < 0)) {
-        throw new IllegalArgumentException("Invalid start or len parameter");
-      }
-
-      if (status.getLen() < offset) {
-        return new BlockLocation[0];
-      }
-
-      final String[] name = {"localhost"};
-      final String[] host = {"localhost"};
-      long blockSize = ADLConfKeys.DEFAULT_EXTENT_SIZE; // Block size must be
-      // non zero
-      int numberOfLocations =
-          (int) (length / blockSize) + ((length % blockSize == 0) ? 0 : 1);
-      BlockLocation[] locations = new BlockLocation[numberOfLocations];
-      for (int i = 0; i < locations.length; i++) {
-        long currentOffset = offset + (i * blockSize);
-        long currentLength = Math
-            .min(blockSize, offset + length - currentOffset);
-        locations[i] = new BlockLocation(name, host, currentOffset,
-            currentLength);
-      }
-
-      return locations;
-    } else {
-      return getFileBlockLocations(status.getPath(), offset, length);
-    }
-  }
-
-  @Override
-  public BlockLocation[] getFileBlockLocations(final Path p, final long offset,
-      final long length) throws IOException {
-    statistics.incrementReadOps(1);
-
-    if (featureGetBlockLocationLocallyBundled) {
-      FileStatus fileStatus = getFileStatus(p);
-      return getFileBlockLocations(fileStatus, offset, length);
-    } else {
-      return super.getFileBlockLocations(p, offset, length);
-    }
-  }
-
-  enum StreamState {
-    Initial,
-    DataCachedInLocalBuffer,
-    StreamEnd
-  }
-
-  class BatchAppendOutputStream extends OutputStream {
-    private Path fsPath;
-    private Param<?, ?>[] parameters;
-    private byte[] data = null;
-    private int offset = 0;
-    private long length = 0;
-    private boolean eof = false;
-    private boolean hadError = false;
-    private byte[] dataBuffers = null;
-    private int bufSize = 0;
-    private boolean streamClosed = false;
-
-    public BatchAppendOutputStream(Path path, int bufferSize,
-        Param<?, ?>... param) throws IOException {
-      if (bufferSize < (ADLConfKeys.DEFAULT_BLOCK_SIZE)) {
-        bufSize = ADLConfKeys.DEFAULT_BLOCK_SIZE;
-      } else {
-        bufSize = bufferSize;
-      }
-
-      this.fsPath = path;
-      this.parameters = param;
-      this.data = getBuffer();
-      FSDataOutputStream createStream = null;
-      try {
-        if (featureRedirectOff) {
-          CreateADLNoRedirectParam skipRedirect = new CreateADLNoRedirectParam(
-              true);
-          Param<?, ?>[] tmpParam = featureFlushWhenEOF ?
-              new Param<?, ?>[param.length + 2] :
-              new Param<?, ?>[param.length + 1];
-          System.arraycopy(param, 0, tmpParam, 0, param.length);
-          tmpParam[param.length] = skipRedirect;
-          if (featureFlushWhenEOF) {
-            tmpParam[param.length + 1] = new ADLFlush(false);
-          }
-          createStream = new FsPathOutputStreamRunner(ADLPutOpParam.Op.CREATE,
-              fsPath, 1, tmpParam).run();
-        } else {
-          createStream = new FsPathOutputStreamRunner(PutOpParam.Op.CREATE,
-              fsPath, 1, param).run();
-        }
-      } finally {
-        if (createStream != null) {
-          createStream.close();
-        }
-      }
-    }
-
-    @Override
-    public final synchronized void write(int b) throws IOException {
-      if (streamClosed) {
-        throw new IOException(fsPath + " stream object is closed.");
-      }
-
-      if (offset == (data.length)) {
-        flush();
-      }
-
-      data[offset] = (byte) b;
-      offset++;
-
-      // Statistics will get incremented again as part of the batch updates,
-      // decrement here to avoid double value
-      if (statistics != null) {
-        statistics.incrementBytesWritten(-1);
-      }
-    }
-
-    @Override
-    public final synchronized void write(byte[] buf, int off, int len)
-        throws IOException {
-      if (streamClosed) {
-        throw new IOException(fsPath + " stream object is closed.");
-      }
-
-      int bytesToWrite = len;
-      int localOff = off;
-      int localLen = len;
-      if (localLen >= data.length) {
-        // Flush data that is already in our internal buffer
-        flush();
-
-        // Keep committing data until we have less than our internal buffers
-        // length left
-        do {
-          try {
-            commit(buf, localOff, data.length, eof);
-          } catch (IOException e) {
-            hadError = true;
-            throw e;
-          }
-          localOff += data.length;
-          localLen -= data.length;
-        } while (localLen >= data.length);
-      }
-
-      // At this point, we have less than data.length left to copy from users
-      // buffer
-      if (offset + localLen >= data.length) {
-        // Users buffer has enough data left to fill our internal buffer
-        int bytesToCopy = data.length - offset;
-        System.arraycopy(buf, localOff, data, offset, bytesToCopy);
-        offset += bytesToCopy;
-
-        // Flush our internal buffer
-        flush();
-        localOff += bytesToCopy;
-        localLen -= bytesToCopy;
-      }
-
-      if (localLen > 0) {
-        // Simply copy the remainder from the users buffer into our internal
-        // buffer
-        System.arraycopy(buf, localOff, data, offset, localLen);
-        offset += localLen;
-      }
-
-      // Statistics will get incremented again as part of the batch updates,
-      // decrement here to avoid double value
-      if (statistics != null) {
-        statistics.incrementBytesWritten(-bytesToWrite);
-      }
-    }
-
-    @Override
-    public final synchronized void flush() throws IOException {
-      if (streamClosed) {
-        throw new IOException(fsPath + " stream object is closed.");
-      }
-
-      if (offset > 0) {
-        try {
-          commit(data, 0, offset, eof);
-        } catch (IOException e) {
-          hadError = true;
-          throw e;
-        }
-      }
-
-      offset = 0;
-    }
-
-    @Override
-    public final synchronized void close() throws IOException {
-      // Stream is closed earlier, return quietly.
-      if(streamClosed) {
-        return;
-      }
-
-      if (featureRedirectOff) {
-        eof = true;
-      }
-
-      boolean flushedSomething = false;
-      if (hadError) {
-        // No point proceeding further since the error has occurred and
-        // stream would be required to upload again.
-        streamClosed = true;
-        return;
-      } else {
-        flushedSomething = offset > 0;
-        try {
-          flush();
-        } finally {
-          streamClosed = true;
-        }
-      }
-
-      if (featureRedirectOff) {
-        // If we didn't flush anything from our internal buffer, we have to
-        // call the service again
-        // with an empty payload and flush=true in the url
-        if (!flushedSomething) {
-          try {
-            commit(null, 0, ADLConfKeys.KB, true);
-          } finally {
-            streamClosed = true;
-          }
-        }
-      }
-    }
-
-    private void commit(byte[] buffer, int off, int len, boolean endOfFile)
-        throws IOException {
-      OutputStream out = null;
-      try {
-        if (featureRedirectOff) {
-          AppendADLNoRedirectParam skipRedirect = new AppendADLNoRedirectParam(
-              true);
-          Param<?, ?>[] tmpParam = featureFlushWhenEOF ?
-              new Param<?, ?>[parameters.length + 3] :
-              new Param<?, ?>[parameters.length + 1];
-          System.arraycopy(parameters, 0, tmpParam, 0, parameters.length);
-          tmpParam[parameters.length] = skipRedirect;
-          if (featureFlushWhenEOF) {
-            tmpParam[parameters.length + 1] = new ADLFlush(endOfFile);
-            tmpParam[parameters.length + 2] = new OffsetParam(length);
-          }
-
-          out = new FsPathOutputStreamRunner(ADLPostOpParam.Op.APPEND, fsPath,
-              len, tmpParam).run();
-        } else {
-          out = new FsPathOutputStreamRunner(ADLPostOpParam.Op.APPEND, fsPath,
-              len, parameters).run();
-        }
-
-        if (buffer != null) {
-          out.write(buffer, off, len);
-          length += len;
-        }
-      } finally {
-        if (out != null) {
-          out.close();
-        }
-      }
-    }
-
-    private byte[] getBuffer() {
-      // Switch between the first and second buffer
-      dataBuffers = new byte[bufSize];
-      return dataBuffers;
-    }
-  }
-
-  /**
-   * Read data from backend in chunks instead of persistent connection. This
-   * is to avoid slow reader causing socket
-   * timeout.
-   */
-  protected class BatchByteArrayInputStream extends FSInputStream {
-
-    private static final int SIZE4MB = 4 * 1024 * 1024;
-    private final URL runner;
-    private byte[] data = null;
-    private long validDataHoldingSize = 0;
-    private int bufferOffset = 0;
-    private long currentFileOffset = 0;
-    private long nextFileOffset = 0;
-    private long fileSize = 0;
-    private StreamState state = StreamState.Initial;
-    private int maxBufferSize;
-    private int maxConcurrentConnection;
-    private Path fsPath;
-    private boolean streamIsClosed;
-    private Future[] subtasks = null;
-
-    BatchByteArrayInputStream(URL url, Path p, int bufferSize,
-        int concurrentConnection) throws IOException {
-      this.runner = url;
-      fsPath = p;
-      FileStatus fStatus = getFileStatus(fsPath);
-      if (!fStatus.isFile()) {
-        throw new IOException("Cannot open the directory " + p + " for " +
-            "reading");
-      }
-      fileSize = fStatus.getLen();
-      this.maxBufferSize = bufferSize;
-      this.maxConcurrentConnection = concurrentConnection;
-      this.streamIsClosed = false;
-    }
-
-    @Override
-    public synchronized final int read(long position, byte[] buffer, int 
offset,
-        int length) throws IOException {
-      if (streamIsClosed) {
-        throw new IOException("Stream already closed");
-      }
-      long oldPos = this.getPos();
-
-      int nread1;
-      try {
-        this.seek(position);
-        nread1 = this.read(buffer, offset, length);
-      } finally {
-        this.seek(oldPos);
-      }
-
-      return nread1;
-    }
-
-    @Override
-    public synchronized final int read() throws IOException {
-      if (streamIsClosed) {
-        throw new IOException("Stream already closed");
-      }
-      int status = doBufferAvailabilityCheck();
-      if (status == -1) {
-        return status;
-      }
-      int ch = data[bufferOffset++] & (0xff);
-      if (statistics != null) {
-        statistics.incrementBytesRead(1);
-      }
-      return ch;
-    }
-
-    @Override
-    public synchronized final void readFully(long position, byte[] buffer,
-        int offset, int length) throws IOException {
-      if (streamIsClosed) {
-        throw new IOException("Stream already closed");
-      }
-
-      super.readFully(position, buffer, offset, length);
-      if (statistics != null) {
-        statistics.incrementBytesRead(length);
-      }
-    }
-
-    @Override
-    public synchronized final int read(byte[] b, int off, int len)
-        throws IOException {
-      if (b == null) {
-        throw new IllegalArgumentException();
-      } else if (off < 0 || len < 0 || len > b.length - off) {
-        throw new IndexOutOfBoundsException();
-      } else if (len == 0) {
-        return 0;
-      }
-
-      if (streamIsClosed) {
-        throw new IOException("Stream already closed");
-      }
-      int status = doBufferAvailabilityCheck();
-      if (status == -1) {
-        return status;
-      }
-
-      int byteRead = 0;
-      long availableBytes = validDataHoldingSize - off;
-      long requestedBytes = bufferOffset + len - off;
-      if (requestedBytes <= availableBytes) {
-        System.arraycopy(data, bufferOffset, b, off, len);
-        bufferOffset += len;
-        byteRead = len;
-      } else {
-        byteRead = super.read(b, off, len);
-      }
-
-      if (statistics != null) {
-        statistics.incrementBytesRead(byteRead);
-      }
-
-      return byteRead;
-    }
-
-    private int doBufferAvailabilityCheck() throws IOException {
-      if (state == StreamState.Initial) {
-        validDataHoldingSize = fill(nextFileOffset);
-      }
-
-      long dataReloadSize = 0;
-      switch ((int) validDataHoldingSize) {
-      case -1:
-        state = StreamState.StreamEnd;
-        return -1;
-      case 0:
-        dataReloadSize = fill(nextFileOffset);
-        if (dataReloadSize <= 0) {
-          state = StreamState.StreamEnd;
-          return (int) dataReloadSize;
-        } else {
-          validDataHoldingSize = dataReloadSize;
-        }
-        break;
-      default:
-        break;
-      }
-
-      if (bufferOffset >= validDataHoldingSize) {
-        dataReloadSize = fill(nextFileOffset);
-      }
-
-      if (bufferOffset >= ((dataReloadSize == 0) ?
-          validDataHoldingSize :
-          dataReloadSize)) {
-        state = StreamState.StreamEnd;
-        return -1;
-      }
-
-      validDataHoldingSize = ((dataReloadSize == 0) ?
-          validDataHoldingSize :
-          dataReloadSize);
-      state = StreamState.DataCachedInLocalBuffer;
-      return 0;
-    }
-
-    private long fill(final long off) throws IOException {
-      if (state == StreamState.StreamEnd) {
-        return -1;
-      }
-
-      if (fileSize <= off) {
-        state = StreamState.StreamEnd;
-        return -1;
-      }
-      int len = maxBufferSize;
-      long fileOffset = 0;
-      boolean isEntireFileCached = true;
-      if ((fileSize <= maxBufferSize)) {
-        len = (int) fileSize;
-        currentFileOffset = 0;
-        nextFileOffset = 0;
-      } else {
-        if (len > (fileSize - off)) {
-          len = (int) (fileSize - off);
-        }
-
-        synchronized (BufferManager.getLock()) {
-          if (BufferManager.getInstance()
-              .hasValidDataForOffset(fsPath.toString(), off)) {
-            len = (int) (
-                BufferManager.getInstance().getBufferOffset() + BufferManager
-                    .getInstance().getBufferSize() - (int) off);
-          }
-        }
-
-        if (len <= 0) {
-          len = maxBufferSize;
-        }
-        fileOffset = off;
-        isEntireFileCached = false;
-      }
-
-      data = null;
-      BufferManager bm = BufferManager.getInstance();
-      data = bm.getEmpty(len);
-      boolean fetchDataOverNetwork = false;
-      synchronized (BufferManager.getLock()) {
-        if (bm.hasData(fsPath.toString(), fileOffset, len)) {
-          try {
-            bm.get(data, fileOffset);
-            validDataHoldingSize = data.length;
-            currentFileOffset = fileOffset;
-          } catch (ArrayIndexOutOfBoundsException e) {
-            fetchDataOverNetwork = true;
-          }
-        } else {
-          fetchDataOverNetwork = true;
-        }
-      }
-
-      if (fetchDataOverNetwork) {
-        int splitSize = getSplitSize(len);
-        try {
-          validDataHoldingSize = fillDataConcurrently(data, len, fileOffset,
-              splitSize);
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new IOException("Interrupted filling buffer", e);
-        }
-
-        synchronized (BufferManager.getLock()) {
-          bm.add(data, fsPath.toString(), fileOffset);
-        }
-        currentFileOffset = nextFileOffset;
-      }
-
-      nextFileOffset += validDataHoldingSize;
-      state = StreamState.DataCachedInLocalBuffer;
-      bufferOffset = isEntireFileCached ? (int) off : 0;
-      return validDataHoldingSize;
-    }
-
-    int getSplitSize(int size) {
-      if (size <= SIZE4MB) {
-        return 1;
-      }
-
-      // Not practical
-      if (size > maxBufferSize) {
-        size = maxBufferSize;
-      }
-
-      int equalBufferSplit = Math.max(size / SIZE4MB, 1);
-      int splitSize = Math.min(equalBufferSplit, maxConcurrentConnection);
-      return splitSize;
-    }
-
-    @Override
-    public synchronized final void seek(long pos) throws IOException {
-      if (pos == -1) {
-        throw new IOException("Bad offset, cannot seek to " + pos);
-      }
-
-      BufferManager bm = BufferManager.getInstance();
-      synchronized (BufferManager.getLock()) {
-        if (bm.hasValidDataForOffset(fsPath.toString(), pos)) {
-          state = StreamState.DataCachedInLocalBuffer;
-        } else if (pos >= 0) {
-          state = StreamState.Initial;
-        }
-      }
-
-      long availableBytes = (currentFileOffset + validDataHoldingSize);
-
-      // Check if this position falls under buffered data
-      if (pos < currentFileOffset || availableBytes <= 0) {
-        validDataHoldingSize = 0;
-        currentFileOffset = pos;
-        nextFileOffset = pos;
-        bufferOffset = 0;
-        return;
-      }
-
-      if (pos < availableBytes && pos >= currentFileOffset) {
-        state = StreamState.DataCachedInLocalBuffer;
-        bufferOffset = (int) (pos - currentFileOffset);
-      } else {
-        validDataHoldingSize = 0;
-        currentFileOffset = pos;
-        nextFileOffset = pos;
-        bufferOffset = 0;
-      }
-    }
-
-    @Override
-    public synchronized final long getPos() throws IOException {
-      if (streamIsClosed) {
-        throw new IOException("Stream already closed");
-      }
-      return bufferOffset + currentFileOffset;
-    }
-
-    @Override
-    public synchronized final int available() throws IOException {
-      if (streamIsClosed) {
-        throw new IOException("Stream already closed");
-      }
-      return Integer.MAX_VALUE;
-    }
-
-    @Override
-    public final boolean seekToNewSource(long targetPos) throws IOException {
-      return false;
-    }
-
-    @SuppressWarnings("unchecked")
-    private int fillDataConcurrently(byte[] byteArray, int length,
-        long globalOffset, int splitSize)
-        throws IOException, InterruptedException {
-      ExecutorService executor = Executors.newFixedThreadPool(splitSize);
-      subtasks = new Future[splitSize];
-      for (int i = 0; i < splitSize; i++) {
-        int offset = i * (length / splitSize);
-        int splitLength = (splitSize == (i + 1)) ?
-            (length / splitSize) + (length % splitSize) :
-            (length / splitSize);
-        subtasks[i] = executor.submit(
-            new BackgroundReadThread(byteArray, offset, splitLength,
-                globalOffset + offset));
-      }
-
-      executor.shutdown();
-      // wait until all tasks are finished
-      executor.awaitTermination(ADLConfKeys.DEFAULT_TIMEOUT_IN_SECONDS,
-          TimeUnit.SECONDS);
-
-      int totalBytePainted = 0;
-      for (int i = 0; i < splitSize; ++i) {
-        try {
-          totalBytePainted += (Integer) subtasks[i].get();
-        } catch (InterruptedException e) {
-          Thread.currentThread().interrupt();
-          throw new IOException(e.getCause());
-        } catch (ExecutionException e) {
-          Thread.currentThread().interrupt();
-          throw new IOException(e.getCause());
-        }
-      }
-
-      if (totalBytePainted != length) {
-        throw new IOException("Expected " + length + " bytes, Got " +
-            totalBytePainted + " bytes");
-      }
-
-      return totalBytePainted;
-    }
-
-    @Override
-    public synchronized final void close() throws IOException {
-      synchronized (BufferManager.getLock()) {
-        BufferManager.getInstance().clear();
-      }
-      //need to cleanup the above code the stream and connection close doesn't
-      // happen here
-      //flag set to mark close happened, cannot use the stream once closed
-      streamIsClosed = true;
-    }
-
-    /**
-     * Reads data from the ADL backend from the specified global offset and
-     * given
-     * length. Read data from ADL backend is copied to buffer array from the
-     * offset value specified.
-     *
-     * @param buffer       Store read data from ADL backend in the buffer.
-     * @param offset       Store read data from ADL backend in the buffer
-     *                     from the
-     *                     offset.
-     * @param length       Size of the data read from the ADL backend.
-     * @param globalOffset Read data from file offset.
-     * @return Number of bytes read from the ADL backend
-     * @throws IOException For any intermittent server issues or internal
-     *                     failures.
-     */
-    private int fillUpData(byte[] buffer, int offset, int length,
-        long globalOffset) throws IOException {
-      int totalBytesRead = 0;
-      final URL offsetUrl = new URL(
-          runner + "&" + new OffsetParam(String.valueOf(globalOffset)) + "&"
-              + new LengthParam(String.valueOf(length)));
-      HttpURLConnection conn = new URLRunner(GetOpParam.Op.OPEN, offsetUrl,
-          true).run();
-      InputStream in = conn.getInputStream();
-      try {
-        int bytesRead = 0;
-        while ((bytesRead = in.read(buffer, (int) offset + totalBytesRead,
-            (int) (length - totalBytesRead))) > 0) {
-          totalBytesRead += bytesRead;
-        }
-
-        // InputStream must be fully consumed to enable http keep-alive
-        if (bytesRead == 0) {
-          // Looking for EOF marker byte needs to be read.
-          if (in.read() != -1) {
-            throw new SocketException(
-                "Server returned more than requested data.");
-          }
-        }
-      } finally {
-        in.close();
-        conn.disconnect();
-      }
-
-      return totalBytesRead;
-    }
-
-    private class BackgroundReadThread implements Callable {
-
-      private final byte[] data;
-      private int offset;
-      private int length;
-      private long globalOffset;
-
-      BackgroundReadThread(byte[] buffer, int off, int size, long position) {
-        this.data = buffer;
-        this.offset = off;
-        this.length = size;
-        this.globalOffset = position;
-      }
-
-      public Object call() throws IOException {
-        return fillUpData(data, offset, length, globalOffset);
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AzureADClientCredentialBasedAccesTokenProvider.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AzureADClientCredentialBasedAccesTokenProvider.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AzureADClientCredentialBasedAccesTokenProvider.java
deleted file mode 100644
index 11d07e7..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/AzureADClientCredentialBasedAccesTokenProvider.java
+++ /dev/null
@@ -1,156 +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.web.oauth2;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-import com.fasterxml.jackson.databind.ObjectReader;
-import com.squareup.okhttp.OkHttpClient;
-import com.squareup.okhttp.Request;
-import com.squareup.okhttp.RequestBody;
-import com.squareup.okhttp.Response;
-import com.squareup.okhttp.MediaType;
-import org.apache.hadoop.classification.InterfaceAudience;
-import org.apache.hadoop.classification.InterfaceStability;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.hdfs.web.URLConnectionFactory;
-import org.apache.hadoop.util.Timer;
-import org.apache.http.HttpStatus;
-
-import java.io.IOException;
-import java.util.Map;
-import java.util.concurrent.TimeUnit;
-
-import static org.apache.hadoop.hdfs.web.oauth2.Utils.notNull;
-
-
-/**
- * Obtain an access token via the credential-based OAuth2 workflow.
- */
-@InterfaceAudience.Public
-@InterfaceStability.Evolving
-public class AzureADClientCredentialBasedAccesTokenProvider
-    extends AccessTokenProvider {
-  private static final ObjectReader READER =
-      new ObjectMapper().reader(Map.class);
-
-  public static final String OAUTH_CREDENTIAL_KEY
-      = "dfs.webhdfs.oauth2.credential";
-
-  public static final String AAD_RESOURCE_KEY
-      = "fs.adls.oauth2.resource";
-
-  public static final String RESOURCE_PARAM_NAME
-      = "resource";
-
-  private static final String OAUTH_CLIENT_ID_KEY
-      = "dfs.webhdfs.oauth2.client.id";
-
-  private static final String OAUTH_REFRESH_URL_KEY
-      = "dfs.webhdfs.oauth2.refresh.url";
-
-
-  public static final String ACCESS_TOKEN = "access_token";
-  public static final String CLIENT_CREDENTIALS = "client_credentials";
-  public static final String CLIENT_ID = "client_id";
-  public static final String CLIENT_SECRET = "client_secret";
-  public static final String EXPIRES_IN = "expires_in";
-  public static final String GRANT_TYPE = "grant_type";
-  public static final MediaType URLENCODED
-          = MediaType.parse("application/x-www-form-urlencoded; 
charset=utf-8");
-
-
-  private AccessTokenTimer timer;
-
-  private String clientId;
-
-  private String refreshURL;
-
-  private String accessToken;
-
-  private String resource;
-
-  private String credential;
-
-  private boolean initialCredentialObtained = false;
-
-  AzureADClientCredentialBasedAccesTokenProvider() {
-    this.timer = new AccessTokenTimer();
-  }
-
-  AzureADClientCredentialBasedAccesTokenProvider(Timer timer) {
-    this.timer = new AccessTokenTimer(timer);
-  }
-
-  @Override
-  public void setConf(Configuration conf) {
-    super.setConf(conf);
-    clientId = notNull(conf, OAUTH_CLIENT_ID_KEY);
-    refreshURL = notNull(conf, OAUTH_REFRESH_URL_KEY);
-    resource = notNull(conf, AAD_RESOURCE_KEY);
-    credential = notNull(conf, OAUTH_CREDENTIAL_KEY);
-  }
-
-  @Override
-  public String getAccessToken() throws IOException {
-    if(timer.shouldRefresh() || !initialCredentialObtained) {
-      refresh();
-      initialCredentialObtained = true;
-    }
-    return accessToken;
-  }
-
-  void refresh() throws IOException {
-    try {
-      OkHttpClient client = new OkHttpClient();
-      client.setConnectTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT,
-          TimeUnit.MILLISECONDS);
-      client.setReadTimeout(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT,
-          TimeUnit.MILLISECONDS);
-
-      String bodyString = Utils.postBody(CLIENT_SECRET, credential,
-          GRANT_TYPE, CLIENT_CREDENTIALS,
-          RESOURCE_PARAM_NAME, resource,
-          CLIENT_ID, clientId);
-
-      RequestBody body = RequestBody.create(URLENCODED, bodyString);
-
-      Request request = new Request.Builder()
-          .url(refreshURL)
-          .post(body)
-          .build();
-      Response responseBody = client.newCall(request).execute();
-
-      if (responseBody.code() != HttpStatus.SC_OK) {
-        throw new IllegalArgumentException("Received invalid http response: "
-            + responseBody.code() + ", text = " + responseBody.toString());
-      }
-
-      Map<?, ?> response = READER.readValue(responseBody.body().string());
-
-      String newExpiresIn = response.get(EXPIRES_IN).toString();
-      timer.setExpiresIn(newExpiresIn);
-
-      accessToken = response.get(ACCESS_TOKEN).toString();
-
-    } catch (Exception e) {
-      throw new IOException("Unable to obtain access token from credential", 
e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/PrivateCachedRefreshTokenBasedAccessTokenProvider.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/PrivateCachedRefreshTokenBasedAccessTokenProvider.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/PrivateCachedRefreshTokenBasedAccessTokenProvider.java
deleted file mode 100644
index d7dce25..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/PrivateCachedRefreshTokenBasedAccessTokenProvider.java
+++ /dev/null
@@ -1,37 +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.web.oauth2;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-
-/**
- * Exposing AccessTokenProvider publicly to extend in com.microsoft.azure
- * .datalake package. Extended version to cache
- * token for the process to gain performance gain.
- */
-@Private
-@Unstable
-public abstract class PrivateCachedRefreshTokenBasedAccessTokenProvider
-    extends AccessTokenProvider {
-
-  // visibility workaround
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
deleted file mode 100644
index 7a9dffa..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/oauth2/package-info.java
+++ /dev/null
@@ -1,24 +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.
- *
- */
-
-/**
- * A distributed implementation of {@link
- * org.apache.hadoop.hdfs.web.oauth2} for oauth2 token management support.
- */
-package org.apache.hadoop.hdfs.web.oauth2;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/package-info.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/package-info.java
deleted file mode 100644
index 1cc8273..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/package-info.java
+++ /dev/null
@@ -1,25 +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.
- *
- */
-
-/**
- * A distributed implementation of {@link org.apache.hadoop.hdfs.web} for
- * reading and writing files on Azure data lake file system. This
- * implementation is derivation from the webhdfs specification.
- */
-package org.apache.hadoop.hdfs.web;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLFlush.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLFlush.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLFlush.java
deleted file mode 100644
index b76aaaa..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLFlush.java
+++ /dev/null
@@ -1,49 +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.web.resources;
-
-/**
- * Query parameter to notify backend server that the all the data has been
- * pushed to over the stream.
- *
- * Used in operation code Create and Append.
- */
-public class ADLFlush extends BooleanParam {
-  /**
-   * Parameter name.
-   */
-  public static final String NAME = "flush";
-
-  private static final Domain DOMAIN = new Domain(NAME);
-
-  /**
-   * Constructor.
-   *
-   * @param value the parameter value.
-   */
-  public ADLFlush(final Boolean value) {
-    super(DOMAIN, value);
-  }
-
-  @Override
-  public final String getName() {
-    return NAME;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLGetOpParam.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLGetOpParam.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLGetOpParam.java
deleted file mode 100644
index 6b3708f..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLGetOpParam.java
+++ /dev/null
@@ -1,96 +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.web.resources;
-
-import java.net.HttpURLConnection;
-
-/**
- * Extended Webhdfs GetOpParam to avoid redirect operation for azure data
- * lake storage.
- */
-public class ADLGetOpParam extends HttpOpParam<ADLGetOpParam.Op> {
-  private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
-
-  /**
-   * Constructor.
-   *
-   * @param str a string representation of the parameter value.
-   */
-  public ADLGetOpParam(final String str) {
-    super(DOMAIN, DOMAIN.parse(str));
-  }
-
-  @Override
-  public final String getName() {
-    return NAME;
-  }
-
-  /**
-   * Get operations.
-   */
-  public static enum Op implements HttpOpParam.Op {
-    OPEN(false, HttpURLConnection.HTTP_OK);
-
-    private final boolean redirect;
-    private final int expectedHttpResponseCode;
-    private final boolean requireAuth;
-
-    Op(final boolean doRedirect, final int expectHttpResponseCode) {
-      this(doRedirect, expectHttpResponseCode, false);
-    }
-
-    Op(final boolean doRedirect, final int expectHttpResponseCode,
-        final boolean doRequireAuth) {
-      this.redirect = doRedirect;
-      this.expectedHttpResponseCode = expectHttpResponseCode;
-      this.requireAuth = doRequireAuth;
-    }
-
-    @Override
-    public HttpOpParam.Type getType() {
-      return HttpOpParam.Type.GET;
-    }
-
-    @Override
-    public boolean getRequireAuth() {
-      return requireAuth;
-    }
-
-    @Override
-    public boolean getDoOutput() {
-      return false;
-    }
-
-    @Override
-    public boolean getRedirect() {
-      return redirect;
-    }
-
-    @Override
-    public int getExpectedHttpResponseCode() {
-      return expectedHttpResponseCode;
-    }
-
-    @Override
-    public String toQueryString() {
-      return NAME + "=" + this;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPostOpParam.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPostOpParam.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPostOpParam.java
deleted file mode 100644
index 7f7e749..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPostOpParam.java
+++ /dev/null
@@ -1,97 +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.web.resources;
-
-import java.net.HttpURLConnection;
-
-/**
- * Extended Webhdfs PostOpParam to avoid redirect during append operation for
- * azure data lake storage.
- */
-
-public class ADLPostOpParam extends HttpOpParam<ADLPostOpParam.Op> {
-  private static final Domain<Op> DOMAIN = new Domain<ADLPostOpParam.Op>(NAME,
-      Op.class);
-
-  /**
-   * Constructor.
-   *
-   * @param str a string representation of the parameter value.
-   */
-  public ADLPostOpParam(final String str) {
-    super(DOMAIN, DOMAIN.parse(str));
-  }
-
-  @Override
-  public final String getName() {
-    return NAME;
-  }
-
-  /**
-   * Post operations.
-   */
-  public static enum Op implements HttpOpParam.Op {
-    APPEND(true, false, HttpURLConnection.HTTP_OK);
-
-    private final boolean redirect;
-    private final boolean doOutput;
-    private final int expectedHttpResponseCode;
-
-    Op(final boolean doOut, final boolean doRedirect,
-        final int expectHttpResponseCode) {
-      this.doOutput = doOut;
-      this.redirect = doRedirect;
-      this.expectedHttpResponseCode = expectHttpResponseCode;
-    }
-
-    @Override
-    public Type getType() {
-      return Type.POST;
-    }
-
-    @Override
-    public boolean getRequireAuth() {
-      return false;
-    }
-
-    @Override
-    public boolean getDoOutput() {
-      return doOutput;
-    }
-
-    @Override
-    public boolean getRedirect() {
-      return redirect;
-    }
-
-    @Override
-    public int getExpectedHttpResponseCode() {
-      return expectedHttpResponseCode;
-    }
-
-    /**
-     * @return a URI query string.
-     */
-    @Override
-    public String toQueryString() {
-      return NAME + "=" + this;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPutOpParam.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPutOpParam.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPutOpParam.java
deleted file mode 100644
index d300a1c..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLPutOpParam.java
+++ /dev/null
@@ -1,94 +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.web.resources;
-
-import java.net.HttpURLConnection;
-
-/**
- * Extended Webhdfs PutOpParam to avoid redirect during Create operation for
- * azure data lake storage.
- */
-public class ADLPutOpParam extends HttpOpParam<ADLPutOpParam.Op> {
-  private static final Domain<Op> DOMAIN = new Domain<Op>(NAME, Op.class);
-
-  /**
-   * Constructor.
-   *
-   * @param str a string representation of the parameter value.
-   */
-  public ADLPutOpParam(final String str) {
-    super(DOMAIN, DOMAIN.parse(str));
-  }
-
-  @Override
-  public final String getName() {
-    return NAME;
-  }
-
-  /**
-   * Put operations.
-   */
-  public static enum Op implements HttpOpParam.Op {
-    CREATE(true, false, HttpURLConnection.HTTP_CREATED);
-
-    private final boolean redirect;
-    private final boolean doOutput;
-    private final int expectedHttpResponseCode;
-    private final boolean requireAuth;
-
-    Op(final boolean doOut, final boolean doRedirect,
-        final int expectHttpResponseCode) {
-      this.doOutput = doOut;
-      this.redirect = doRedirect;
-      this.expectedHttpResponseCode = expectHttpResponseCode;
-      this.requireAuth = false;
-    }
-
-    @Override
-    public HttpOpParam.Type getType() {
-      return HttpOpParam.Type.PUT;
-    }
-
-    @Override
-    public boolean getRequireAuth() {
-      return requireAuth;
-    }
-
-    @Override
-    public boolean getDoOutput() {
-      return doOutput;
-    }
-
-    @Override
-    public boolean getRedirect() {
-      return redirect;
-    }
-
-    @Override
-    public int getExpectedHttpResponseCode() {
-      return expectedHttpResponseCode;
-    }
-
-    @Override
-    public String toQueryString() {
-      return NAME + "=" + this;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLVersionInfo.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLVersionInfo.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLVersionInfo.java
deleted file mode 100644
index 0bfe521..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ADLVersionInfo.java
+++ /dev/null
@@ -1,51 +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.web.resources;
-
-import org.apache.hadoop.hdfs.web.ADLConfKeys;
-
-import java.util.regex.Pattern;
-
-/**
- * Capture ADL Jar version information. Require for debugging and analysis
- * purpose in the backend.
- */
-public class ADLVersionInfo extends StringParam {
-  /**
-   * Parameter name.
-   */
-  public static final String NAME = ADLConfKeys.ADL_WEBSDK_VERSION_KEY;
-
-  private static final StringParam.Domain DOMAIN = new StringParam.Domain(NAME,
-      Pattern.compile(".+"));
-
-  /**
-   * Constructor.
-   * @param featureSetVersion Enabled featured information
-   */
-  public ADLVersionInfo(String featureSetVersion) {
-    super(DOMAIN, featureSetVersion);
-  }
-
-  @Override
-  public final String getName() {
-    return NAME;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/AppendADLNoRedirectParam.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/AppendADLNoRedirectParam.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/AppendADLNoRedirectParam.java
deleted file mode 100644
index b9ea79e..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/AppendADLNoRedirectParam.java
+++ /dev/null
@@ -1,45 +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.web.resources;
-
-/**
- * Overwrite parameter.
- */
-public class AppendADLNoRedirectParam extends BooleanParam {
-  /**
-   * Parameter name.
-   */
-  public static final String NAME = "append";
-
-  private static final Domain DOMAIN = new Domain(NAME);
-
-  /**
-   * Constructor.
-   *
-   * @param value the parameter value.
-   */
-  public AppendADLNoRedirectParam(final Boolean value) {
-    super(DOMAIN, value);
-  }
-
-  @Override
-  public final String getName() {
-    return NAME;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateADLNoRedirectParam.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateADLNoRedirectParam.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateADLNoRedirectParam.java
deleted file mode 100644
index 83f3970..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/CreateADLNoRedirectParam.java
+++ /dev/null
@@ -1,44 +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.web.resources;
-
-/**
- * Overwrite parameter.
- */
-public class CreateADLNoRedirectParam extends BooleanParam {
-  /**
-   * Parameter name.
-   */
-  public static final String NAME = "write";
-
-  private static final Domain DOMAIN = new Domain(NAME);
-
-  /**
-   * Constructor.
-   *
-   * @param value the parameter value.
-   */
-  public CreateADLNoRedirectParam(final Boolean value) {
-    super(DOMAIN, value);
-  }
-
-  @Override
-  public final String getName() {
-    return NAME;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/LeaseParam.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/LeaseParam.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/LeaseParam.java
deleted file mode 100644
index 6801235..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/LeaseParam.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.web.resources;
-
-/**
- * To support single writer semantics. Notify to ADL backend if the stream
- * needs to locked in order to protect
- * concurrent write operation on the same stream.
- *
- * Used in append operation.
- */
-public class LeaseParam extends StringParam {
-
-  public static final String NAME = "leaseId";
-  /**
-   * Default parameter value.
-   */
-  public static final String DEFAULT = NULL;
-
-  private static final StringParam.Domain DOMAIN = new StringParam.Domain(NAME,
-      null);
-
-  /**
-   * Constructor.
-   *
-   * @param str a string representation of the parameter value.
-   */
-  public LeaseParam(final String str) {
-    super(DOMAIN, str == null || str.equals(DEFAULT) ? null : str);
-  }
-
-  @Override
-  public final String getName() {
-    return NAME;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ReadADLNoRedirectParam.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ReadADLNoRedirectParam.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ReadADLNoRedirectParam.java
deleted file mode 100644
index a600161..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/ReadADLNoRedirectParam.java
+++ /dev/null
@@ -1,44 +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.web.resources;
-
-/**
- * Overwrite parameter.
- */
-public class ReadADLNoRedirectParam extends BooleanParam {
-  /**
-   * Parameter name.
-   */
-  public static final String NAME = "read";
-
-  private static final Domain DOMAIN = new Domain(NAME);
-
-  /**
-   * Constructor.
-   *
-   * @param value the parameter value.
-   */
-  public ReadADLNoRedirectParam(final Boolean value) {
-    super(DOMAIN, value);
-  }
-
-  @Override
-  public final String getName() {
-    return NAME;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/package-info.java
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/package-info.java
 
b/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/package-info.java
deleted file mode 100644
index 2231cc2..0000000
--- 
a/hadoop-tools/hadoop-azure-datalake/src/main/java/org/apache/hadoop/hdfs/web/resources/package-info.java
+++ /dev/null
@@ -1,27 +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.
- *
- */
-
-/**
- * A distributed implementation of {@link
- * org.apache.hadoop.hdfs.web.resources} for reading or extending query
- * parameter for webhdfs specification. ADL
- * specific
- * query parameter also goes in the same package.
- */
-package org.apache.hadoop.hdfs.web.resources;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/main/resources/META-INF/org.apache.hadoop.fs.FileSystem
----------------------------------------------------------------------
diff --git 
a/hadoop-tools/hadoop-azure-datalake/src/main/resources/META-INF/org.apache.hadoop.fs.FileSystem
 
b/hadoop-tools/hadoop-azure-datalake/src/main/resources/META-INF/org.apache.hadoop.fs.FileSystem
new file mode 100644
index 0000000..7ec7812
--- /dev/null
+++ 
b/hadoop-tools/hadoop-azure-datalake/src/main/resources/META-INF/org.apache.hadoop.fs.FileSystem
@@ -0,0 +1,16 @@
+# 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.
+
+org.apache.hadoop.fs.adl.AdlFileSystem
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c61ad24/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
----------------------------------------------------------------------
diff --git a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md 
b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
index 15a58fb..00825d1 100644
--- a/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
+++ b/hadoop-tools/hadoop-azure-datalake/src/site/markdown/index.md
@@ -19,9 +19,7 @@
 * [Limitations](#Limitations)
 * [Usage](#Usage)
     * [Concepts](#Concepts)
-        * [Webhdfs Compliance](#Webhdfs_Specification_Compliance)
         * [OAuth2 Support](#OAuth2_Support)
-        * [Read Ahead Buffer Management](Read_Ahead_Buffer_Management)
     * [Configuring Credentials & FileSystem](#Configuring_Credentials)
         * [Using Refresh Token](#Refresh_Token)
         * [Using Client Keys](#Client_Credential_Token)
@@ -38,7 +36,6 @@ The jar file is named azure-datalake-store.jar.
 ## <a name="Features" />Features
 
 * Read and write data stored in an Azure Data Lake Storage account.
-* Partial support for [Webhdfs Specification 
2.7.0](https://hadoop.apache.org/docs/r2.7.0/hadoop-project-dist/hadoop-hdfs/WebHDFS.html)
 * Reference file system paths using URLs using the `adl` scheme for Secure 
Webhdfs i.e. SSL
   encrypted access.
 * Can act as a source of data in a MapReduce job, or a sink.
@@ -46,14 +43,14 @@ The jar file is named azure-datalake-store.jar.
 * Tested for scale.
 
 ## <a name="Limitations" />Limitations
-Partial or no support for the following operations in [Webhdfs Specification 
2.7.0](https://hadoop.apache.org/docs/r2.7.0/hadoop-project-dist/hadoop-hdfs/WebHDFS.html):
+Partial or no support for the following operations :
 
 * Operation on Symbolic Link
 * Proxy Users
 * File Truncate
 * File Checksum
 * File replication factor
-* Home Directory Partial supported based on OAuth2 token information and not 
the active user on Hadoop cluster.
+* Home directory the active user on Hadoop cluster.
 * Extended Attributes(XAttrs) Operations
 * Snapshot Operations
 * Delegation Token Operations
@@ -68,101 +65,23 @@ Azure Data Lake Storage access path syntax is
 
 Get started with azure data lake account with 
[https://azure.microsoft.com/en-in/documentation/articles/data-lake-store-get-started-portal/](https://azure.microsoft.com/en-in/documentation/articles/data-lake-store-get-started-portal/)
 
-#### <a name="Webhdfs_Specification_Compliance" />Webhdfs Compliance
-Azure Data Lake Storage exposes a public REST endpoint as per [Webhdfs 
Specification 
2.7.0](https://hadoop.apache.org/docs/r2.7.0/hadoop-project-dist/hadoop-hdfs/WebHDFS.html)
 to access storage file system.
-
-Syntax to access Azure data lake storage account over [Webhdfs Specification 
2.7.0](https://hadoop.apache.org/docs/r2.7.0/hadoop-project-dist/hadoop-hdfs/WebHDFS.html)
 is
-
-    https://<Account Name>.azuredatalakestore.net/webhdfs/v1/<File System 
Path>?<Query paramaters>
-
-
 #### <a name="#OAuth2_Support" />OAuth2 Support
 Usage of Azure Data Lake Storage requires OAuth2 bearer token to be present as 
part of the HTTPS header as per OAuth2 specification. Valid OAuth2 bearer token 
should be obtained from Azure Active Directory for valid users who have  access 
to Azure Data Lake Storage Account.
 
-Azure Active Directory (Azure AD) is Microsoft’s multi-tenant cloud based 
directory and identity management service. See 
[https://azure.microsoft.com/en-in/documentation/articles/active-directory-whatis/](https://azure.microsoft.com/en-in/documentation/articles/active-directory-whatis/)
+Azure Active Directory (Azure AD) is Microsoft's multi-tenant cloud based 
directory and identity management service. See 
[https://azure.microsoft.com/en-in/documentation/articles/active-directory-whatis/](https://azure.microsoft.com/en-in/documentation/articles/active-directory-whatis/)
 
 Following sections describes on OAuth2 configuration in core-site.xml.
 
-#### <a name="#Read_Ahead_Buffer_Management" />Read Ahead Buffer Management
-Azure Data Lake Storage offers high throughput. To maximize throughput, 
applications can  use this feature to buffer data concurrently, in memory 
during read operation. This data is cached in memory per process per stream.
-
-
-To Enable/Disable read ahead feature.
-
-    <property>
-        <name>adl.feature.override.readahead</name>
-        <value>true</value>
-        <description>
-            Enables read aheads in the ADL client, the feature is used to 
improve read throughput.
-            This works in conjunction with the value set in 
adl.feature.override.readahead.max.buffersize.
-            When set to false the read ahead feature is turned off.
-            Default : True if not configured.
-        </description>
-    </property>
-
-To configure read ahead buffer size.
-
-    <property>
-        <name>adl.feature.override.readahead.max.buffersize</name>
-        <value>8388608</value>
-        <description>
-            Define maximum buffer size to cache read ahead data, this is 
allocated per process to
-            cache read ahead data. Applicable only when 
adl.feature.override.readahead is set to true.
-            Default : 8388608 Byte i.e. 8MB if not configured.
-        </description>
-    </property>
-
-To configure number of concurrent connection to Azure Data Lake Storage 
Account.
-
-    <property>
-        <name>adl.feature.override.readahead.max.concurrent.connection</name>
-        <value>2</value>
-        <description>
-            Define maximum concurrent connection can be established to
-            read ahead. If the data size is<4MB then only 1 read n/w connection
-            is set. If the data size is >4MB but<8MB then 2 read n/w
-            connection
-            is set. Data >8MB then value set under the property would
-             take
-            effect. Applicable only when adl.feature.override.readahead is set
-            to true and buffer size is >8MB.
-            It is recommended to reset this property if the 
adl.feature.override.readahead.max.buffersize
-            is < 8MB to gain performance. Application has to consider
-             throttling
-            limit for the account as well before configuring large buffer size.
-        </description>
-    </property>
-
 ## <a name="Configuring_Credentials" />Configuring Credentials & FileSystem
 Credentials can be configured using either a refresh token (associated with a 
user) or a client credential (analogous to a service principal).
 
 ### <a name="Refresh_Token" />Using Refresh Token
 
-Update core-site.xml for OAuth2 configuration
-
-         <property>
-            
<name>dfs.webhdfs.oauth2.refresh.token.expires.ms.since.epoch</name>
-            <value>0</value>
-         </property>
-
-         <property>
-            <name>dfs.webhdfs.oauth2.credential</name>
-            <value>bearer.and.refresh.token</value>
-         </property>
-
-        <property>
-            <name>dfs.webhdfs.oauth2.access.token</name>
-            <value>NOT_SET</value>
-        </property>
-
-        <property>
-            <name>dfs.webhdfs.oauth2.refresh.url</name>
-            <value>https://login.windows.net/common/oauth2/token/</value>
-        </property>
+Add the following properties to your core-site.xml
 
         <property>
-            <name>dfs.webhdfs.oauth2.access.token.provider</name>
-            
<value>org.apache.hadoop.fs.adl.oauth2.CachedRefreshTokenBasedAccessTokenProvider</value>
+            <name>dfs.adls.oauth2.access.token.provider.type</name>
+            <value>RefreshToken</value>
         </property>
 
 Application require to set Client id and OAuth2 refresh token from Azure 
Active Directory associated with client id. See 
[https://github.com/AzureAD/azure-activedirectory-library-for-java](https://github.com/AzureAD/azure-activedirectory-library-for-java).
@@ -170,12 +89,12 @@ Application require to set Client id and OAuth2 refresh 
token from Azure Active
 **Do not share client id and refresh token, it must be kept secret.**
 
         <property>
-            <name>dfs.webhdfs.oauth2.client.id</name>
+            <name>dfs.adls.oauth2.client.id</name>
             <value></value>
         </property>
 
         <property>
-            <name>dfs.webhdfs.oauth2.refresh.token</name>
+            <name>dfs.adls.oauth2.refresh.token</name>
             <value></value>
         </property>
 
@@ -205,30 +124,20 @@ Application require to set Client id and OAuth2 refresh 
token from Azure Active
 Add the following properties to your core-site.xml
 
     <property>
-      <name>dfs.webhdfs.oauth2.access.token.provider</name>
-      
<value>org.apache.hadoop.hdfs.web.oauth2.AzureADClientCredentialBasedAccesTokenProvider</value>
-    </property>
-
-    <property>
-      <name>dfs.webhdfs.oauth2.refresh.url</name>
+      <name>dfs.adls.oauth2.refresh.url</name>
       <value>TOKEN ENDPOINT FROM STEP 7 ABOVE</value>
     </property>
 
     <property>
-      <name>dfs.webhdfs.oauth2.client.id</name>
+      <name>dfs.adls.oauth2.client.id</name>
       <value>CLIENT ID FROM STEP 7 ABOVE</value>
     </property>
 
     <property>
-      <name>dfs.webhdfs.oauth2.credential</name>
+      <name>dfs.adls.oauth2.credential</name>
       <value>PASSWORD FROM STEP 7 ABOVE</value>
     </property>
 
-    <property>
-      <name>fs.adls.oauth2.resource</name>
-      <value>https://management.core.windows.net/</value>
-    </property>
-
 
 
 ## <a name="Enabling_ADL" />Enabling ADL Filesystem
@@ -273,7 +182,12 @@ The hadoop-azure module includes a full suite of unit 
tests. Most of the tests w
 
 A selection of tests can run against the Azure Data Lake Storage. To run tests 
against Adl storage. Please configure contract-test-options.xml with Adl 
account information mentioned in the above sections. Also turn on contract test 
execution flag to trigger tests against Azure Data Lake Storage.
 
-    <property>
-      <name>dfs.adl.test.contract.enable</name>
-      <value>true</value>
-    </property>
+        <property>
+            <name>dfs.adl.test.contract.enable</name>
+            <value>true</value>
+        </property>
+
+        <property>
+            <name>test.fs.adl.name</name>
+            <value>adl://yourcontainer.azuredatalakestore.net</value>
+        </property>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-commits-h...@hadoop.apache.org

Reply via email to