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 2021/01/14 12:31:48 UTC

[GitHub] [iceberg] XuQianJin-Stars opened a new pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

XuQianJin-Stars opened a new pull request #2092:
URL: https://github.com/apache/iceberg/pull/2092


   An implementation of Spark Structured Streaming Offset, to track the current processed files of Iceberg table, This PR is a split of the [PR-796](https://github.com/apache/iceberg/pull/796) of Structured streaming read for Iceberg.


----------------------------------------------------------------
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 #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.spark.source;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+
+class StreamingOffset extends Offset {
+  static final StreamingOffset START_OFFSET = new StreamingOffset(-1L, -1, false, true);
+
+  private static final int CURR_VERSION = 1;
+  private static final String VERSION = "version";
+  private static final String SNAPSHOT_ID = "snapshot_id";
+  private static final String INDEX = "index";
+  private static final String SCAN_ALL_FILES = "scan_all_files";
+  private static final String SNAPSHOT_FULLY_PROCESSED = "snapshot_fully_processed";
+
+  private final long snapshotId;
+  private final int index;
+  private final boolean scanAllFiles;
+  private final boolean snapshotFullyProcessed;
+
+  /**
+   * An implementation of Spark Structured Streaming Offset, to track the current processed files of
+   * Iceberg table.
+   *
+   * @param snapshotId             The current processed snapshot id.
+   * @param index                  The index of last scanned file in snapshot.
+   * @param scanAllFiles           Denote whether to scan all files in a snapshot, currently we only
+   *                               scan all files in the starting snapshot.
+   * @param snapshotFullyProcessed Denote whether the current snapshot is fully processed, to avoid
+   *                               revisiting the processed snapshot.
+   */
+  StreamingOffset(long snapshotId, int index, boolean scanAllFiles,
+      boolean snapshotFullyProcessed) {

Review comment:
       Nit: looks like this newline is unnecessary.




----------------------------------------------------------------
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 #2092: StreamingOffset Of Structured streaming read for Iceberg

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


   Will do. Thanks for pinging 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] [iceberg] kbendick commented on a change in pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.spark.source;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+
+/**
+ * An implementation of Spark Structured Streaming Offset, to track the current processed
+ * files of Iceberg table. This StreamingOffset consists of:
+ *
+ * version: The version of StreamingOffset. The offset was created with a version number used to validate
+ * when deserializing from json string.
+ * snapshot_id: The current processed snapshot id.
+ * index: The index of last scanned file in snapshot.
+ * scan_all_files: Denote whether to scan all files in a snapshot, currently we only scan all files in the starting
+ * snapshot.
+ * snapshot_fully_processed: Denote whether the current snapshot is fully processed, to avoid revisiting the processed
+ * snapshot.
+ */

Review comment:
       Somewhat of a nit: Can you please make this a proper javadoc comment, such as using `@param` before the constructor parameters, listing out all of the constructor parameters in order, as well as formatting the constructor parameters the way that they are in the code (i.e. using camel case and not snake case)? I would say also that the line `This StreamingOffset consists of:` will be unnecessary if you do that.




----------------------------------------------------------------
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] XuQianJin-Stars commented on pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #2092:
URL: https://github.com/apache/iceberg/pull/2092#issuecomment-781751681


   hi @rdblue would you please take another look at this PR at your convenience.


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

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



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


[GitHub] [iceberg] kbendick commented on a change in pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.spark.source;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+
+class StreamingOffset extends Offset {
+  static final StreamingOffset START_OFFSET = new StreamingOffset(-1L, -1, false, true);
+
+  private static final int CURR_VERSION = 1;
+  private static final String VERSION = "version";
+  private static final String SNAPSHOT_ID = "snapshot_id";
+  private static final String INDEX = "index";
+  private static final String SCAN_ALL_FILES = "scan_all_files";
+  private static final String SNAPSHOT_FULLY_PROCESSED = "snapshot_fully_processed";
+
+  private final long snapshotId;
+  private final int index;
+  private final boolean scanAllFiles;
+  private final boolean snapshotFullyProcessed;
+
+  /**
+   * An implementation of Spark Structured Streaming Offset, to track the current processed files of
+   * Iceberg table.
+   *
+   * @param snapshotId             The current processed snapshot id.
+   * @param index                  The index of last scanned file in snapshot.
+   * @param scanAllFiles           Denote whether to scan all files in a snapshot, currently we only
+   *                               scan all files in the starting snapshot.
+   * @param snapshotFullyProcessed Denote whether the current snapshot is fully processed, to avoid
+   *                               revisiting the processed snapshot.
+   */
+  StreamingOffset(long snapshotId, int index, boolean scanAllFiles,
+      boolean snapshotFullyProcessed) {
+    this.snapshotId = snapshotId;
+    this.index = index;
+    this.scanAllFiles = scanAllFiles;
+    this.snapshotFullyProcessed = snapshotFullyProcessed;
+  }
+
+  static StreamingOffset fromJson(String json) {
+    Preconditions.checkNotNull(json, "The input JSON string is null");

Review comment:
       Nit: it might be best to have a more explanatory message in the Preconditions check. Something like `The input JSON string representation of a StreamingOffset cannot be null`. I'll leave that up to you / the others to decide as it's a minor nit.




----------------------------------------------------------------
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 #2092: StreamingOffset Of Structured streaming read for Iceberg

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


   Looks good now. I'll merge this. Thanks @XuQianJin-Stars!


----------------------------------------------------------------
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 #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.spark.source;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+
+class StreamingOffset extends Offset {
+  static final StreamingOffset START_OFFSET = new StreamingOffset(-1L, -1, false, true);
+
+  private static final int CURR_VERSION = 1;
+  private static final String VERSION = "version";
+  private static final String SNAPSHOT_ID = "snapshot_id";
+  private static final String INDEX = "index";
+  private static final String SCAN_ALL_FILES = "scan_all_files";
+  private static final String SNAPSHOT_FULLY_PROCESSED = "snapshot_fully_processed";
+
+  private final long snapshotId;
+  private final int index;
+  private final boolean scanAllFiles;
+  private final boolean snapshotFullyProcessed;
+
+  /**
+   * An implementation of Spark Structured Streaming Offset, to track the current processed files of
+   * Iceberg table.
+   *
+   * @param snapshotId             The current processed snapshot id.
+   * @param index                  The index of last scanned file in snapshot.
+   * @param scanAllFiles           Denote whether to scan all files in a snapshot, currently we only
+   *                               scan all files in the starting snapshot.
+   * @param snapshotFullyProcessed Denote whether the current snapshot is fully processed, to avoid
+   *                               revisiting the processed snapshot.
+   */
+  StreamingOffset(long snapshotId, int index, boolean scanAllFiles,
+      boolean snapshotFullyProcessed) {
+    this.snapshotId = snapshotId;
+    this.index = index;
+    this.scanAllFiles = scanAllFiles;
+    this.snapshotFullyProcessed = snapshotFullyProcessed;
+  }
+
+  static StreamingOffset fromJson(String json) {
+    Preconditions.checkNotNull(json, "The input JSON string representation of a StreamingOffset cannot be null");
+
+    try {
+      JsonNode node = JsonUtil.mapper().readValue(json, JsonNode.class);
+      // The version of StreamingOffset. The offset was created with a version number
+      // used to validate when deserializing from json string.
+      int version = JsonUtil.getInt(VERSION, node);
+      if (version > CURR_VERSION) {
+        throw new IOException(String.format("This version of iceberg only supports version %s", CURR_VERSION));
+      }
+
+      long snapshotId = JsonUtil.getLong(SNAPSHOT_ID, node);
+      int index = JsonUtil.getInt(INDEX, node);
+      boolean shouldScanAllFiles = JsonUtil.getBool(SCAN_ALL_FILES, node);
+      boolean snapshotFullyProcessed = JsonUtil.getBool(SNAPSHOT_FULLY_PROCESSED, node);
+
+      return new StreamingOffset(snapshotId, index, shouldScanAllFiles, snapshotFullyProcessed);
+    } catch (IOException e) {
+      throw new IllegalStateException(String.format("Failed to parse StreamingOffset from JSON string %s", json), e);

Review comment:
       I think this should be `IllegalArgumentException` instead. There is no state in this static method.




----------------------------------------------------------------
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] kbendick commented on a change in pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.spark.source;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+
+class StreamingOffset extends Offset {
+  static final StreamingOffset START_OFFSET = new StreamingOffset(-1L, -1, false, true);
+
+  private static final int CURR_VERSION = 1;
+  private static final String VERSION = "version";
+  private static final String SNAPSHOT_ID = "snapshot_id";
+  private static final String INDEX = "index";
+  private static final String SCAN_ALL_FILES = "scan_all_files";
+  private static final String SNAPSHOT_FULLY_PROCESSED = "snapshot_fully_processed";
+
+  private final long snapshotId;
+  private final int index;
+  private final boolean scanAllFiles;
+  private final boolean snapshotFullyProcessed;
+
+  /**
+   * An implementation of Spark Structured Streaming Offset, to track the current processed files of
+   * Iceberg table.
+   *
+   * @param snapshotId             The current processed snapshot id.
+   * @param index                  The index of last scanned file in snapshot.
+   * @param scanAllFiles           Denote whether to scan all files in a snapshot, currently we only
+   *                               scan all files in the starting snapshot.
+   * @param snapshotFullyProcessed Denote whether the current snapshot is fully processed, to avoid
+   *                               revisiting the processed snapshot.
+   */
+  StreamingOffset(long snapshotId, int index, boolean scanAllFiles,
+      boolean snapshotFullyProcessed) {
+    this.snapshotId = snapshotId;
+    this.index = index;
+    this.scanAllFiles = scanAllFiles;
+    this.snapshotFullyProcessed = snapshotFullyProcessed;
+  }
+
+  static StreamingOffset fromJson(String json) {
+    Preconditions.checkNotNull(json, "The input JSON string is null");
+
+    try {
+      JsonNode node = JsonUtil.mapper().readValue(json, JsonNode.class);
+      // The version of StreamingOffset. The offset was created with a version number
+      // used to validate when deserializing from json string.
+      int version = JsonUtil.getInt(VERSION, node);
+      if (version > CURR_VERSION) {

Review comment:
       Do we not plan to support version 2 snapshot files yet? I suppose that makes sense given the need to process deletes etc with the version 2 snapshots, but looking to understand what other reasons might exist for not supporting them yet.




----------------------------------------------------------------
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 #2092: StreamingOffset Of Structured streaming read for Iceberg

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


   Thanks for updating this, @XuQianJin-Stars. I think the implementation looks good now. We just need to fix up the tests. Thanks!


----------------------------------------------------------------
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 #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.spark.source;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+
+class StreamingOffset extends Offset {
+  static final StreamingOffset START_OFFSET = new StreamingOffset(-1L, -1, false, true);
+
+  private static final int CURR_VERSION = 1;
+  private static final String VERSION = "version";
+  private static final String SNAPSHOT_ID = "snapshot_id";
+  private static final String INDEX = "index";
+  private static final String SCAN_ALL_FILES = "scan_all_files";
+  private static final String SNAPSHOT_FULLY_PROCESSED = "snapshot_fully_processed";

Review comment:
       Now that data and delete files can report the row position within a manifest, I don't think that we need to use `SNAPSHOT_FULLY_PROCESSED` anymore. Even if manifests are filtered, the position will be reliable. So by using position for the `index` field in this offset, we can check whether a given manifest has been completely processed by comparing the index with the number of entries in the manifest.
   
   I think that getting rid of this field and using a simpler offset is a good improvement.




----------------------------------------------------------------
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 #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.spark.source;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+
+class StreamingOffset extends Offset {
+  static final StreamingOffset START_OFFSET = new StreamingOffset(-1L, -1, false, true);
+
+  private static final int CURR_VERSION = 1;
+  private static final String VERSION = "version";
+  private static final String SNAPSHOT_ID = "snapshot_id";
+  private static final String INDEX = "index";
+  private static final String SCAN_ALL_FILES = "scan_all_files";
+  private static final String SNAPSHOT_FULLY_PROCESSED = "snapshot_fully_processed";
+
+  private final long snapshotId;
+  private final int index;
+  private final boolean scanAllFiles;
+  private final boolean snapshotFullyProcessed;
+
+  /**
+   * An implementation of Spark Structured Streaming Offset, to track the current processed files of
+   * Iceberg table.
+   *
+   * @param snapshotId             The current processed snapshot id.
+   * @param index                  The index of last scanned file in snapshot.
+   * @param scanAllFiles           Denote whether to scan all files in a snapshot, currently we only
+   *                               scan all files in the starting snapshot.
+   * @param snapshotFullyProcessed Denote whether the current snapshot is fully processed, to avoid
+   *                               revisiting the processed snapshot.
+   */
+  StreamingOffset(long snapshotId, int index, boolean scanAllFiles,
+      boolean snapshotFullyProcessed) {
+    this.snapshotId = snapshotId;
+    this.index = index;
+    this.scanAllFiles = scanAllFiles;
+    this.snapshotFullyProcessed = snapshotFullyProcessed;
+  }
+
+  static StreamingOffset fromJson(String json) {
+    Preconditions.checkNotNull(json, "The input JSON string representation of a StreamingOffset cannot be null");

Review comment:
       Error messages typically follow the pattern "Invalid ..." or "Cannot (action): (reason) [suggestion]". Here, there is no suggestion for fixing it, so it would be something like "Cannot parse offset JSON: 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



---------------------------------------------------------------------
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 #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.spark.source;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+
+class StreamingOffset extends Offset {
+  static final StreamingOffset START_OFFSET = new StreamingOffset(-1L, -1, false, true);
+
+  private static final int CURR_VERSION = 1;
+  private static final String VERSION = "version";
+  private static final String SNAPSHOT_ID = "snapshot_id";
+  private static final String INDEX = "index";
+  private static final String SCAN_ALL_FILES = "scan_all_files";
+  private static final String SNAPSHOT_FULLY_PROCESSED = "snapshot_fully_processed";
+
+  private final long snapshotId;
+  private final int index;
+  private final boolean scanAllFiles;
+  private final boolean snapshotFullyProcessed;
+
+  /**
+   * An implementation of Spark Structured Streaming Offset, to track the current processed files of
+   * Iceberg table.
+   *
+   * @param snapshotId             The current processed snapshot id.
+   * @param index                  The index of last scanned file in snapshot.
+   * @param scanAllFiles           Denote whether to scan all files in a snapshot, currently we only
+   *                               scan all files in the starting snapshot.
+   * @param snapshotFullyProcessed Denote whether the current snapshot is fully processed, to avoid
+   *                               revisiting the processed snapshot.
+   */
+  StreamingOffset(long snapshotId, int index, boolean scanAllFiles,
+      boolean snapshotFullyProcessed) {
+    this.snapshotId = snapshotId;
+    this.index = index;
+    this.scanAllFiles = scanAllFiles;
+    this.snapshotFullyProcessed = snapshotFullyProcessed;
+  }
+
+  static StreamingOffset fromJson(String json) {
+    Preconditions.checkNotNull(json, "The input JSON string is null");
+
+    try {
+      JsonNode node = JsonUtil.mapper().readValue(json, JsonNode.class);
+      // The version of StreamingOffset. The offset was created with a version number
+      // used to validate when deserializing from json string.
+      int version = JsonUtil.getInt(VERSION, node);
+      if (version > CURR_VERSION) {

Review comment:
       Right now, I think we want to focus on v1 data. But I think this version is actually referring to the version of the offset JSON, in case we need to change it 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] [iceberg] rdblue merged pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

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


   


----------------------------------------------------------------
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] XuQianJin-Stars commented on pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #2092:
URL: https://github.com/apache/iceberg/pull/2092#issuecomment-770500766


   hi @rdblueThanks again, I have updating 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] [iceberg] RussellSpitzer commented on a change in pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java
##########
@@ -0,0 +1,150 @@
+/*
+ * 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.spark.source;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+
+/**
+ * An implementation of Spark Structured Streaming Offset, to track the current processed
+ * files of Iceberg table. This StreamingOffset consists of:
+ *
+ * version: The version of StreamingOffset. The offset was created with a version number used to validate
+ * when deserializing from json string.
+ * snapshot_id: The current processed snapshot id.
+ * index: The index of last scanned file in snapshot.
+ * scan_all_files: Denote whether to scan all files in a snapshot, currently we only scan all files in the starting
+ * snapshot.
+ * snapshot_fully_processed: Denote whether the current snapshot is fully processed, to avoid revisiting the processed
+ * snapshot.
+ */
+class StreamingOffset extends Offset {
+  static final StreamingOffset START_OFFSET = new StreamingOffset(-1L, -1, false, true);
+
+  private static final int CURR_VERSION = 1;
+  private static final String VERSION = "version";
+  private static final String SNAPSHOT_ID = "snapshot_id";
+  private static final String INDEX = "index";
+  private static final String SCAN_ALL_FILES = "scan_all_files";
+  private static final String SNAPSHOT_FULLY_PROCESSED = "snapshot_fully_processed";
+
+  private final long snapshotId;
+  private final int index;
+  private final boolean scanAllFiles;
+  private final boolean snapshotFullyProcessed;
+
+  StreamingOffset(long snapshotId, int index, boolean scanAllFiles, boolean snapshotFullyProcessed) {
+    this.snapshotId = snapshotId;
+    this.index = index;
+    this.scanAllFiles = scanAllFiles;
+    this.snapshotFullyProcessed = snapshotFullyProcessed;
+  }
+
+  static StreamingOffset fromJson(String json) {
+    Preconditions.checkNotNull(json, "The input JSON string is null");
+
+    try {
+      JsonNode node = JsonUtil.mapper().readValue(json, JsonNode.class);
+      int version = JsonUtil.getInt(VERSION, node);
+      if (version > CURR_VERSION) {
+        throw new IOException(String.format("Cannot deserialize a JSON offset from version %d. %d is not compatible " +
+            "with the version of Iceberg %d and cannot be used. Please use a compatible version of Iceberg " +

Review comment:
       This is a bit confusing because it's not the Iceberg version, but the Streaming Offset Version. Maybe just phrase it instead "This version of iceberg only supports version $curversion"




----------------------------------------------------------------
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 #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/test/java/org/apache/iceberg/spark/source/TestStreamingOffset.java
##########
@@ -0,0 +1,174 @@
+/*
+ * 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.spark.source;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.ManifestFile;
+import org.apache.iceberg.ManifestFiles;
+import org.apache.iceberg.ManifestReader;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestStreamingOffset {
+  private static final Configuration CONF = new Configuration();
+  private static final Schema SCHEMA = new Schema(
+      optional(1, "id", Types.IntegerType.get()),
+      optional(2, "data", Types.StringType.get())
+  );
+  protected static final int INIT_SCANNED_FILE_INDEX = -1;
+  protected static final int END_SCANNED_FILE_INDEX = 3;
+  protected static SparkSession spark = null;
+  protected static Path parent = null;
+  protected static File tableLocation = null;
+  protected static Table table = null;
+  protected static List<SimpleRecord> expected = null;
+  protected static final FileIO FILE_IO = new TestTables.LocalFileIO();
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+  @Rule
+  public ExpectedException exceptionRule = ExpectedException.none();
+
+  @BeforeClass
+  public static void startSpark() throws IOException {

Review comment:
       This PR introduces a simple class that can serialize/deserialize itself using JSON.
   
   I don't see any value in having a test that creates a Spark session and does dataframe operations. I think what needs to be tested is that you can create an offset, get the expected values from the getter methods, serialize, and deserialize correctly. Those don't involve Spark, so I don't see a reason to slow down tests overall by adding Spark tests here.
   
   Can you remove the Spark code and do some basic tests for serialization?




----------------------------------------------------------------
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] XuQianJin-Stars commented on pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #2092:
URL: https://github.com/apache/iceberg/pull/2092#issuecomment-760895776


   hi @aokolnychyi @rdblue please take a look at this PR at your convenience.


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

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead.java
##########
@@ -0,0 +1,104 @@
+/*
+ * 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.spark.source;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.List;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.hadoop.HadoopTables;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SparkSession;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.rules.ExpectedException;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+
+public abstract class TestStructuredStreamingRead {

Review comment:
       This class has no tests? Can you remove 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] [iceberg] XuQianJin-Stars commented on pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #2092:
URL: https://github.com/apache/iceberg/pull/2092#issuecomment-762585584


   Thanks again to @rdblue  and @kbendick review this PR. I have addressed 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] [iceberg] rdblue commented on a change in pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.spark.source;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+
+class StreamingOffset extends Offset {
+  static final StreamingOffset START_OFFSET = new StreamingOffset(-1L, -1, false, true);
+
+  private static final int CURR_VERSION = 1;
+  private static final String VERSION = "version";
+  private static final String SNAPSHOT_ID = "snapshot_id";
+  private static final String INDEX = "index";
+  private static final String SCAN_ALL_FILES = "scan_all_files";
+  private static final String SNAPSHOT_FULLY_PROCESSED = "snapshot_fully_processed";
+
+  private final long snapshotId;
+  private final int index;
+  private final boolean scanAllFiles;
+  private final boolean snapshotFullyProcessed;
+
+  /**
+   * An implementation of Spark Structured Streaming Offset, to track the current processed files of
+   * Iceberg table.
+   *
+   * @param snapshotId             The current processed snapshot id.
+   * @param index                  The index of last scanned file in snapshot.
+   * @param scanAllFiles           Denote whether to scan all files in a snapshot, currently we only
+   *                               scan all files in the starting snapshot.

Review comment:
       Docs should not refer to `currently`. If you want to give context for why this might be used, then use "for example". "Whether to scan all files in a snapshot; for example, to read all data when starting a stream"




----------------------------------------------------------------
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 #2092: StreamingOffset Of Structured streaming read for Iceberg

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



##########
File path: spark/src/main/java/org/apache/iceberg/spark/source/StreamingOffset.java
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.spark.source;
+
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.databind.JsonNode;
+import java.io.IOException;
+import java.io.StringWriter;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.relocated.com.google.common.base.Objects;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.JsonUtil;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+
+class StreamingOffset extends Offset {
+  static final StreamingOffset START_OFFSET = new StreamingOffset(-1L, -1, false, true);
+
+  private static final int CURR_VERSION = 1;
+  private static final String VERSION = "version";
+  private static final String SNAPSHOT_ID = "snapshot_id";
+  private static final String INDEX = "index";
+  private static final String SCAN_ALL_FILES = "scan_all_files";
+  private static final String SNAPSHOT_FULLY_PROCESSED = "snapshot_fully_processed";
+
+  private final long snapshotId;
+  private final int index;
+  private final boolean scanAllFiles;
+  private final boolean snapshotFullyProcessed;
+
+  /**
+   * An implementation of Spark Structured Streaming Offset, to track the current processed files of
+   * Iceberg table.
+   *
+   * @param snapshotId             The current processed snapshot id.
+   * @param index                  The index of last scanned file in snapshot.
+   * @param scanAllFiles           Denote whether to scan all files in a snapshot, currently we only
+   *                               scan all files in the starting snapshot.
+   * @param snapshotFullyProcessed Denote whether the current snapshot is fully processed, to avoid
+   *                               revisiting the processed snapshot.
+   */
+  StreamingOffset(long snapshotId, int index, boolean scanAllFiles,
+      boolean snapshotFullyProcessed) {
+    this.snapshotId = snapshotId;
+    this.index = index;
+    this.scanAllFiles = scanAllFiles;
+    this.snapshotFullyProcessed = snapshotFullyProcessed;
+  }
+
+  static StreamingOffset fromJson(String json) {
+    Preconditions.checkNotNull(json, "The input JSON string representation of a StreamingOffset cannot be null");
+
+    try {
+      JsonNode node = JsonUtil.mapper().readValue(json, JsonNode.class);
+      // The version of StreamingOffset. The offset was created with a version number
+      // used to validate when deserializing from json string.
+      int version = JsonUtil.getInt(VERSION, node);
+      if (version > CURR_VERSION) {
+        throw new IOException(String.format("This version of iceberg only supports version %s", CURR_VERSION));

Review comment:
       This should not throw `IOException`, I think it would be `IllegalArgumentException`, which means you could use a `Precondition`. Also, I'd prefer to have the error message follow conventions:
   
   ```java
       Preconditions.check(version == CURR_VERSION,
           "Cannot parse offset JSON: offset version %s is not supported", version);
   ```




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

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



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


[GitHub] [iceberg] XuQianJin-Stars commented on pull request #2092: StreamingOffset Of Structured streaming read for Iceberg

Posted by GitBox <gi...@apache.org>.
XuQianJin-Stars commented on pull request #2092:
URL: https://github.com/apache/iceberg/pull/2092#issuecomment-760891071


   Thanks @RussellSpitzer @kbendick review 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