Selaa lähdekoodia

HDDS-820. Use more strict data format for the Last-Modified headers of s3 gateway. Contributed by Elek, Marton.

Márton Elek 6 vuotta sitten
vanhempi
commit
753f421350

+ 28 - 16
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/endpoint/ObjectEndpoint.java

@@ -35,7 +35,6 @@ import java.io.InputStream;
 import java.time.Instant;
 import java.time.ZoneId;
 import java.time.ZonedDateTime;
-import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
 import java.util.List;
 
@@ -48,22 +47,27 @@ import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
 import org.apache.hadoop.ozone.s3.SignedChunksInputStream;
 import org.apache.hadoop.ozone.s3.exception.OS3Exception;
 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.RFC1123Util;
 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 com.google.common.annotations.VisibleForTesting;
+import static javax.ws.rs.core.HttpHeaders.LAST_MODIFIED;
+import org.apache.commons.io.IOUtils;
+import static org.apache.hadoop.ozone.s3.util.S3Consts.ACCEPT_RANGE_HEADER;
+import static org.apache.hadoop.ozone.s3.util.S3Consts.CONTENT_RANGE_HEADER;
+import static org.apache.hadoop.ozone.s3.util.S3Consts.COPY_SOURCE_HEADER;
+import static org.apache.hadoop.ozone.s3.util.S3Consts.RANGE_HEADER;
+import static org.apache.hadoop.ozone.s3.util.S3Consts.RANGE_HEADER_SUPPORTED_UNIT;
+import static org.apache.hadoop.ozone.s3.util.S3Consts.STORAGE_CLASS_HEADER;
 import org.apache.http.HttpStatus;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import static org.apache.hadoop.ozone.s3.util.S3Consts.*;
-
 /**
  * Key level rest endpoints.
  */
@@ -241,7 +245,7 @@ public class ObjectEndpoint extends EndpointBase {
           responseBuilder.header(responseHeader, headerValue);
         }
       }
-
+      addLastModifiedDate(responseBuilder, keyDetails);
       return responseBuilder.build();
     } catch (IOException ex) {
       if (ex.getMessage().contains("NOT_FOUND")) {
@@ -254,6 +258,18 @@ public class ObjectEndpoint extends EndpointBase {
     }
   }
 
+  private void addLastModifiedDate(
+      ResponseBuilder responseBuilder, OzoneKeyDetails key) {
+
+    ZonedDateTime lastModificationTime =
+        Instant.ofEpochMilli(key.getModificationTime())
+            .atZone(ZoneId.of("GMT"));
+
+    responseBuilder
+        .header(LAST_MODIFIED,
+            RFC1123Util.FORMAT.format(lastModificationTime));
+  }
+
   /**
    * Rest endpoint to check existence of an object in a bucket.
    * <p>
@@ -279,16 +295,12 @@ public class ObjectEndpoint extends EndpointBase {
       }
     }
 
-    ZonedDateTime lastModificationTime =
-        Instant.ofEpochMilli(key.getModificationTime())
-            .atZone(ZoneId.of("GMT"));
-
-    return Response.ok().status(HttpStatus.SC_OK)
-        .header("Last-Modified",
-            DateTimeFormatter.RFC_1123_DATE_TIME.format(lastModificationTime))
+    ResponseBuilder response = Response.ok().status(HttpStatus.SC_OK)
         .header("ETag", "" + key.getModificationTime())
         .header("Content-Length", key.getDataSize())
-        .header("Content-Type", "binary/octet-stream")
+        .header("Content-Type", "binary/octet-stream");
+    addLastModifiedDate(response, key);
+    return response
         .build();
   }
 

+ 98 - 0
hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/util/RFC1123Util.java

@@ -0,0 +1,98 @@
+/*
+ * 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 java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeFormatterBuilder;
+import java.time.format.SignStyle;
+import java.util.HashMap;
+import java.util.Map;
+
+import static java.time.temporal.ChronoField.DAY_OF_MONTH;
+import static java.time.temporal.ChronoField.DAY_OF_WEEK;
+import static java.time.temporal.ChronoField.HOUR_OF_DAY;
+import static java.time.temporal.ChronoField.MINUTE_OF_HOUR;
+import static java.time.temporal.ChronoField.MONTH_OF_YEAR;
+import static java.time.temporal.ChronoField.SECOND_OF_MINUTE;
+import static java.time.temporal.ChronoField.YEAR;
+
+/**
+ * Stricter RFC1123 data format.
+ * <p>
+ * This format always use two digits for the days to make it compatible with
+ * golang clients.
+ */
+public final class RFC1123Util {
+
+  private RFC1123Util() {
+  }
+
+  /**
+   * An RFC-1123 compatible file format which always use two digits for the
+   * days.
+   */
+  public static final DateTimeFormatter FORMAT;
+
+  static {
+    Map<Long, String> dow = new HashMap<>();
+    dow.put(1L, "Mon");
+    dow.put(2L, "Tue");
+    dow.put(3L, "Wed");
+    dow.put(4L, "Thu");
+    dow.put(5L, "Fri");
+    dow.put(6L, "Sat");
+    dow.put(7L, "Sun");
+    Map<Long, String> moy = new HashMap<>();
+    moy.put(1L, "Jan");
+    moy.put(2L, "Feb");
+    moy.put(3L, "Mar");
+    moy.put(4L, "Apr");
+    moy.put(5L, "May");
+    moy.put(6L, "Jun");
+    moy.put(7L, "Jul");
+    moy.put(8L, "Aug");
+    moy.put(9L, "Sep");
+    moy.put(10L, "Oct");
+    moy.put(11L, "Nov");
+    moy.put(12L, "Dec");
+    FORMAT = new DateTimeFormatterBuilder()
+        .parseCaseInsensitive()
+        .parseLenient()
+        .optionalStart()
+        .appendText(DAY_OF_WEEK, dow)
+        .appendLiteral(", ")
+        .optionalEnd()
+        .appendValue(DAY_OF_MONTH, 2, 2, SignStyle.NOT_NEGATIVE)
+        .appendLiteral(' ')
+        .appendText(MONTH_OF_YEAR, moy)
+        .appendLiteral(' ')
+        .appendValue(YEAR, 4)
+        .appendLiteral(' ')
+        .appendValue(HOUR_OF_DAY, 2)
+        .appendLiteral(':')
+        .appendValue(MINUTE_OF_HOUR, 2)
+        .optionalStart()
+        .appendLiteral(':')
+        .appendValue(SECOND_OF_MINUTE, 2)
+        .optionalEnd()
+        .appendLiteral(' ')
+        .appendOffset("+HHMM", "GMT")
+        .toFormatter();
+  }
+}

+ 7 - 1
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/endpoint/TestObjectGet.java

@@ -21,9 +21,11 @@
 package org.apache.hadoop.ozone.s3.endpoint;
 
 import javax.ws.rs.core.HttpHeaders;
+import javax.ws.rs.core.Response;
 import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.nio.charset.Charset;
+import java.time.format.DateTimeFormatter;
 
 import org.apache.hadoop.ozone.client.OzoneBucket;
 import org.apache.hadoop.ozone.client.OzoneClientStub;
@@ -68,7 +70,7 @@ public class TestObjectGet {
         new ByteArrayInputStream(CONTENT.getBytes(UTF_8));
 
     //WHEN
-    rest.get("b1", "key1", body);
+    Response response = rest.get("b1", "key1", body);
 
     //THEN
     OzoneInputStream ozoneInputStream =
@@ -78,5 +80,9 @@ public class TestObjectGet {
         IOUtils.toString(ozoneInputStream, Charset.forName("UTF-8"));
 
     Assert.assertEquals(CONTENT, keyContent);
+
+    DateTimeFormatter.RFC_1123_DATE_TIME
+        .parse(response.getHeaderString("Last-Modified"));
+
   }
 }

+ 44 - 0
hadoop-ozone/s3gateway/src/test/java/org/apache/hadoop/ozone/s3/util/TestRFC1123Util.java

@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+package org.apache.hadoop.ozone.s3.util;
+
+import java.time.temporal.TemporalAccessor;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test for RFC1123 util.
+ */
+public class TestRFC1123Util {
+
+  @Test
+  public void parse() {
+    //one digit day
+    String dateStr = "Mon, 5 Nov 2018 15:04:05 GMT";
+
+    TemporalAccessor date = RFC1123Util.FORMAT.parse(dateStr);
+
+    String formatted = RFC1123Util.FORMAT.format(date);
+
+    //two digits day
+    Assert.assertEquals("Mon, 05 Nov 2018 15:04:05 GMT", formatted);
+
+  }
+}