You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/08/23 12:34:17 UTC

[GitHub] [iceberg] zhangdove opened a new pull request #1368: add date functions with zone offset parameter

zhangdove opened a new pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368


   As discussed earlier in #1355 , I tried to redo a PR.
   This PR is mainly to solve the problem that different users processing data in different time zones may cause some exceptions to write data due to the default time zone of the system. A time zone parameter has been added to the time function so that users in different time zones can use the time partition normally.
   
   cc @rdblue @RussellSpitzer , I'd appreciate if you could review it at your convenience.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#issuecomment-692968192


   @zhangdove, I brought up this PR at the last Iceberg sync to talk through the possible effects with the broader community. The general consensus was that it isn't a good idea to parameterize the date/time transforms. The main concern was that this would appear to be correct, but daylight savings time would unexpectedly result in a different required offset and we would have the same problem.
   
   We came up with a few alternatives:
   * Use hourly partitioning to ensure you can drop data at any hour boundary
   * Add the ability to register custom partition functions
   * Use v2 tables and delete files to avoid the need to align deletes or overwrites with a partitions. This option seems like the best in the long term, but it isn't very helpful right now
   
   I think that those are good alternatives to parameterizing these functions. What do you think? Would any of those work? The last two options will require some work, but I think would be better solutions in the long term. If you want to discuss this more, maybe you could start a thread on the dev list to discuss any issues that you see with these options. I think it would be great to have a wider discussion on this, since I found the comments during the sync to be valuable.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r476848415



##########
File path: api/src/main/java/org/apache/iceberg/PartitionSpec.java
##########
@@ -380,58 +381,74 @@ public Builder identity(String sourceName) {
       return identity(sourceName, sourceName);
     }
 
-    public Builder year(String sourceName, String targetName) {
+    public Builder year(String sourceName, String targetName, ZoneOffset zoneOffset) {

Review comment:
       I would rather not expose Java's time library in Iceberg's API. Could we use a string instead?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r477444302



##########
File path: api/src/main/java/org/apache/iceberg/transforms/TimestampTransform.java
##########
@@ -28,22 +28,47 @@
 import org.apache.iceberg.expressions.Expression;
 import org.apache.iceberg.expressions.Expressions;
 import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
 import org.apache.iceberg.types.Type;
 import org.apache.iceberg.types.Types;
 
-enum Timestamps implements Transform<Long, Integer> {
-  YEAR(ChronoUnit.YEARS, "year"),
-  MONTH(ChronoUnit.MONTHS, "month"),
-  DAY(ChronoUnit.DAYS, "day"),
-  HOUR(ChronoUnit.HOURS, "hour");
+abstract class TimestampTransform implements Transform<Long, Integer> {
 
   private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private final ChronoUnit granularity;
-  private final String name;
 
-  Timestamps(ChronoUnit granularity, String name) {
+  private ChronoUnit granularity;
+  private String name;
+  private ZoneOffset zoneOffset;

Review comment:
       I think these should be final so that the transforms are immutable. Also, can you move them below the static fields and methods?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangdove commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
zhangdove commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r477143664



##########
File path: api/src/main/java/org/apache/iceberg/transforms/Timestamps.java
##########
@@ -127,6 +133,6 @@ public String toHumanString(Integer value) {
 
   @Override
   public String toString() {
-    return name;
+    return name + "[" + zoneOffset.getTotalSeconds() + "]";

Review comment:
       This way look good to understand,I will change it.
   
   `toString` return `name`, When the value of `zoneOffset.getTotalSeconds()  ` is zero . This will also be compatible with the historical version.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#issuecomment-694485109


   > Would it also be abnormal for users in two different time zones to write data to the same iceberg table(created by day partition)?
   
   All timestamps are passed to Iceberg using a fixed representation. For timestamp with time zone, values are converted to UTC and passed to Iceberg as micros from epoch. All values, regardless of the SQL session time zone are passed this way. So it doesn't matter to Iceberg what the SQL session time zone was. It just has to handle UTC timestamp values.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r477448237



##########
File path: api/src/main/java/org/apache/iceberg/transforms/TimestampTransform.java
##########
@@ -28,22 +28,47 @@
 import org.apache.iceberg.expressions.Expression;
 import org.apache.iceberg.expressions.Expressions;
 import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
 import org.apache.iceberg.types.Type;
 import org.apache.iceberg.types.Types;
 
-enum Timestamps implements Transform<Long, Integer> {
-  YEAR(ChronoUnit.YEARS, "year"),
-  MONTH(ChronoUnit.MONTHS, "month"),
-  DAY(ChronoUnit.DAYS, "day"),
-  HOUR(ChronoUnit.HOURS, "hour");
+abstract class TimestampTransform implements Transform<Long, Integer> {
 
   private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private final ChronoUnit granularity;
-  private final String name;
 
-  Timestamps(ChronoUnit granularity, String name) {
+  private ChronoUnit granularity;
+  private String name;
+  private ZoneOffset zoneOffset;
+
+  @SuppressWarnings("unchecked")
+  static TimestampTransform get(Type type, String name) {
+    if (type.typeId() == Type.TypeID.TIMESTAMP) {
+      switch (name.toUpperCase()) {
+        case "YEAR":
+          return new TimestampTransform.TimestampYear(name.toLowerCase());

Review comment:
       Also, this should use `Locale.ENGLISH` or `Locale.ROOT` instead of the default locale.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangdove closed pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
zhangdove closed pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangdove commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
zhangdove commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r478137863



##########
File path: api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java
##########
@@ -55,17 +56,24 @@
       } else if (transform instanceof Truncate) {
         results.add(visitor.truncate(sourceName, field.sourceId(),
             ((Truncate<?>) transform).width()));
-      } else if (transform == Dates.YEAR || transform == Timestamps.YEAR) {
+      } else if (transform == Dates.YEAR || isTimestamp(transform, ChronoUnit.YEARS)) {

Review comment:
       So far I don't have a good idea about this. I just modify the reference to timestamps accordingly.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangdove commented on pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
zhangdove commented on pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#issuecomment-678780832


   It's frustrating, Java8 compiles and Java11 fails.
   Uhh..... My local environment is jdk1.8. Later I will change a jdk1.11 environment locally to debug the environment.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangdove commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
zhangdove commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r478141385



##########
File path: api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java
##########
@@ -55,17 +56,24 @@
       } else if (transform instanceof Truncate) {
         results.add(visitor.truncate(sourceName, field.sourceId(),
             ((Truncate<?>) transform).width()));
-      } else if (transform == Dates.YEAR || transform == Timestamps.YEAR) {
+      } else if (transform == Dates.YEAR || isTimestamp(transform, ChronoUnit.YEARS)) {

Review comment:
       After modification,`timeZone` is already `final`.
   Is there any validation work that needs to be done here?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangdove commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
zhangdove commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r477143269



##########
File path: api/src/main/java/org/apache/iceberg/transforms/Timestamps.java
##########
@@ -54,12 +55,17 @@ public Integer apply(Long timestampMicros) {
 
     // discards fractional seconds, not needed for calculation
     OffsetDateTime timestamp = Instant
-        .ofEpochSecond(timestampMicros / 1_000_000)
+        .ofEpochSecond(timestampMicros / 1_000_000 + zoneOffset.getTotalSeconds())
         .atOffset(ZoneOffset.UTC);
 
     return (int) granularity.between(EPOCH, timestamp);
   }
 
+  public Timestamps zoneOffset(ZoneOffset newZoneOffset) {
+    this.zoneOffset = newZoneOffset;

Review comment:
       >This changes the zone offset for all uses of the given granularity in this JVM and can unexpectedly change the behavior of another table. I think it would be a serious correctness bug.
   
   The variable zoneOffset is not a global static variable,I'm not sure why it affects the other tables. 
   But I may have realized that enumerating classes is not really a good way to use them.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r477442782



##########
File path: api/src/main/java/org/apache/iceberg/PartitionSpec.java
##########
@@ -380,58 +381,78 @@ public Builder identity(String sourceName) {
       return identity(sourceName, sourceName);
     }
 
-    public Builder year(String sourceName, String targetName) {
+    public Builder year(String sourceName, String targetName, String offsetId) {
       checkAndAddPartitionName(targetName);
       Types.NestedField sourceColumn = findSourceColumn(sourceName);
       PartitionField field = new PartitionField(
-          sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.year(sourceColumn.type()));
+          sourceColumn.fieldId(), nextFieldId(), targetName,
+          Transforms.year(sourceColumn.type(), ZoneOffset.of(offsetId)));
       checkForRedundantPartitions(field);
       fields.add(field);
       return this;
     }
 
+    public Builder year(String sourceName, String targetName) {
+      return year(sourceName, targetName, "Z");
+    }
+
     public Builder year(String sourceName) {
       return year(sourceName, sourceName + "_year");
     }
 
-    public Builder month(String sourceName, String targetName) {
+    public Builder month(String sourceName, String targetName, String offsetId) {
       checkAndAddPartitionName(targetName);
       Types.NestedField sourceColumn = findSourceColumn(sourceName);
       PartitionField field = new PartitionField(
-          sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.month(sourceColumn.type()));
+          sourceColumn.fieldId(), nextFieldId(), targetName,
+          Transforms.month(sourceColumn.type(), ZoneOffset.of(offsetId)));

Review comment:
       `ZoneOffset.of` supports a broader range of values than Iceberg (which wouldn't support `Z`, for example). I'm a bit concerned that using it here erases what the user passed in. What about keeping the string and making `ZoneOffset` internal to `TimestampTransform`? And here, we could validate the string format: `(?+|-)\d\d:\d\d`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r476854186



##########
File path: api/src/main/java/org/apache/iceberg/transforms/Transforms.java
##########
@@ -94,14 +110,18 @@ private Transforms() {
    */
   @SuppressWarnings("unchecked")
   public static <T> Transform<T, Integer> year(Type type) {
+    return year(type, ZoneOffset.UTC);
+  }
+
+  public static <T> Transform<T, Integer> year(Type type, ZoneOffset zoneOffset) {

Review comment:
       The cases for `date` need to assert that the offset is either null or UTC. I don't think it makes sense for a date to use the offset, so if there is an offset it should be rejected and not ignored.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r477439181



##########
File path: api/src/main/java/org/apache/iceberg/transforms/TimestampTransform.java
##########
@@ -28,22 +28,47 @@
 import org.apache.iceberg.expressions.Expression;
 import org.apache.iceberg.expressions.Expressions;
 import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
 import org.apache.iceberg.types.Type;
 import org.apache.iceberg.types.Types;
 
-enum Timestamps implements Transform<Long, Integer> {
-  YEAR(ChronoUnit.YEARS, "year"),
-  MONTH(ChronoUnit.MONTHS, "month"),
-  DAY(ChronoUnit.DAYS, "day"),
-  HOUR(ChronoUnit.HOURS, "hour");
+abstract class TimestampTransform implements Transform<Long, Integer> {
 
   private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private final ChronoUnit granularity;
-  private final String name;
 
-  Timestamps(ChronoUnit granularity, String name) {
+  private ChronoUnit granularity;
+  private String name;
+  private ZoneOffset zoneOffset;
+
+  @SuppressWarnings("unchecked")
+  static TimestampTransform get(Type type, String name) {
+    if (type.typeId() == Type.TypeID.TIMESTAMP) {
+      switch (name.toUpperCase()) {
+        case "YEAR":
+          return new TimestampTransform.TimestampYear(name.toLowerCase());
+        case "MONTH":
+          return new TimestampTransform.TimestampMonth(name.toLowerCase());
+        case "DAY":
+          return new TimestampTransform.TimestampDay(name.toLowerCase());
+        case "HOUR":
+          return new TimestampTransform.TimestampHour(name.toLowerCase());
+        default:
+          throw new UnsupportedOperationException("Unsupported timestamp method: " + name);
+      }
+    }
+    throw new UnsupportedOperationException(
+        "TimestampTransform cannot transform type: " + type);
+  }
+
+  private TimestampTransform(ChronoUnit granularity, String name) {
     this.granularity = granularity;
     this.name = name;
+    this.zoneOffset = ZoneOffset.UTC;
+  }
+
+  public TimestampTransform zoneOffset(ZoneOffset newZoneOffset) {
+    this.zoneOffset = newZoneOffset;

Review comment:
       Why is `zoneOffset` mutable?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r476841037



##########
File path: api/src/main/java/org/apache/iceberg/transforms/Timestamps.java
##########
@@ -127,6 +133,6 @@ public String toHumanString(Integer value) {
 
   @Override
   public String toString() {
-    return name;
+    return name + "[" + zoneOffset.getTotalSeconds() + "]";

Review comment:
       Is it possible to use `+HH:mm` or `-HH:mm` instead?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r477442782



##########
File path: api/src/main/java/org/apache/iceberg/PartitionSpec.java
##########
@@ -380,58 +381,78 @@ public Builder identity(String sourceName) {
       return identity(sourceName, sourceName);
     }
 
-    public Builder year(String sourceName, String targetName) {
+    public Builder year(String sourceName, String targetName, String offsetId) {
       checkAndAddPartitionName(targetName);
       Types.NestedField sourceColumn = findSourceColumn(sourceName);
       PartitionField field = new PartitionField(
-          sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.year(sourceColumn.type()));
+          sourceColumn.fieldId(), nextFieldId(), targetName,
+          Transforms.year(sourceColumn.type(), ZoneOffset.of(offsetId)));
       checkForRedundantPartitions(field);
       fields.add(field);
       return this;
     }
 
+    public Builder year(String sourceName, String targetName) {
+      return year(sourceName, targetName, "Z");
+    }
+
     public Builder year(String sourceName) {
       return year(sourceName, sourceName + "_year");
     }
 
-    public Builder month(String sourceName, String targetName) {
+    public Builder month(String sourceName, String targetName, String offsetId) {
       checkAndAddPartitionName(targetName);
       Types.NestedField sourceColumn = findSourceColumn(sourceName);
       PartitionField field = new PartitionField(
-          sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.month(sourceColumn.type()));
+          sourceColumn.fieldId(), nextFieldId(), targetName,
+          Transforms.month(sourceColumn.type(), ZoneOffset.of(offsetId)));

Review comment:
       `ZoneOffset.of` supports a broader range of values than Iceberg (which wouldn't support `Z`, for example). I'm a bit concerned that using it here erases what the user passed in. What about keeping the string and making `ZoneOffset` internal to `TimestampTransform`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r477444758



##########
File path: api/src/main/java/org/apache/iceberg/transforms/TimestampTransform.java
##########
@@ -127,6 +152,57 @@ public String toHumanString(Integer value) {
 
   @Override
   public String toString() {
-    return name;
+    if (zoneOffset.getTotalSeconds() == 0) {
+      return name;
+    } else {
+      return name + "[" + zoneOffset.getId() + "]";
+    }
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (this == other) {
+      return true;
+    }
+    if (other == null || getClass() != other.getClass()) {
+      return false;
+    }
+    TimestampTransform that = (TimestampTransform) other;
+    return granularity == that.granularity &&
+        Objects.equal(name, that.name) &&
+        Objects.equal(zoneOffset, that.zoneOffset);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hashCode(granularity, name, zoneOffset);
+  }
+
+  public ChronoUnit getGranularity() {

Review comment:
       In Iceberg, we don't use `get` because it doesn't add value. Can you change this to `granularity`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r476858595



##########
File path: api/src/main/java/org/apache/iceberg/transforms/Timestamps.java
##########
@@ -54,12 +55,17 @@ public Integer apply(Long timestampMicros) {
 
     // discards fractional seconds, not needed for calculation
     OffsetDateTime timestamp = Instant
-        .ofEpochSecond(timestampMicros / 1_000_000)
+        .ofEpochSecond(timestampMicros / 1_000_000 + zoneOffset.getTotalSeconds())
         .atOffset(ZoneOffset.UTC);
 
     return (int) granularity.between(EPOCH, timestamp);
   }
 
+  public Timestamps zoneOffset(ZoneOffset newZoneOffset) {
+    this.zoneOffset = newZoneOffset;

Review comment:
       This changes the zone offset for all uses of the given granularity in this JVM and can unexpectedly change the behavior of another table. I think it would be a serious correctness bug.
   
   Instead, I think this PR needs to create a new `TimestampTransform` class like the `BucketTransform` class and remove old references to this enum.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r477443922



##########
File path: api/src/main/java/org/apache/iceberg/transforms/TimestampTransform.java
##########
@@ -28,22 +28,47 @@
 import org.apache.iceberg.expressions.Expression;
 import org.apache.iceberg.expressions.Expressions;
 import org.apache.iceberg.expressions.UnboundPredicate;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
 import org.apache.iceberg.types.Type;
 import org.apache.iceberg.types.Types;
 
-enum Timestamps implements Transform<Long, Integer> {
-  YEAR(ChronoUnit.YEARS, "year"),
-  MONTH(ChronoUnit.MONTHS, "month"),
-  DAY(ChronoUnit.DAYS, "day"),
-  HOUR(ChronoUnit.HOURS, "hour");
+abstract class TimestampTransform implements Transform<Long, Integer> {
 
   private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private final ChronoUnit granularity;
-  private final String name;
 
-  Timestamps(ChronoUnit granularity, String name) {
+  private ChronoUnit granularity;
+  private String name;
+  private ZoneOffset zoneOffset;
+
+  @SuppressWarnings("unchecked")
+  static TimestampTransform get(Type type, String name) {
+    if (type.typeId() == Type.TypeID.TIMESTAMP) {
+      switch (name.toUpperCase()) {
+        case "YEAR":
+          return new TimestampTransform.TimestampYear(name.toLowerCase());

Review comment:
       It's strange to me that this converts name to both upper and lower case. Why not convert to lower case once, use the variable in the `switch` statement, and pass the variable here as well?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r476851581



##########
File path: api/src/main/java/org/apache/iceberg/transforms/Transforms.java
##########
@@ -40,6 +41,7 @@ private Transforms() {
   }
 
   private static final Pattern HAS_WIDTH = Pattern.compile("(\\w+)\\[(\\d+)\\]");
+  private static final Pattern HAS_TIME_OFFSET = Pattern.compile("(\\w+)\\[(\\-?\\d+)\\]");

Review comment:
       The pattern should require either `-` or `+`, and should also require 2-digit values for hour and minute. Using hour/minute instead of seconds makes this much easier to read and work with for users.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r477445928



##########
File path: api/src/main/java/org/apache/iceberg/transforms/Transforms.java
##########
@@ -192,4 +230,10 @@ private Transforms() {
   public static <T> Transform<T, Void> alwaysNull() {
     return VoidTransform.get();
   }
+
+  private static void checkZoneOffsetIsUTC(ZoneOffset zoneOffset) {
+    Preconditions.checkArgument(zoneOffset == null || zoneOffset.getTotalSeconds() == 0,
+        "Expect zone offset is null or UTC, but is", zoneOffset);

Review comment:
       The offset should always be null because it is invalid to call `day` with a date field and a zone offset. Even if the offset is UTC, it is not allowed.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangdove commented on pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
zhangdove commented on pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#issuecomment-693140815


   @rdblue Thanks for your replay. I also noticed that you recently mentioned iceberg sync in the dev list.
   
   > but daylight savings time would unexpectedly result in a different required offset and we would have the same problem.
   
   I just want to make sure. Would it also be abnormal for users in two different time zones to write data to the same iceberg table(created by day partition)? it's so bad and it does seem that way.
   
   I thought about it for a moment, and I couldn't think of some application scenario tha users in two different time zones to write data to same iceberg table(created by day partition).Perhaps we can suggest that different users write their data into different iceberg tables? Just some simple ideas of mine.
   
   I'm not particularly sure if the second solution will solve the problem, but I'd rather see the third solution implemented by the community.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangdove commented on pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
zhangdove commented on pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#issuecomment-694595353


   I close this PR. And look forward to new features soon about the third solution.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r476855177



##########
File path: api/src/test/java/org/apache/iceberg/expressions/TestExpressionHelpers.java
##########
@@ -137,16 +137,16 @@ public void testRewriteNot() {
   @Test
   public void testTransformExpressions() {
     Assert.assertEquals("Should produce the correct expression string",
-        "year(ref(name=\"ts\")) == \"2019\"",
+        "year[0](ref(name=\"ts\")) == \"2019\"",

Review comment:
       UTC should continue to produce the old names for compatibility. Otherwise, this will break old readers.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangdove commented on pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
zhangdove commented on pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#issuecomment-680765083


   @rdblue Thank you very much for your many good Suggestions. I have finished the modification.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r477447101



##########
File path: api/src/main/java/org/apache/iceberg/transforms/Transforms.java
##########
@@ -57,10 +60,26 @@ private Transforms() {
       return Identity.get(type);
     }
 
+    if (type.typeId() == Type.TypeID.TIMESTAMP) {
+      try {
+        Matcher timeZoneOffset = HAS_TIME_OFFSET.matcher(transform);
+        if (timeZoneOffset.matches()) {
+          String name = timeZoneOffset.group(1);
+          String offsetId = timeZoneOffset.group(2);
+          return TimestampTransform.get(type, name.toLowerCase(Locale.ENGLISH))

Review comment:
       `TimestampTransform.get` is responsible for handling mixed case, so there is no need to call `toLowerCase` here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r476855472



##########
File path: api/src/main/java/org/apache/iceberg/transforms/Transforms.java
##########
@@ -154,9 +182,13 @@ private Transforms() {
    */
   @SuppressWarnings("unchecked")
   public static <T> Transform<T, Integer> hour(Type type) {
+    return hour(type, ZoneOffset.UTC);
+  }
+
+  public static <T> Transform<T, Integer> hour(Type type, ZoneOffset zoneOffset) {
     Preconditions.checkArgument(type.typeId() == Type.TypeID.TIMESTAMP,
-        "Cannot partition type %s by hour", type);
-    return (Transform<T, Integer>) Timestamps.HOUR;
+            "Cannot partition type %s by hour", type);

Review comment:
       Nit: This is an unnecessary change to indentation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r477443440



##########
File path: api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java
##########
@@ -55,17 +56,24 @@
       } else if (transform instanceof Truncate) {
         results.add(visitor.truncate(sourceName, field.sourceId(),
             ((Truncate<?>) transform).width()));
-      } else if (transform == Dates.YEAR || transform == Timestamps.YEAR) {
+      } else if (transform == Dates.YEAR || isTimestamp(transform, ChronoUnit.YEARS)) {

Review comment:
       Will users of the visitor not need to know the zone offset?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] zhangdove commented on a change in pull request #1368: add date functions with zone offset parameter

Posted by GitBox <gi...@apache.org>.
zhangdove commented on a change in pull request #1368:
URL: https://github.com/apache/iceberg/pull/1368#discussion_r475228405



##########
File path: api/src/main/java/org/apache/iceberg/transforms/Transforms.java
##########
@@ -57,10 +59,22 @@ private Transforms() {
       return Identity.get(type);
     }
 
+    Matcher timeZoneOffset = HAS_TIME_OFFSET.matcher(transform);
+    if (timeZoneOffset.matches()) {

Review comment:
       Seems incompatible with the older partition.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org