[ 
https://issues.apache.org/jira/browse/HADOOP-13560?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15571698#comment-15571698
 ] 

ASF GitHub Bot commented on HADOOP-13560:
-----------------------------------------

Github user pieterreuse commented on a diff in the pull request:

    https://github.com/apache/hadoop/pull/130#discussion_r83198713
  
    --- Diff: 
hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/S3ADataBlocks.java
 ---
    @@ -0,0 +1,811 @@
    +/*
    + * 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.s3a;
    +
    +import java.io.BufferedOutputStream;
    +import java.io.ByteArrayInputStream;
    +import java.io.ByteArrayOutputStream;
    +import java.io.Closeable;
    +import java.io.EOFException;
    +import java.io.File;
    +import java.io.FileInputStream;
    +import java.io.FileNotFoundException;
    +import java.io.FileOutputStream;
    +import java.io.FilterInputStream;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.ByteBuffer;
    +import java.util.concurrent.atomic.AtomicBoolean;
    +import java.util.concurrent.atomic.AtomicInteger;
    +
    +import com.google.common.base.Preconditions;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.hadoop.fs.FSExceptionMessages;
    +import org.apache.hadoop.util.DirectBufferPool;
    +
    +import static org.apache.hadoop.fs.s3a.S3ADataBlocks.DataBlock.DestState.*;
    +
    +/**
    + * Set of classes to support output streaming into blocks which are then
    + * uploaded as partitions.
    + */
    +final class S3ADataBlocks {
    +
    +  private static final Logger LOG = 
LoggerFactory.getLogger(S3ADataBlocks.class);
    +
    +  private S3ADataBlocks() {
    +  }
    +
    +  /**
    +   * Validate args to a write command. These are the same validation checks
    +   * expected for any implementation of {@code OutputStream.write()}.
    +   * @param b byte array containing data
    +   * @param off offset in array where to start
    +   * @param len number of bytes to be written
    +   * @throws NullPointerException for a null buffer
    +   * @throws IndexOutOfBoundsException if indices are out of range
    +   */
    +  static void validateWriteArgs(byte[] b, int off, int len)
    +      throws IOException {
    +    Preconditions.checkNotNull(b);
    +    if ((off < 0) || (off > b.length) || (len < 0) ||
    +        ((off + len) > b.length) || ((off + len) < 0)) {
    +      throw new IndexOutOfBoundsException(
    +          "write (b[" + b.length + "], " + off + ", " + len + ')');
    +    }
    +  }
    +
    +  /**
    +   * Create a factory.
    +   * @param owner factory owner
    +   * @param name factory name -the option from {@link Constants}.
    +   * @return the factory, ready to be initialized.
    +   * @throws IllegalArgumentException if the name is unknown.
    +   */
    +  static BlockFactory createFactory(S3AFileSystem owner,
    +      String name) {
    +    switch (name) {
    +    case Constants.FAST_UPLOAD_BUFFER_ARRAY:
    +      return new ArrayBlockFactory(owner);
    +    case Constants.FAST_UPLOAD_BUFFER_DISK:
    +      return new DiskBlockFactory(owner);
    +    case Constants.FAST_UPLOAD_BYTEBUFFER:
    +      return new ByteBufferBlockFactory(owner);
    +    default:
    +      throw new IllegalArgumentException("Unsupported block buffer" +
    +          " \"" + name + '"');
    +    }
    +  }
    +
    +  /**
    +   * Base class for block factories.
    +   */
    +  static abstract class BlockFactory implements Closeable {
    +
    +    /**
    +     * Owner.
    +     */
    +    protected final S3AFileSystem owner;
    +
    +    protected BlockFactory(S3AFileSystem owner) {
    +      this.owner = owner;
    +    }
    +
    +    /**
    +     * Create a block.
    +     * @param limit limit of the block.
    +     * @return a new block.
    +     */
    +    abstract DataBlock create(int limit) throws IOException;
    +
    +    /**
    +     * Implement any close/cleanup operation.
    +     * Base class is a no-op
    +     * @throws IOException -ideally, it shouldn't.
    +     */
    +    @Override
    +    public void close() throws IOException {
    +    }
    +  }
    +
    +  /**
    +   * This represents a block being uploaded.
    +   */
    +  static abstract class DataBlock implements Closeable {
    +
    +    private volatile DestState state = Writing;
    --- End diff --
    
    I'm nitpicking here, but wouldn't it make more sense to define DestState 
here instead of on line 272? Moving that line here would improve code 
readability imo but wouldn't change any behaviour.


> S3ABlockOutputStream to support huge (many GB) file writes
> ----------------------------------------------------------
>
>                 Key: HADOOP-13560
>                 URL: https://issues.apache.org/jira/browse/HADOOP-13560
>             Project: Hadoop Common
>          Issue Type: Sub-task
>          Components: fs/s3
>    Affects Versions: 2.9.0
>            Reporter: Steve Loughran
>            Assignee: Steve Loughran
>         Attachments: HADOOP-13560-branch-2-001.patch, 
> HADOOP-13560-branch-2-002.patch, HADOOP-13560-branch-2-003.patch, 
> HADOOP-13560-branch-2-004.patch
>
>
> An AWS SDK [issue|https://github.com/aws/aws-sdk-java/issues/367] highlights 
> that metadata isn't copied on large copies.
> 1. Add a test to do that large copy/rname and verify that the copy really 
> works
> 2. Verify that metadata makes it over.
> Verifying large file rename is important on its own, as it is needed for very 
> large commit operations for committers using rename



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

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

Reply via email to