فهرست منبع

HDDS-791. Support Range header for ozone s3 object download. Contributed by Bharat Viswanadham.

Márton Elek 6 سال پیش
والد
کامیت
22867deffa

+ 115 - 1
hadoop-ozone/dist/src/main/smoketest/s3/objectputget.robot

@@ -29,14 +29,128 @@ ${BUCKET}             generated
 *** Test Cases ***
 
 Put object to s3
-                        Execute                    date > /tmp/testfile
+                        Execute                    echo "Randomtext" > /tmp/testfile
     ${result} =         Execute AWSS3ApiCli        put-object --storage-class REDUCED_REDUNDANCY --bucket ${BUCKET} --key putobject/f1 --body /tmp/testfile
     ${result} =         Execute AWSS3ApiCli        list-objects --bucket ${BUCKET} --prefix putobject/
                         Should contain             ${result}         f1
 
+                        Execute                    touch -f /tmp/zerobyte
+    ${result} =         Execute AWSS3ApiCli        put-object --storage-class REDUCED_REDUNDANCY --bucket ${BUCKET} --key putobject/zerobyte --body /tmp/zerobyte
+    ${result} =         Execute AWSS3ApiCli        list-objects --bucket ${BUCKET} --prefix putobject/
+                        Should contain             ${result}         zerobyte
+
 #This test depends on the previous test case. Can't be executes alone
 Get object from s3
     ${result} =                 Execute AWSS3ApiCli        get-object --bucket ${BUCKET} --key putobject/f1 /tmp/testfile.result
     ${checksumbefore} =         Execute                    md5sum /tmp/testfile | awk '{print $1}'
     ${checksumafter} =          Execute                    md5sum /tmp/testfile.result | awk '{print $1}'
                                 Should Be Equal            ${checksumbefore}            ${checksumafter}
+
+Get Partial object from s3 with both start and endoffset
+    ${result} =                 Execute AWSS3ApiCli        get-object --bucket ${BUCKET} --key putobject/f1 --range bytes=0-4 /tmp/testfile1.result
+                                Should contain             ${result}        ContentRange
+                                Should contain             ${result}        bytes 0-4/11
+                                Should contain             ${result}        AcceptRanges
+    ${expectedData} =           Execute                    dd if=/tmp/testfile skip=0 bs=1 count=5 2>/dev/null
+    ${actualData} =             Execute                    cat /tmp/testfile1.result
+                                Should Be Equal            ${expectedData}            ${actualData}
+
+    ${result} =                 Execute AWSS3ApiCli        get-object --bucket ${BUCKET} --key putobject/f1 --range bytes=2-4 /tmp/testfile1.result1
+                                Should contain             ${result}        ContentRange
+                                Should contain             ${result}        bytes 2-4/11
+                                Should contain             ${result}        AcceptRanges
+    ${expectedData} =           Execute                    dd if=/tmp/testfile skip=2 bs=1 count=3 2>/dev/null
+    ${actualData} =             Execute                    cat /tmp/testfile1.result1
+                                Should Be Equal            ${expectedData}            ${actualData}
+
+# end offset greater than file size and start with in file length
+    ${result} =                 Execute AWSS3ApiCli        get-object --bucket ${BUCKET} --key putobject/f1 --range bytes=2-1000 /tmp/testfile1.result2
+                                Should contain             ${result}        ContentRange
+                                Should contain             ${result}        bytes 2-10/11
+                                Should contain             ${result}        AcceptRanges
+    ${expectedData} =           Execute                    dd if=/tmp/testfile skip=2 bs=1 count=9 2>/dev/null
+    ${actualData} =             Execute                    cat /tmp/testfile1.result2
+                                Should Be Equal            ${expectedData}            ${actualData}
+
+Get Partial object from s3 with both start and endoffset(start offset and endoffset is greater than file size)
+    ${result} =                 Execute AWSS3APICli and checkrc        get-object --bucket ${BUCKET} --key putobject/f1 --range bytes=10000-10000 /tmp/testfile2.result   255
+                                Should contain             ${result}        InvalidRange
+
+
+Get Partial object from s3 with both start and endoffset(end offset is greater than file size)
+    ${result} =                 Execute AWSS3ApiCli        get-object --bucket ${BUCKET} --key putobject/f1 --range bytes=0-10000 /tmp/testfile2.result
+                                Should contain             ${result}        ContentRange
+                                Should contain             ${result}        bytes 0-10/11
+                                Should contain             ${result}        AcceptRanges
+    ${expectedData} =           Execute                    cat /tmp/testfile
+    ${actualData} =             Execute                    cat /tmp/testfile2.result
+                                Should Be Equal            ${expectedData}            ${actualData}
+
+Get Partial object from s3 with only start offset
+    ${result} =                 Execute AWSS3ApiCli        get-object --bucket ${BUCKET} --key putobject/f1 --range bytes=0- /tmp/testfile3.result
+                                Should contain             ${result}        ContentRange
+                                Should contain             ${result}        bytes 0-10/11
+                                Should contain             ${result}        AcceptRanges
+    ${expectedData} =           Execute                    cat /tmp/testfile
+    ${actualData} =             Execute                    cat /tmp/testfile3.result
+                                Should Be Equal            ${expectedData}            ${actualData}
+
+Get Partial object from s3 with both start and endoffset which are equal
+    ${result} =                 Execute AWSS3ApiCli        get-object --bucket ${BUCKET} --key putobject/f1 --range bytes=0-0 /tmp/testfile4.result
+                                Should contain             ${result}        ContentRange
+                                Should contain             ${result}        bytes 0-0/11
+                                Should contain             ${result}        AcceptRanges
+    ${expectedData} =           Execute                    dd if=/tmp/testfile skip=0 bs=1 count=1 2>/dev/null
+    ${actualData} =             Execute                    cat /tmp/testfile4.result
+                                Should Be Equal            ${expectedData}            ${actualData}
+
+    ${result} =                 Execute AWSS3ApiCli        get-object --bucket ${BUCKET} --key putobject/f1 --range bytes=4-4 /tmp/testfile5.result
+                                Should contain             ${result}        ContentRange
+                                Should contain             ${result}        bytes 4-4/11
+                                Should contain             ${result}        AcceptRanges
+    ${expectedData} =           Execute                    dd if=/tmp/testfile skip=4 bs=1 count=1 2>/dev/null
+    ${actualData} =             Execute                    cat /tmp/testfile5.result
+                                Should Be Equal            ${expectedData}            ${actualData}
+
+Get Partial object from s3 to get last n bytes
+    ${result} =                 Execute AWSS3ApiCli        get-object --bucket ${BUCKET} --key putobject/f1 --range bytes=-4 /tmp/testfile6.result
+                                Should contain             ${result}        ContentRange
+                                Should contain             ${result}        bytes 7-10/11
+                                Should contain             ${result}        AcceptRanges
+    ${expectedData} =           Execute                    dd if=/tmp/testfile skip=7 bs=1 count=4 2>/dev/null
+    ${actualData} =             Execute                    cat /tmp/testfile6.result
+                                Should Be Equal            ${expectedData}            ${actualData}
+
+# if end is greater than file length, returns whole file
+    ${result} =                 Execute AWSS3ApiCli        get-object --bucket ${BUCKET} --key putobject/f1 --range bytes=-10000 /tmp/testfile7.result
+                                Should contain             ${result}        ContentRange
+                                Should contain             ${result}        bytes 0-10/11
+                                Should contain             ${result}        AcceptRanges
+    ${expectedData} =           Execute                    cat /tmp/testfile
+    ${actualData} =             Execute                    cat /tmp/testfile7.result
+                                Should Be Equal            ${expectedData}            ${actualData}
+
+Incorrect values for end and start offset
+    ${result} =                 Execute AWSS3ApiCli        get-object --bucket ${BUCKET} --key putobject/f1 --range bytes=-11-10000 /tmp/testfile8.result
+                                Should not contain         ${result}        ContentRange
+                                Should contain             ${result}        AcceptRanges
+    ${expectedData} =           Execute                    cat /tmp/testfile
+    ${actualData} =             Execute                    cat /tmp/testfile8.result
+                                Should Be Equal            ${expectedData}            ${actualData}
+
+    ${result} =                 Execute AWSS3ApiCli        get-object --bucket ${BUCKET} --key putobject/f1 --range bytes=11-8 /tmp/testfile9.result
+                                Should not contain         ${result}        ContentRange
+                                Should contain             ${result}        AcceptRanges
+    ${expectedData} =           Execute                    cat /tmp/testfile
+    ${actualData} =             Execute                    cat /tmp/testfile8.result
+                                Should Be Equal            ${expectedData}            ${actualData}
+
+Zero byte file
+    ${result} =                 Execute AWSS3APICli and checkrc        get-object --bucket ${BUCKET} --key putobject/zerobyte --range bytes=0-0 /tmp/testfile2.result   255
+                                Should contain             ${result}        InvalidRange
+
+    ${result} =                 Execute AWSS3APICli and checkrc        get-object --bucket ${BUCKET} --key putobject/zerobyte --range bytes=0-1 /tmp/testfile2.result   255
+                                Should contain             ${result}        InvalidRange
+
+    ${result} =                 Execute AWSS3APICli and checkrc        get-object --bucket ${BUCKET} --key putobject/zerobyte --range bytes=0-10000 /tmp/testfile2.result   255
+                                Should contain             ${result}        InvalidRange

+ 67 - 7
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java

@@ -51,15 +51,18 @@ import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 
 import com.google.common.annotations.VisibleForTesting;
 import org.apache.commons.io.IOUtils;
+
+import org.apache.hadoop.ozone.s3.io.S3WrapperInputStream;
+import org.apache.hadoop.ozone.s3.util.RangeHeader;
 import org.apache.hadoop.ozone.s3.util.S3StorageType;
+import org.apache.hadoop.ozone.s3.util.S3utils;
 import org.apache.hadoop.ozone.web.utils.OzoneUtils;
 import org.apache.hadoop.util.Time;
 import org.apache.http.HttpStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.hadoop.ozone.s3.util.S3Consts.COPY_SOURCE_HEADER;
-import static org.apache.hadoop.ozone.s3.util.S3Consts.STORAGE_CLASS_HEADER;
+import static org.apache.hadoop.ozone.s3.util.S3Consts.*;
 
 /**
  * Key level rest endpoints.
@@ -165,16 +168,73 @@ public class ObjectEndpoint extends EndpointBase {
       @PathParam("bucket") String bucketName,
       @PathParam("path") String keyPath,
       InputStream body) throws IOException, OS3Exception {
-
     try {
       OzoneBucket bucket = getBucket(bucketName);
 
-      OzoneInputStream key = bucket
-          .readKey(keyPath);
+      OzoneKeyDetails keyDetails = bucket.getKey(keyPath);
+
+      long length = keyDetails.getDataSize();
+
+      LOG.debug("Data length of the key {} is {}", keyPath, length);
+
+      String rangeHeaderVal = headers.getHeaderString(RANGE_HEADER);
+      RangeHeader rangeHeader = null;
+
+      LOG.debug("range Header provided value is {}", rangeHeaderVal);
+
+      if (rangeHeaderVal != null) {
+        rangeHeader = S3utils.parseRangeHeader(rangeHeaderVal,
+            length);
+        LOG.debug("range Header provided value is {}", rangeHeader);
+        if (rangeHeader.isInValidRange()) {
+          OS3Exception exception = S3ErrorTable.newError(S3ErrorTable
+              .INVALID_RANGE, rangeHeaderVal);
+          throw exception;
+        }
+      }
+      ResponseBuilder responseBuilder;
+
+      if (rangeHeaderVal == null || rangeHeader.isReadFull()) {
+        StreamingOutput output = dest -> {
+          try (OzoneInputStream key = bucket.readKey(keyPath)) {
+            IOUtils.copy(key, dest);
+          }
+        };
+        responseBuilder = Response.ok(output);
+
+      } else {
+        LOG.info("range Header provided value is {}", rangeHeader);
+        OzoneInputStream key = bucket.readKey(keyPath);
+
+        long startOffset = rangeHeader.getStartOffset();
+        long endOffset = rangeHeader.getEndOffset();
+        long copyLength;
+        if (startOffset == endOffset) {
+          // if range header is given as bytes=0-0, then we should return 1
+          // byte from start offset
+          copyLength = 1;
+        } else {
+          copyLength = rangeHeader.getEndOffset() - rangeHeader
+              .getStartOffset() + 1;
+        }
+        StreamingOutput output = dest -> {
+          try (S3WrapperInputStream s3WrapperInputStream =
+              new S3WrapperInputStream(
+                  key.getInputStream())) {
+            IOUtils.copyLarge(s3WrapperInputStream, dest, startOffset,
+                copyLength);
+          }
+        };
+        responseBuilder = Response.ok(output);
 
-      StreamingOutput output = dest -> IOUtils.copy(key, dest);
-      ResponseBuilder responseBuilder = Response.ok(output);
+        String contentRangeVal = RANGE_HEADER_SUPPORTED_UNIT + " " +
+            rangeHeader.getStartOffset() + "-" + rangeHeader.getEndOffset() +
+            "/" + length;
 
+        responseBuilder.header(CONTENT_RANGE_HEADER, contentRangeVal);
+      }
+      responseBuilder.header(ACCEPT_RANGE_HEADER,
+          RANGE_HEADER_SUPPORTED_UNIT);
       for (String responseHeader : customizableGetHeaders) {
         String headerValue = headers.getHeaderString(responseHeader);
         if (headerValue != null) {

+ 5 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/exception/S3ErrorTable.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.ozone.s3.exception;
 import static java.net.HttpURLConnection.HTTP_BAD_REQUEST;
 import static java.net.HttpURLConnection.HTTP_CONFLICT;
 import static java.net.HttpURLConnection.HTTP_NOT_FOUND;
+import static org.apache.hadoop.ozone.s3.util.S3Consts.RANGE_NOT_SATISFIABLE;
 
 /**
  * This class represents errors from Ozone S3 service.
@@ -58,6 +59,10 @@ public final class S3ErrorTable {
   public static final OS3Exception INVALID_REQUEST = new OS3Exception(
       "InvalidRequest", "Invalid Request", HTTP_BAD_REQUEST);
 
+  public static final OS3Exception INVALID_RANGE = new OS3Exception(
+      "InvalidRange", "The requested range is not satisfiable",
+      RANGE_NOT_SATISFIABLE);
+
   /**
    * Create a new instance of Error.
    * @param e Error Template

+ 79 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/io/S3WrapperInputStream.java

@@ -0,0 +1,79 @@
+/**
+ * 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.ozone.s3.io;
+
+import org.apache.hadoop.fs.FSInputStream;
+import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * S3Wrapper Input Stream which encapsulates ChunkGroupInputStream from ozone.
+ */
+public class S3WrapperInputStream extends FSInputStream {
+  private final ChunkGroupInputStream inputStream;
+
+  /**
+   * Constructs S3WrapperInputStream with ChunkInputStream.
+   *
+   * @param inputStream
+   */
+  public S3WrapperInputStream(InputStream inputStream) {
+    this.inputStream = (ChunkGroupInputStream) inputStream;
+  }
+
+  @Override
+  public int read() throws IOException {
+    return inputStream.read();
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    return inputStream.read(b, off, len);
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    inputStream.close();
+  }
+
+  @Override
+  public int available() throws IOException {
+    return inputStream.available();
+  }
+
+  public InputStream getInputStream() {
+    return inputStream;
+  }
+
+  @Override
+  public void seek(long pos) throws IOException {
+    inputStream.seek(pos);
+  }
+  @Override
+  public long getPos() throws IOException {
+    return inputStream.getPos();
+  }
+
+  @Override
+  public boolean seekToNewSource(long targetPos) throws IOException {
+    return false;
+  }
+}

+ 23 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/io/package-info.java

@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+/**
+ * This package contains Ozone S3 wrapper stream related classes.
+ */
+
+package org.apache.hadoop.ozone.s3.io;

+ 89 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/RangeHeader.java

@@ -0,0 +1,89 @@
+/*
+ * 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.ozone.s3.util;
+
+/**
+ * Ranger Header class which hold startoffset, endoffset of the Range header
+ * value provided as part of get object.
+ *
+ */
+public class RangeHeader {
+  private long startOffset;
+  private long endOffset;
+  private boolean readFull;
+  private boolean inValidRange;
+
+
+  /**
+   * Construct RangeHeader object.
+   * @param startOffset
+   * @param endOffset
+   */
+  public RangeHeader(long startOffset, long endOffset, boolean full,
+                     boolean invalid) {
+    this.startOffset = startOffset;
+    this.endOffset = endOffset;
+    this.readFull = full;
+    this.inValidRange = invalid;
+  }
+
+  /**
+   * Return startOffset.
+   *
+   * @return startOffset
+   */
+  public long getStartOffset() {
+    return startOffset;
+  }
+
+  /**
+   * Return endoffset.
+   *
+   * @return endoffset
+   */
+  public long getEndOffset() {
+    return endOffset;
+  }
+
+  /**
+   * Return a flag whether after parsing range header, when the provided
+   * values are with in a range, and whole file read is required.
+   *
+   * @return readFull
+   */
+  public boolean isReadFull() {
+    return readFull;
+  }
+
+  /**
+   * Return a flag, whether range header values are correct or not.
+   *
+   * @return isInValidRange
+   */
+  public boolean isInValidRange() {
+    return inValidRange;
+  }
+
+
+  public String toString() {
+    return "startOffset - [" + startOffset + "]" + "endOffset - [" +
+        endOffset + "]" + " readFull - [ " + readFull + "]" + " invalidRange " +
+        "- [ " + inValidRange + "]";
+  }
+}

+ 15 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3Consts.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.ozone.s3.util;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 
+import java.util.regex.Pattern;
+
 /**
  * Set of constants used for S3 implementation.
  */
@@ -35,4 +37,17 @@ public final class S3Consts {
   public static final String STORAGE_CLASS_HEADER = "x-amz-storage-class";
   public static final String ENCODING_TYPE = "url";
 
+  // Constants related to Range Header
+  public static final String RANGE_HEADER_SUPPORTED_UNIT = "bytes";
+  public static final String RANGE_HEADER = "Range";
+  public static final String ACCEPT_RANGE_HEADER = "Accept-Ranges";
+  public static final String CONTENT_RANGE_HEADER = "Content-Range";
+
+
+  public static final Pattern RANGE_HEADER_MATCH_PATTERN =
+      Pattern.compile("bytes=(?<start>[0-9]*)-(?<end>[0-9]*)");
+
+  //Error code 416 is Range Not Satisfiable
+  public static final int RANGE_NOT_SATISFIABLE = 416;
+
 }

+ 68 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/S3utils.java

@@ -21,14 +21,20 @@ package org.apache.hadoop.ozone.s3.util;
 import org.apache.commons.codec.DecoderException;
 import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.codec.digest.DigestUtils;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 import org.apache.hadoop.ozone.s3.exception.S3ErrorTable;
 
+
 import java.nio.charset.StandardCharsets;
+import java.util.regex.Matcher;
 
+import static org.apache.hadoop.ozone.s3.util.S3Consts
+    .RANGE_HEADER_MATCH_PATTERN;
 /**
  * Utility class for S3.
  */
+@InterfaceAudience.Private
 public final class S3utils {
 
   private S3utils() {
@@ -88,4 +94,66 @@ public final class S3utils {
       return null;
     }
   }
+
+
+  /**
+   * Parse the rangeHeader and set the start and end offset.
+   * @param rangeHeaderVal
+   * @param length
+   *
+   * @return RangeHeader
+   */
+  public static RangeHeader parseRangeHeader(String rangeHeaderVal, long
+      length) {
+    long start = 0;
+    long end = 0;
+    boolean noStart = false;
+    boolean readFull = false;
+    boolean inValidRange = false;
+    RangeHeader rangeHeader;
+    Matcher matcher = RANGE_HEADER_MATCH_PATTERN.matcher(rangeHeaderVal);
+    if (matcher.matches()) {
+      if (!matcher.group("start").equals("")) {
+        start = Integer.parseInt(matcher.group("start"));
+      } else {
+        noStart = true;
+      }
+      if (!matcher.group("end").equals("")) {
+        end = Integer.parseInt(matcher.group("end"));
+      } else {
+        end = length - 1;
+      }
+      if (noStart) {
+        if (end < length) {
+          start = length - end;
+        } else {
+          start = 0;
+        }
+        end = length - 1;
+      } else {
+        if (start >= length)  {
+          readFull = true;
+          if (end >= length) {
+            inValidRange = true;
+          } else {
+            start = 0;
+            end = length - 1;
+          }
+        } else {
+          if (end >= length) {
+            end = length - 1;
+          }
+        }
+      }
+    } else {
+      // Byte specification is not matching or start and endoffset provided
+      // are not matching with regex.
+      start = 0;
+      end = length - 1;
+      readFull = true;
+    }
+    rangeHeader = new RangeHeader(start, end, readFull, inValidRange);
+    return rangeHeader;
+
+  }
 }

+ 93 - 0
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestS3utils.java

@@ -0,0 +1,93 @@
+/**
+ * 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.ozone.s3.util;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test class to test S3utils.
+ */
+public class TestS3utils {
+
+  @Test
+  public void testRangeHeaderParser() {
+
+    RangeHeader rangeHeader;
+
+
+    //range is with in file length
+    rangeHeader = S3utils.parseRangeHeader("bytes=0-8", 10);
+    assertEquals(0, rangeHeader.getStartOffset());
+    assertEquals(8, rangeHeader.getEndOffset());
+    assertEquals(false, rangeHeader.isReadFull());
+    assertEquals(false, rangeHeader.isInValidRange());
+
+    //range is with in file length, both start and end offset are same
+    rangeHeader = S3utils.parseRangeHeader("bytes=0-0", 10);
+    assertEquals(0, rangeHeader.getStartOffset());
+    assertEquals(0, rangeHeader.getEndOffset());
+    assertEquals(false, rangeHeader.isReadFull());
+    assertEquals(false, rangeHeader.isInValidRange());
+
+    //range is not with in file length, both start and end offset are greater
+    // than length
+    rangeHeader = S3utils.parseRangeHeader("bytes=11-10", 10);
+    assertEquals(true, rangeHeader.isInValidRange());
+
+    // range is satisfying, one of the range is with in the length. So, read
+    // full file
+    rangeHeader = S3utils.parseRangeHeader("bytes=11-8", 10);
+    assertEquals(0, rangeHeader.getStartOffset());
+    assertEquals(9, rangeHeader.getEndOffset());
+    assertEquals(true, rangeHeader.isReadFull());
+    assertEquals(false, rangeHeader.isInValidRange());
+
+    // bytes spec is wrong
+    rangeHeader = S3utils.parseRangeHeader("mb=11-8", 10);
+    assertEquals(0, rangeHeader.getStartOffset());
+    assertEquals(9, rangeHeader.getEndOffset());
+    assertEquals(true, rangeHeader.isReadFull());
+    assertEquals(false, rangeHeader.isInValidRange());
+
+    // range specified is invalid
+    rangeHeader = S3utils.parseRangeHeader("bytes=-11-8", 10);
+    assertEquals(0, rangeHeader.getStartOffset());
+    assertEquals(9, rangeHeader.getEndOffset());
+    assertEquals(true, rangeHeader.isReadFull());
+    assertEquals(false, rangeHeader.isInValidRange());
+
+    //Last n bytes
+    rangeHeader = S3utils.parseRangeHeader("bytes=-6", 10);
+    assertEquals(4, rangeHeader.getStartOffset());
+    assertEquals(9, rangeHeader.getEndOffset());
+    assertEquals(false, rangeHeader.isReadFull());
+    assertEquals(false, rangeHeader.isInValidRange());
+
+    rangeHeader = S3utils.parseRangeHeader("bytes=-106", 10);
+    assertEquals(0, rangeHeader.getStartOffset());
+    assertEquals(9, rangeHeader.getEndOffset());
+    assertEquals(false, rangeHeader.isInValidRange());
+
+
+
+  }
+
+}