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/04/18 14:47:05 UTC

[GitHub] [incubator-iceberg] rdsr opened a new pull request #939: [WIP] Support UpdateEvent for Update Operations

rdsr opened a new pull request #939: [WIP] Support UpdateEvent for Update Operations
URL: https://github.com/apache/incubator-iceberg/pull/939
 
 
   Want feedback on the approach
   
   TODO
   1. Haven't added all events, want to discuss approach.
   2. Some tests are failing, since `snapshotOperation` is null

----------------------------------------------------------------
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


With regards,
Apache Git Services

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


[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r411598482



##########
File path: core/src/main/java/org/apache/iceberg/events/CreateSnapshotEvent.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.iceberg.events;
+
+import java.util.Map;
+
+public final class CreateSnapshotEvent implements UpdateEvent {
+  private final String operation;
+  private final String tablePath;
+  private final Map<String, String> summary;
+  private final Iterable<String> addedFiles;
+  private final Iterable<String> appendedManifests;
+  private final Iterable<String> deletePaths;
+
+  public CreateSnapshotEvent(
+      String operation, String tablePath, Map<String, String> summary,
+      Iterable<String> addedFiles, Iterable<String> appendedManifests, Iterable<String> deletePaths) {

Review comment:
       Maybe I can pass in the `name` as a property which can reside in table metadata. I can access that metadata during commit and I don't have to add any new api in `TableOperations` ?




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/events/CreateSnapshotEvent.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.iceberg.events;
+
+import java.util.List;
+import java.util.Map;
+
+
+public final class CreateSnapshotEvent {
+  private final String tableName;
+  private final String operation;
+  private final long snapshotId;
+  private final long sequenceNumber;
+  private final Map<String, String> summary;
+  private final List<String> addedFiles;
+  private final List<String> appendedManifests;
+  private final List<String> deletePaths;
+
+  public CreateSnapshotEvent(
+      String tableName, String operation, long snapshotId, long sequenceNumber, Map<String, String> summary,
+      List<String> addedFiles, List<String> appendedManifests, List<String> deletePaths) {
+    this.tableName = tableName;
+    this.operation = operation;
+    this.snapshotId = snapshotId;
+    this.sequenceNumber = sequenceNumber;
+    this.summary = summary;
+    this.addedFiles = addedFiles;
+    this.appendedManifests = appendedManifests;
+    this.deletePaths = deletePaths;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public String operation() {
+    return operation;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public long sequenceNumber() {
+    return sequenceNumber;
+  }
+
+  public Map<String, String> summary() {
+    return summary;
+  }
+
+  public List<String> addedFiles() {
+    return addedFiles;
+  }
+
+  public List<String> appendedManifests() {
+    return appendedManifests;
+  }
+
+  public List<String> deletePaths() {

Review comment:
       Minor: the other methods use past tense and refer to data files as "files", so this would be `deletedFiles()` right?




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r411579807



##########
File path: api/src/main/java/org/apache/iceberg/events/UpdateEvent.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.iceberg.events;
+
+public interface UpdateEvent {

Review comment:
       ~~I was thinking that there may be some update operations where commit is being~~ ~~performed in their superclass. In those cases, we have to get the sub class' update~~
   ~~event. Seeing through the code again, I don't see this and where a commit it being~~ ~~performed in a class it has all the information to send the event.~~
   ~~I'll delete this event interface~~




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#issuecomment-620359060


   @rdblue . Thanks for the helpful review. I've address your comments and also added events for FastAppends and SnapshotsManager


----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/FastAppend.java
##########
@@ -49,7 +49,8 @@
   private ManifestFile newManifest = null;
   private boolean hasNewFiles = false;
 
-  FastAppend(TableOperations ops) {
+  FastAppend(String tableName, TableOperations ops) {
+    //TODO we use tableName to publish notification

Review comment:
       I think the append event should be created in this PR if we are sending events from `MergingSnapshotProducer`. I don't think it is necessary to implement events for `SnapshotManager` yet, though.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr edited a comment on pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr edited a comment on pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#issuecomment-620119288


   > I think the file lists in the event might be incomplete. I'd suggest removing them and just sending the summary, unless you have a use case for them.
   
   I thin sending the summary is fine by me.


----------------------------------------------------------------
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] [incubator-iceberg] rdsr edited a comment on pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr edited a comment on pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#issuecomment-620119288


   > I think the file lists in the event might be incomplete. I'd suggest removing them and just sending the summary, unless you have a use case for them.
   
   I thin sending the summary is fine by me. I'll add the following information in our create snapshot event - tableName,  operation,  snapshotId,  sequenceNumber and  summary


----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/events/CreateSnapshotEvent.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.iceberg.events;
+
+import java.util.List;
+import java.util.Map;
+
+
+public final class CreateSnapshotEvent {
+  private final String tableName;
+  private final String operation;
+  private final long snapshotId;
+  private final long sequenceNumber;
+  private final Map<String, String> summary;
+  private final List<String> addedFiles;
+  private final List<String> deleteFiles;
+
+  public CreateSnapshotEvent(
+      String tableName, String operation, long snapshotId, long sequenceNumber,
+      Map<String, String> summary, List<String> addedFiles, List<String> deleteFiles) {
+    this.tableName = tableName;
+    this.operation = operation;
+    this.snapshotId = snapshotId;
+    this.sequenceNumber = sequenceNumber;
+    this.summary = summary;
+    this.addedFiles = addedFiles;
+    this.deleteFiles = deleteFiles;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public String operation() {
+    return operation;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public long sequenceNumber() {
+    return sequenceNumber;
+  }
+
+  public Map<String, String> summary() {
+    return summary;
+  }
+
+  public List<String> addedFiles() {

Review comment:
       What happens if manifests are added? Is this incomplete?




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r419564231



##########
File path: core/src/main/java/org/apache/iceberg/FastAppend.java
##########
@@ -141,6 +144,18 @@ private ManifestFile copyManifest(ManifestFile manifest) {
     return newManifests;
   }
 
+  @Override
+  public Object updateEvent() {
+    long snapshotId = snapshotId();
+    long sequenceNumber = ops.refresh().snapshot(snapshotId).sequenceNumber();

Review comment:
       fixed




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r411579807



##########
File path: api/src/main/java/org/apache/iceberg/events/UpdateEvent.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.iceberg.events;
+
+public interface UpdateEvent {

Review comment:
       I was thinking that there may be some update operations where commit is being performed in their superclass. In those cases, we have to get the sub class' update event. Seeing through the code again, I don't see this and where a commit it being performed in a class it has all the information to send the event.  
   
   I'll delete this event interface




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotManager.java
##########
@@ -127,6 +127,32 @@ public ManageSnapshots rollbackTo(long snapshotId) {
     return setCurrentSnapshot(snapshotId);
   }
 
+  @Override
+  public Object updateEvent() {
+    if (targetSnapshotId == null) {
+      // NOOP operation, no snapshot created
+      return null;
+    }
+
+    switch (managerOperation) {
+      case ROLLBACK:
+        // rollback does not create a new snapshot

Review comment:
       Maybe a history modification event with a type that could be "rollback", "fast-forward", "cherry-pick", or "set-current"?




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/events/CreateSnapshotEvent.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.iceberg.events;
+
+import java.util.Map;
+
+public final class CreateSnapshotEvent implements UpdateEvent {
+  private final String operation;
+  private final String tablePath;
+  private final Map<String, String> summary;
+  private final Iterable<String> addedFiles;
+  private final Iterable<String> appendedManifests;
+  private final Iterable<String> deletePaths;
+
+  public CreateSnapshotEvent(
+      String operation, String tablePath, Map<String, String> summary,
+      Iterable<String> addedFiles, Iterable<String> appendedManifests, Iterable<String> deletePaths) {

Review comment:
       Table creates all of these operations, so it can pass in its name as well as `ops`. I'd prefer that to trying to parse a location.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr edited a comment on pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr edited a comment on pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#issuecomment-620119288


   > I think the file lists in the event might be incomplete. I'd suggest removing them and just sending the summary, unless you have a use case for them.
   
   I think sending the summary is fine by me. I'll add the following information in our create snapshot event - tableName,  operation,  snapshotId,  sequenceNumber and  summary


----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/events/CreateSnapshotEvent.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.iceberg.events;
+
+import java.util.Map;
+
+public final class CreateSnapshotEvent implements UpdateEvent {
+  private final String operation;
+  private final String tablePath;
+  private final Map<String, String> summary;
+  private final Iterable<String> addedFiles;
+  private final Iterable<String> appendedManifests;
+  private final Iterable<String> deletePaths;
+
+  public CreateSnapshotEvent(
+      String operation, String tablePath, Map<String, String> summary,
+      Iterable<String> addedFiles, Iterable<String> appendedManifests, Iterable<String> deletePaths) {

Review comment:
       Why use table path/location instead of name? I think we should send a `name` field with whatever was used to load the table. We might need to add `name` to the `Table` interface to do that, but adding that method is easier than adding something to `TableOperations` because all the implementations are internal.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r411574034



##########
File path: core/src/main/java/org/apache/iceberg/events/CreateSnapshotEvent.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.iceberg.events;
+
+import java.util.Map;
+
+public final class CreateSnapshotEvent implements UpdateEvent {
+  private final String operation;
+  private final String tablePath;
+  private final Map<String, String> summary;
+  private final Iterable<String> addedFiles;
+  private final Iterable<String> appendedManifests;
+  private final Iterable<String> deletePaths;
+
+  public CreateSnapshotEvent(
+      String operation, String tablePath, Map<String, String> summary,
+      Iterable<String> addedFiles, Iterable<String> appendedManifests, Iterable<String> deletePaths) {

Review comment:
       I think we might need a reference to `Table` . Where we call commit and publish events we only have access to `TableOperations` . Any thoughts on how to get the `name` during `commits`?




----------------------------------------------------------------
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] [incubator-iceberg] rdsr edited a comment on pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr edited a comment on pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#issuecomment-620359060


   @rdblue . Thanks for the helpful review. I've addressed your comments and also added events for FastAppends and SnapshotsManager


----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java
##########
@@ -25,8 +25,8 @@
 
 public class BaseReplacePartitions
     extends MergingSnapshotProducer<ReplacePartitions> implements ReplacePartitions {
-  BaseReplacePartitions(TableOperations ops) {
-    super(ops);
+  BaseReplacePartitions(String tableName, TableOperations ops) {
+    super(tableName, ops);

Review comment:
       Nit: Other places use `name`.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#issuecomment-619445945


   This is blocked on #968


----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r415997256



##########
File path: core/src/main/java/org/apache/iceberg/BaseTable.java
##########
@@ -136,37 +136,37 @@ public RewriteManifests rewriteManifests() {
 
   @Override
   public OverwriteFiles newOverwrite() {
-    return new BaseOverwriteFiles(ops);
+    return new BaseOverwriteFiles(name, ops);
   }
 
   @Override
   public ReplacePartitions newReplacePartitions() {
-    return new BaseReplacePartitions(ops);
+    return new BaseReplacePartitions(name, ops);
   }
 
   @Override
   public DeleteFiles newDelete() {
-    return new StreamingDelete(ops);
+    return new StreamingDelete(name, ops);
   }
 
   @Override
-  public ExpireSnapshots expireSnapshots() {
+  public RemoveSnapshots expireSnapshots() {

Review comment:
       Fixed. My bad.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr edited a comment on issue #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr edited a comment on issue #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#issuecomment-617935315


   @rdblue . Sorry I think this PR was published by mistake. This is not ready for review yet [Thought the final version would be somewhat similar]


----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r419553134



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotManager.java
##########
@@ -127,6 +127,32 @@ public ManageSnapshots rollbackTo(long snapshotId) {
     return setCurrentSnapshot(snapshotId);
   }
 
+  @Override
+  public Object updateEvent() {
+    if (targetSnapshotId == null) {
+      // NOOP operation, no snapshot created
+      return null;
+    }
+
+    switch (managerOperation) {
+      case ROLLBACK:
+        // rollback does not create a new snapshot

Review comment:
       Yes. I'll create a followup for this. I presume this would be a specific Rollback event?




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r411571438



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,6 +294,8 @@ public void commit() {
     } catch (RuntimeException e) {
       LOG.warn("Failed to load committed table metadata, skipping manifest clean-up", e);
     }
+
+    Listeners.notifyAll(updateEvent());

Review comment:
       Makes sense




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotProducer.java
##########
@@ -293,6 +294,8 @@ public void commit() {
     } catch (RuntimeException e) {
       LOG.warn("Failed to load committed table metadata, skipping manifest clean-up", e);
     }
+
+    Listeners.notifyAll(updateEvent());

Review comment:
       This needs to be in a `try`/`catch` block to ensure that no exceptions are thrown. It is important for correctness that this returns successfully if the commit was successful. If the commit succeeds, but this throws an exception, then higher-level retries will re-attempt the operation on the same data and produce duplicates.
   
   Also, if this is sent after commit, then it should include the sequence number that was committed.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r416003747



##########
File path: core/src/main/java/org/apache/iceberg/FastAppend.java
##########
@@ -49,7 +49,8 @@
   private ManifestFile newManifest = null;
   private boolean hasNewFiles = false;
 
-  FastAppend(TableOperations ops) {
+  FastAppend(String tableName, TableOperations ops) {
+    //TODO we use tableName to publish notification

Review comment:
       Scratch that. I think this can be supported. I'll do that as part of this pr




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java
##########
@@ -317,6 +321,22 @@ private ManifestFile copyManifest(ManifestFile manifest) {
     }
   }
 
+  @Override
+  public Object updateEvent() {
+    long snapshotId = snapshotId();
+    //long sequenceNumber = ops.refresh().snapshot(snapshotId).sequenceNumber();

Review comment:
       Why comment this out? Looks correct to me.




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/FastAppend.java
##########
@@ -141,6 +144,18 @@ private ManifestFile copyManifest(ManifestFile manifest) {
     return newManifests;
   }
 
+  @Override
+  public Object updateEvent() {
+    long snapshotId = snapshotId();
+    long sequenceNumber = ops.refresh().snapshot(snapshotId).sequenceNumber();

Review comment:
       The commit is going to update the current metadata, so there shouldn't be a need to `refresh` here. You'll get the correct information using `current`.

##########
File path: core/src/main/java/org/apache/iceberg/FastAppend.java
##########
@@ -141,6 +144,18 @@ private ManifestFile copyManifest(ManifestFile manifest) {
     return newManifests;
   }
 
+  @Override
+  public Object updateEvent() {
+    long snapshotId = snapshotId();
+    long sequenceNumber = ops.refresh().snapshot(snapshotId).sequenceNumber();

Review comment:
       The commit is going to update the current metadata, so there shouldn't be a need to `refresh` here. You'll get the correct information using `current`, with less chance of failure.




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
##########
@@ -93,7 +93,7 @@ public Transaction newCreateTableTransaction(
     String baseLocation = location != null ? location : defaultWarehouseLocation(identifier);
     Map<String, String> tableProperties = properties != null ? properties : Maps.newHashMap();
     TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, baseLocation, tableProperties);
-    return Transactions.createTableTransaction(ops, metadata);
+    return Transactions.createTableTransaction(identifier.name(), ops, metadata);

Review comment:
       Did you mean to use `identifier.toString` 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] [incubator-iceberg] rdblue commented on pull request #939: Support UpdateEvent for Update Operations

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


   Thanks @rdsr, this looks mostly good, although I think the file lists in the event might be incomplete. I'd suggest removing them and just sending the summary, unless you have a use case for them.
   
   I'd also like to get fast appends and snapshot manager changes done as soon as we can -- ideally in this PR. That way, we can guarantee that all changes to table contents produce events. Events aren't as useful unless you know you're getting all of the changes.


----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/TableOperations.java
##########
@@ -30,6 +30,8 @@
  */
 public interface TableOperations {
 
+  String tablePath();

Review comment:
       Why not use `table.location()`? You can access that using `ops.current().location()`.
   
   I'm really reluctant to add more methods to this interface since it is an important place where people can plug in their own customizations.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r415129062



##########
File path: core/src/main/java/org/apache/iceberg/FastAppend.java
##########
@@ -49,7 +49,8 @@
   private ManifestFile newManifest = null;
   private boolean hasNewFiles = false;
 
-  FastAppend(TableOperations ops) {
+  FastAppend(String tableName, TableOperations ops) {
+    //TODO we use tableName to publish notification

Review comment:
       So `FastAppend` extends `SnapshotProducer` . If I change `SnapshotProducer`, I'll have to change `BaseRewriteManifests`. OK to make that change in this PR?




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on pull request #939: Support UpdateEvent for Update Operations

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


   Overall, I think this is ready. I'd prefer to use `current` instead of `refresh` in the `updateEvent` methods, though. I'd probably wait to merge until then.
   
   It would also be good to follow up with events in snapshot manager: rollback, fast-forward, and set current snapshot.
   
   Last, I think that this is going to send events each time an individual operation is committed in a transaction. That's okay for now, but we may want to go back through and suppress events during a transaction and then send them all at once when the transaction succeeds. That will handle retries better.


----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r415924555



##########
File path: core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java
##########
@@ -93,7 +93,7 @@ public Transaction newCreateTableTransaction(
     String baseLocation = location != null ? location : defaultWarehouseLocation(identifier);
     Map<String, String> tableProperties = properties != null ? properties : Maps.newHashMap();
     TableMetadata metadata = TableMetadata.newTableMetadata(schema, spec, baseLocation, tableProperties);
-    return Transactions.createTableTransaction(ops, metadata);
+    return Transactions.createTableTransaction(identifier.name(), ops, metadata);

Review comment:
       Fixed.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r411569305



##########
File path: core/src/main/java/org/apache/iceberg/TableOperations.java
##########
@@ -30,6 +30,8 @@
  */
 public interface TableOperations {
 
+  String tablePath();

Review comment:
       I guess table location is also fine. We can parse out table name and db name from it.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r411569906



##########
File path: core/src/main/java/org/apache/iceberg/SchemaUpdate.java
##########
@@ -476,4 +476,6 @@ public Type primitive(Type.PrimitiveType primitive) {
     newFields.addAll(adds);
     return Types.StructType.of(newFields);
   }
+
+  //TODO: updateEvent

Review comment:
       Makes sense




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/SchemaUpdate.java
##########
@@ -476,4 +476,6 @@ public Type primitive(Type.PrimitiveType primitive) {
     newFields.addAll(adds);
     return Types.StructType.of(newFields);
   }
+
+  //TODO: updateEvent

Review comment:
       I think we can add update events individually.




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/FastAppend.java
##########
@@ -49,7 +49,8 @@
   private ManifestFile newManifest = null;
   private boolean hasNewFiles = false;
 
-  FastAppend(TableOperations ops) {
+  FastAppend(String tableName, TableOperations ops) {
+    //TODO we use tableName to publish notification

Review comment:
       Are you saying that `SnapshotProducer` would send the event? What needs to be changed?




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r415904530



##########
File path: core/src/main/java/org/apache/iceberg/events/CreateSnapshotEvent.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.iceberg.events;
+
+import java.util.List;
+import java.util.Map;
+
+
+public final class CreateSnapshotEvent {
+  private final String tableName;
+  private final String operation;
+  private final long snapshotId;
+  private final long sequenceNumber;
+  private final Map<String, String> summary;
+  private final List<String> addedFiles;
+  private final List<String> appendedManifests;
+  private final List<String> deletePaths;
+
+  public CreateSnapshotEvent(
+      String tableName, String operation, long snapshotId, long sequenceNumber, Map<String, String> summary,
+      List<String> addedFiles, List<String> appendedManifests, List<String> deletePaths) {
+    this.tableName = tableName;
+    this.operation = operation;
+    this.snapshotId = snapshotId;
+    this.sequenceNumber = sequenceNumber;
+    this.summary = summary;
+    this.addedFiles = addedFiles;
+    this.appendedManifests = appendedManifests;
+    this.deletePaths = deletePaths;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public String operation() {
+    return operation;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public long sequenceNumber() {
+    return sequenceNumber;
+  }
+
+  public Map<String, String> summary() {
+    return summary;
+  }
+
+  public List<String> addedFiles() {
+    return addedFiles;
+  }
+
+  public List<String> appendedManifests() {
+    return appendedManifests;
+  }
+
+  public List<String> deletePaths() {

Review comment:
       fixed.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r411584985



##########
File path: api/src/main/java/org/apache/iceberg/events/UpdateEvent.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.iceberg.events;
+
+public interface UpdateEvent {

Review comment:
       The issue is that I need to call the listeners after commit, but I don't have all the information to construct the event. For instance for `SnapshotProducer` where we are committing  I don't have information on how to construct the event and I'm relying on the subclass `MergingSnapshotProducer` to construct the right event properly.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r415903979



##########
File path: core/src/main/java/org/apache/iceberg/events/CreateSnapshotEvent.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.iceberg.events;
+
+import java.util.List;
+import java.util.Map;
+
+
+public final class CreateSnapshotEvent {
+  private final String tableName;
+  private final String operation;
+  private final long snapshotId;
+  private final long sequenceNumber;
+  private final Map<String, String> summary;
+  private final List<String> addedFiles;
+  private final List<String> appendedManifests;
+  private final List<String> deletePaths;
+
+  public CreateSnapshotEvent(
+      String tableName, String operation, long snapshotId, long sequenceNumber, Map<String, String> summary,
+      List<String> addedFiles, List<String> appendedManifests, List<String> deletePaths) {
+    this.tableName = tableName;
+    this.operation = operation;
+    this.snapshotId = snapshotId;
+    this.sequenceNumber = sequenceNumber;
+    this.summary = summary;
+    this.addedFiles = addedFiles;
+    this.appendedManifests = appendedManifests;
+    this.deletePaths = deletePaths;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public String operation() {
+    return operation;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public long sequenceNumber() {
+    return sequenceNumber;
+  }
+
+  public Map<String, String> summary() {
+    return summary;
+  }
+
+  public List<String> addedFiles() {
+    return addedFiles;
+  }
+
+  public List<String> appendedManifests() {

Review comment:
       Removed for now.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#issuecomment-620054913


   This rb is ready for review again. As for #968 .  For now we can override `updateEvent()` to return a null. Later we can fix it as described here: https://github.com/apache/incubator-iceberg/pull/968#discussion_r415416694


----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/BaseTable.java
##########
@@ -39,6 +39,11 @@ public BaseTable(TableOperations ops, String name) {
     this.name = name;
   }
 
+  @Override
+  public String name() {

Review comment:
       Couldn't this method be added in a separate commit?




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/events/CreateSnapshotEvent.java
##########
@@ -0,0 +1,66 @@
+/*
+ * 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.iceberg.events;
+
+import java.util.Map;
+
+public final class CreateSnapshotEvent implements UpdateEvent {
+  private final String operation;
+  private final String tablePath;
+  private final Map<String, String> summary;
+  private final Iterable<String> addedFiles;
+  private final Iterable<String> appendedManifests;
+  private final Iterable<String> deletePaths;
+
+  public CreateSnapshotEvent(
+      String operation, String tablePath, Map<String, String> summary,
+      Iterable<String> addedFiles, Iterable<String> appendedManifests, Iterable<String> deletePaths) {

Review comment:
       I don't think we need to add to `TableOperations`. Table keeps track of its name, and is responsible for creating the operations classes, like `FastAppend`. So we can just pass the name in along side `TableOperations`:
   
   ```java
     @Override
     public AppendFiles newFastAppend() {
       return new FastAppend(name, ops);
     }
   ```




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on issue #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on issue #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#issuecomment-617935315


   @rdblue . Sorry I think this went in by mistake. This is not ready for review yet [Thought the final version would be somewhat similar]


----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/SnapshotManager.java
##########
@@ -127,6 +127,32 @@ public ManageSnapshots rollbackTo(long snapshotId) {
     return setCurrentSnapshot(snapshotId);
   }
 
+  @Override
+  public Object updateEvent() {
+    if (targetSnapshotId == null) {
+      // NOOP operation, no snapshot created
+      return null;
+    }
+
+    switch (managerOperation) {
+      case ROLLBACK:
+        // rollback does not create a new snapshot

Review comment:
       It doesn't create a new snapshot, but should it send an event? Maybe as a follow-up?




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r415921538



##########
File path: core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java
##########
@@ -25,8 +25,8 @@
 
 public class BaseReplacePartitions
     extends MergingSnapshotProducer<ReplacePartitions> implements ReplacePartitions {
-  BaseReplacePartitions(TableOperations ops) {
-    super(ops);
+  BaseReplacePartitions(String tableName, TableOperations ops) {
+    super(tableName, ops);

Review comment:
       I felt tableName was more appropriate. I have changed it as such in the code I added/modified. 
   There are still uses of name though, e.g in Transaction api. I haven't changed them.  Let me know if u feel strong in using `name` or changing to `tableName` everywhere. I'm fine with either




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: api/src/main/java/org/apache/iceberg/events/UpdateEvent.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.iceberg.events;
+
+public interface UpdateEvent {

Review comment:
       Makes sense. We could also have `updateEvent()` return `Object`, since there are no requirements for objects passed to `Listeners.notifyAll`.




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/events/CreateSnapshotEvent.java
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.iceberg.events;
+
+import java.util.List;
+import java.util.Map;
+
+
+public final class CreateSnapshotEvent {
+  private final String tableName;
+  private final String operation;
+  private final long snapshotId;
+  private final long sequenceNumber;
+  private final Map<String, String> summary;
+  private final List<String> addedFiles;
+  private final List<String> appendedManifests;
+  private final List<String> deletePaths;
+
+  public CreateSnapshotEvent(
+      String tableName, String operation, long snapshotId, long sequenceNumber, Map<String, String> summary,
+      List<String> addedFiles, List<String> appendedManifests, List<String> deletePaths) {
+    this.tableName = tableName;
+    this.operation = operation;
+    this.snapshotId = snapshotId;
+    this.sequenceNumber = sequenceNumber;
+    this.summary = summary;
+    this.addedFiles = addedFiles;
+    this.appendedManifests = appendedManifests;
+    this.deletePaths = deletePaths;
+  }
+
+  public String tableName() {
+    return tableName;
+  }
+
+  public String operation() {
+    return operation;
+  }
+
+  public long snapshotId() {
+    return snapshotId;
+  }
+
+  public long sequenceNumber() {
+    return sequenceNumber;
+  }
+
+  public Map<String, String> summary() {
+    return summary;
+  }
+
+  public List<String> addedFiles() {
+    return addedFiles;
+  }
+
+  public List<String> appendedManifests() {

Review comment:
       Is this the set of manifests that were added to the dataset, or the ones that were passed to `appendManifest`? I think we should be careful that we know exactly what is in these messages. If it isn't obvious what these should be, then we should remove the data. We can always add more information later.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#issuecomment-623559713


   Thanks @rdblue . I'll create followups for the remaining items


----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/PropertiesUpdate.java
##########
@@ -37,6 +37,7 @@
 import static org.apache.iceberg.TableProperties.COMMIT_TOTAL_RETRY_TIME_MS_DEFAULT;
 
 class PropertiesUpdate implements UpdateProperties {
+

Review comment:
       Nit: this file doesn't need to change.




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java
##########
@@ -317,6 +319,18 @@ private ManifestFile copyManifest(ManifestFile manifest) {
     }
   }
 
+  @Override
+  public UpdateEvent updateEvent() {
+    return new CreateSnapshotEvent(

Review comment:
       I think this should also include the snapshot ID and sequence number that were committed.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r411569305



##########
File path: core/src/main/java/org/apache/iceberg/TableOperations.java
##########
@@ -30,6 +30,8 @@
  */
 public interface TableOperations {
 
+  String tablePath();

Review comment:
       I guess table location is also fine. We can parse out table name and db name from it. Though is it unclean, as we are relying on hidden/internal implementations on how the location looks like.




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

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



##########
File path: api/src/main/java/org/apache/iceberg/events/UpdateEvent.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.iceberg.events;
+
+public interface UpdateEvent {

Review comment:
       What is the value of having an interface for all types of updates with no methods?
   
   The listener code routes these events using `listeners.get(event.getClass())` so the interface doesn't help to subscribe to all events.




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r411579807



##########
File path: api/src/main/java/org/apache/iceberg/events/UpdateEvent.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.iceberg.events;
+
+public interface UpdateEvent {

Review comment:
       I was thinking that there may be some update operations where commit is being performed in their superclass. In those cases, we have to get the sub class' update event. Seeing through the code again, I don't see this and where a commit it being performed in a class it has all the information to send the event.  
   
   I'll revert this change




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r416003747



##########
File path: core/src/main/java/org/apache/iceberg/FastAppend.java
##########
@@ -49,7 +49,8 @@
   private ManifestFile newManifest = null;
   private boolean hasNewFiles = false;
 
-  FastAppend(TableOperations ops) {
+  FastAppend(String tableName, TableOperations ops) {
+    //TODO we use tableName to publish notification

Review comment:
       My bad. I think this can be supported. I'll do that as part of this pr




----------------------------------------------------------------
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] [incubator-iceberg] rdblue commented on a change in pull request #939: Support UpdateEvent for Update Operations

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



##########
File path: core/src/main/java/org/apache/iceberg/BaseTable.java
##########
@@ -136,37 +136,37 @@ public RewriteManifests rewriteManifests() {
 
   @Override
   public OverwriteFiles newOverwrite() {
-    return new BaseOverwriteFiles(ops);
+    return new BaseOverwriteFiles(name, ops);
   }
 
   @Override
   public ReplacePartitions newReplacePartitions() {
-    return new BaseReplacePartitions(ops);
+    return new BaseReplacePartitions(name, ops);
   }
 
   @Override
   public DeleteFiles newDelete() {
-    return new StreamingDelete(ops);
+    return new StreamingDelete(name, ops);
   }
 
   @Override
-  public ExpireSnapshots expireSnapshots() {
+  public RemoveSnapshots expireSnapshots() {

Review comment:
       Why did this change? Can we not return the interface?




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: [WIP] Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r411584985



##########
File path: api/src/main/java/org/apache/iceberg/events/UpdateEvent.java
##########
@@ -0,0 +1,26 @@
+/*
+ * 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.iceberg.events;
+
+public interface UpdateEvent {

Review comment:
       The issue is that I need to call the listeners after commit, but I don't have all the information to construct the event. For instance for `SnapshotProducer` where we are committing  I don't have information on how to construct the event and I'm relying on the subclass `MergingSnapshotProducer` to construct the right event properly




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#issuecomment-620119288


   > I think the file lists in the event might be incomplete. I'd suggest removing them and just sending the summary, unless you have a use case for them.
   I thin sending the summary is fine by me.


----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r415921538



##########
File path: core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java
##########
@@ -25,8 +25,8 @@
 
 public class BaseReplacePartitions
     extends MergingSnapshotProducer<ReplacePartitions> implements ReplacePartitions {
-  BaseReplacePartitions(TableOperations ops) {
-    super(ops);
+  BaseReplacePartitions(String tableName, TableOperations ops) {
+    super(tableName, ops);

Review comment:
       I felt tableName was more appropriate. I have changed it as such in the code I added/modified. 
   There are still uses of name though, e.g in Transaction api. I haven't changed them.  Let me know if u feel strongly in using `name` or changing to `tableName` everywhere. I'm fine with either




----------------------------------------------------------------
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] [incubator-iceberg] rdsr commented on a change in pull request #939: Support UpdateEvent for Update Operations

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #939:
URL: https://github.com/apache/incubator-iceberg/pull/939#discussion_r415902716



##########
File path: core/src/main/java/org/apache/iceberg/BaseTable.java
##########
@@ -39,6 +39,11 @@ public BaseTable(TableOperations ops, String name) {
     this.name = name;
   }
 
+  @Override
+  public String name() {

Review comment:
       Removed




----------------------------------------------------------------
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