Github user marmbrus commented on a diff in the pull request:

    https://github.com/apache/spark/pull/11645#discussion_r56373034
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala
 ---
    @@ -0,0 +1,465 @@
    +/*
    + * 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.spark.sql.execution.streaming.state
    +
    +import scala.collection.mutable
    +import scala.util.Random
    +import scala.util.control.NonFatal
    +
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileStatus, Path}
    +
    +import org.apache.spark.{Logging, SparkConf}
    +import org.apache.spark.serializer.{DeserializationStream, KryoSerializer, 
SerializationStream}
    +import org.apache.spark.sql.catalyst.InternalRow
    +import org.apache.spark.sql.catalyst.expressions.JoinedRow
    +import org.apache.spark.util.{CompletionIterator, Utils}
    +
    +
    +/**
    + * An implementation of [[StateStoreProvider]] and [[StateStore]] in which 
all the data is backed
    + * by files in a HDFS-compatible file system. All updates to the store has 
to be done in sets
    + * transactionally, and each set of updates increments the store's 
version. These versions can
    + * be used to re-execute the updates (by retries in RDD operations) on the 
correct version of
    + * the store, and regenerate the store version.
    + *
    + * Usage:
    + * To update the data in the state store, the following order of 
operations are needed.
    + *
    + * - val store = StateStore.get(operatorId, partitionId, version) // to 
get the right store
    + * - store.update(...)
    + * - store.remove(...)
    + * - store.commit()    // commits all the updates to made with version 
number
    + * - store.iterator()  // key-value data after last commit as an iterator
    + * - store.updates()   // updates made in the last as an iterator
    + *
    + * Fault-tolerance model:
    + * - Every set of updates is written to a delta file before committing.
    + * - The state store is responsible for managing, collapsing and cleaning 
up of delta files.
    + * - Multiple attempts to commit the same version of updates must have the 
same updates.
    --- End diff --
    
    Maybe even say that if they are different, which one wins is undefined? (It 
is at least atomic right?)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org

Reply via email to