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

    https://github.com/apache/spark/pull/9143#discussion_r44174435
  
    --- Diff: 
streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala
 ---
    @@ -0,0 +1,206 @@
    +/*
    + * 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.streaming.util
    +
    +import java.nio.ByteBuffer
    +import java.util.concurrent.LinkedBlockingQueue
    +import java.util.{Iterator => JIterator}
    +
    +import scala.collection.JavaConverters._
    +import scala.collection.mutable.ArrayBuffer
    +import scala.concurrent.{Await, Promise}
    +import scala.concurrent.duration._
    +import scala.util.control.NonFatal
    +
    +import org.apache.spark.{SparkException, Logging}
    +import org.apache.spark.util.Utils
    +
    +/**
    + * A wrapper for a WriteAheadLog that batches records before writing data. 
Handles aggregation
    + * during writes, and de-aggregation in the `readAll` method. The end 
consumer has to handle
    + * de-aggregation after the `read` method. In addition, the 
`WriteAheadLogRecordHandle` returned
    + * after the write will contain the batch of records rather than 
individual records.
    + *
    + * When writing a batch of records, the `time` passed to the `wrappedLog` 
will be the timestamp
    + * of the latest record in the batch. This is very important in achieving 
correctness. Consider the
    + * following example:
    + * We receive records with timestamps 1, 3, 5, 7. We use "log-1" as the 
filename. Once we receive
    + * a clean up request for timestamp 3, we would clean up the file "log-1", 
and lose data regarding
    + * 5 and 7.
    + *
    + * All other methods of the WriteAheadLog interface will be passed on to 
the wrapped WriteAheadLog.
    + */
    +private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog)
    +  extends WriteAheadLog with Logging {
    +
    +  import BatchedWriteAheadLog._
    +
    +  // exposed for tests
    +  private val walWriteQueue = new LinkedBlockingQueue[Record]()
    +
    +  // Whether the writer thread is active
    +  @volatile private var active: Boolean = true
    +  private val buffer = new ArrayBuffer[Record]()
    +
    +  private val batchedWriterThread = startBatchedWriterThread()
    +
    +  /**
    +   * Write a byte buffer to the log file. This method adds the byteBuffer 
to a queue and blocks
    +   * until the record is properly written by the parent.
    +   */
    +  override def write(byteBuffer: ByteBuffer, time: Long): 
WriteAheadLogRecordHandle = {
    +    val promise = Promise[WriteAheadLogRecordHandle]()
    +    walWriteQueue.offer(Record(byteBuffer, time, promise))
    +    Await.result(promise.future, WAL_WRITE_STATUS_TIMEOUT.milliseconds)
    --- End diff --
    
    I think it's better to add a configuration for `WAL_WRITE_STATUS_TIMEOUT`. 
The default value is 5 seconds. It may be too short considering now we write a 
batch of records.


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