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/05/28 00:34:39 UTC

[GitHub] [iceberg] rdblue opened a new pull request #1073: Support cherry-pick for dynamic partition overwrites

rdblue opened a new pull request #1073:
URL: https://github.com/apache/iceberg/pull/1073


   This updates the SnapshotManager to support cherry-picking overwrite commits that were created by ReplacePartitions.
   
   The cherry-pick is validated by checking that there are no missing files that were deleted in the original commit, and that there are no new files added to any partition that has been replaced. This new behavior is tested in TestSnapshotManager.


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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1073: Support cherry-pick for dynamic partition overwrites

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



##########
File path: core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java
##########
@@ -27,6 +27,7 @@
     extends MergingSnapshotProducer<ReplacePartitions> implements ReplacePartitions {
   BaseReplacePartitions(String tableName, TableOperations ops) {
     super(tableName, ops);
+    set("replace-partitions", "true");

Review comment:
       Done.




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

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



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


[GitHub] [iceberg] prodeezy commented on pull request #1073: Support cherry-pick for dynamic partition overwrites

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


   thanks @rdblue will take a look today


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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1073: Support cherry-pick for dynamic partition overwrites

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



##########
File path: core/src/test/java/org/apache/iceberg/TestSnapshotManager.java
##########
@@ -0,0 +1,256 @@
+/*
+ * 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;
+
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;

Review comment:
       Yes. The relocated dependency comes from our shaded module and we've excluded the unshaded version from our compile classpaths.




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

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



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


[GitHub] [iceberg] prodeezy commented on a change in pull request #1073: Support cherry-pick for dynamic partition overwrites

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



##########
File path: core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java
##########
@@ -27,6 +27,7 @@
     extends MergingSnapshotProducer<ReplacePartitions> implements ReplacePartitions {
   BaseReplacePartitions(String tableName, TableOperations ops) {
     super(tableName, ops);
+    set("replace-partitions", "true");

Review comment:
       can we make `replace-partitions` a static variable in `SnaphotSummary.java`?




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

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



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


[GitHub] [iceberg] danielcweeks merged pull request #1073: Support cherry-pick for dynamic partition overwrites

Posted by GitBox <gi...@apache.org>.
danielcweeks merged pull request #1073:
URL: https://github.com/apache/iceberg/pull/1073


   


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

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



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


[GitHub] [iceberg] danielcweeks commented on a change in pull request #1073: Support cherry-pick for dynamic partition overwrites

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



##########
File path: core/src/test/java/org/apache/iceberg/TestSnapshotManager.java
##########
@@ -0,0 +1,256 @@
+/*
+ * 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;
+
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;

Review comment:
       I guess I missed the relocated guava work.  I suppose this 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] [iceberg] danielcweeks commented on a change in pull request #1073: Support cherry-pick for dynamic partition overwrites

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



##########
File path: core/src/test/java/org/apache/iceberg/TestSnapshotManager.java
##########
@@ -0,0 +1,256 @@
+/*
+ * 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;
+
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;

Review comment:
       nit: do we need to depend on relocated Iterables for tests?




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

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



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


[GitHub] [iceberg] danielcweeks commented on pull request #1073: Support cherry-pick for dynamic partition overwrites

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


   +1 


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

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



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


[GitHub] [iceberg] rdblue closed pull request #1073: Support cherry-pick for dynamic partition overwrites

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


   


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

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



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


[GitHub] [iceberg] rdblue commented on pull request #1073: Support cherry-pick for dynamic partition overwrites

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


   @prodeezy, you may be interested in this. It adds the ability to cherry-pick partition replacement 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] [iceberg] prodeezy edited a comment on pull request #1073: Support cherry-pick for dynamic partition overwrites

Posted by GitBox <gi...@apache.org>.
prodeezy edited a comment on pull request #1073:
URL: https://github.com/apache/iceberg/pull/1073#issuecomment-637630489


   thanks @rdblue will take a look today. cc @mehtaashish23 @rominparekh 


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

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



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


[GitHub] [iceberg] prodeezy commented on a change in pull request #1073: Support cherry-pick for dynamic partition overwrites

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



##########
File path: core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java
##########
@@ -27,6 +27,7 @@
     extends MergingSnapshotProducer<ReplacePartitions> implements ReplacePartitions {
   BaseReplacePartitions(String tableName, TableOperations ops) {
     super(tableName, ops);
+    set("replace-partitions", "true");

Review comment:
       can we make `replace-partitions` property a static variable in `SnaphotSummary.java`?




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

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



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


[GitHub] [iceberg] rdblue commented on pull request #1073: Support cherry-pick for dynamic partition overwrites

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


   Thanks for reviewing, @prodeezy and @danielcweeks!


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