tustvold commented on code in PR #6526:
URL: https://github.com/apache/arrow-datafusion/pull/6526#discussion_r1217885192


##########
datafusion/core/src/datasource/file_format/mod.rs:
##########
@@ -87,6 +98,277 @@ pub trait FileFormat: Send + Sync + fmt::Debug {
         conf: FileScanConfig,
         filters: Option<&Arc<dyn PhysicalExpr>>,
     ) -> Result<Arc<dyn ExecutionPlan>>;
+
+    /// Take a list of files and the configuration to convert it to the
+    /// appropriate writer executor according to this file format.
+    async fn create_writer_physical_plan(
+        &self,
+        _input: Arc<dyn ExecutionPlan>,
+        _state: &SessionState,
+        _conf: FileSinkConfig,
+    ) -> Result<Arc<dyn ExecutionPlan>> {
+        let msg = "Writer not implemented for this format".to_owned();
+        Err(DataFusionError::NotImplemented(msg))
+    }
+}
+
+/// `AsyncPutWriter` is an object that facilitates asynchronous writing to 
object stores.
+/// It is specifically designed for the `object_store` crate's `put` method 
and sends
+/// whole bytes at once when the buffer is flushed.
+pub struct AsyncPutWriter {
+    /// Object metadata
+    object_meta: ObjectMeta,
+    /// A shared reference to the object store
+    store: Arc<dyn ObjectStore>,
+    /// A buffer that stores the bytes to be sent
+    current_buffer: Vec<u8>,
+    /// Used for async handling in flush method
+    inner_state: AsyncPutState,
+}
+
+impl AsyncPutWriter {
+    /// Constructor for the `AsyncPutWriter` object
+    pub fn new(object_meta: ObjectMeta, store: Arc<dyn ObjectStore>) -> Self {
+        Self {
+            object_meta,
+            store,
+            current_buffer: vec![],
+            // The writer starts out in buffering mode
+            inner_state: AsyncPutState::Buffer,
+        }
+    }
+
+    /// Separate implementation function that unpins the [`AsyncPutWriter`] so
+    /// that partial borrows work correctly
+    fn poll_shutdown_inner(
+        &mut self,
+        cx: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        loop {
+            match &mut self.inner_state {
+                AsyncPutState::Buffer => {
+                    // Convert the current buffer to bytes and take ownership 
of it
+                    let bytes = Bytes::from(mem::take(&mut 
self.current_buffer));
+                    // Set the inner state to Put variant with the bytes
+                    self.inner_state = AsyncPutState::Put { bytes }
+                }
+                AsyncPutState::Put { bytes } => {
+                    // Send the bytes to the object store's put method
+                    return Poll::Ready(
+                        ready!(self
+                            .store
+                            .put(&self.object_meta.location, bytes.clone())
+                            .poll_unpin(cx))
+                        .map_err(Error::from),
+                    );
+                }
+            }
+        }
+    }
+}
+
+/// An enum that represents the inner state of AsyncPut
+enum AsyncPutState {
+    /// Building Bytes struct in this state
+    Buffer,
+    /// Data in the buffer is being sent to the object store
+    Put { bytes: Bytes },
+}
+
+impl AsyncWrite for AsyncPutWriter {
+    // Define the implementation of the AsyncWrite trait for the 
`AsyncPutWriter` struct
+    fn poll_write(
+        mut self: Pin<&mut Self>,
+        _: &mut Context<'_>,
+        buf: &[u8],
+    ) -> Poll<std::result::Result<usize, Error>> {
+        // Extend the current buffer with the incoming buffer
+        self.current_buffer.extend_from_slice(buf);
+        // Return a ready poll with the length of the incoming buffer
+        Poll::Ready(Ok(buf.len()))
+    }
+
+    fn poll_flush(
+        self: Pin<&mut Self>,
+        _: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        // Return a ready poll with an empty result
+        Poll::Ready(Ok(()))
+    }
+
+    fn poll_shutdown(
+        mut self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        // Call the poll_shutdown_inner method to handle the actual sending of 
data to the object store
+        self.poll_shutdown_inner(cx)
+    }
+}
+
+/// A simple wrapper around an `AsyncWrite` type that implements 
`FileWriterExt`.
+pub struct AsyncPut<W: AsyncWrite + Unpin + Send> {
+    writer: W,
+}
+
+impl<W: AsyncWrite + Unpin + Send> AsyncPut<W> {
+    /// Create a new `AsyncPut` instance with the given writer.
+    pub fn new(writer: W) -> Self {
+        Self { writer }
+    }
+}
+
+impl<W: AsyncWrite + Unpin + Send> AsyncWrite for AsyncPut<W> {
+    fn poll_write(
+        self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+        buf: &[u8],
+    ) -> Poll<std::result::Result<usize, Error>> {
+        Pin::new(&mut self.get_mut().writer).poll_write(cx, buf)
+    }
+
+    fn poll_flush(
+        self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        Pin::new(&mut self.get_mut().writer).poll_flush(cx)
+    }
+
+    fn poll_shutdown(
+        self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        Pin::new(&mut self.get_mut().writer).poll_shutdown(cx)
+    }
+}
+
+impl<W: AsyncWrite + Unpin + Send> FileWriterExt for AsyncPut<W> {}
+
+/// An extension trait for `AsyncWrite` types that adds an `abort_writer` 
method.

Review Comment:
   What do you think about of instead of using a trait to add abort, just 
having a struct like
   
   ```
   struct AbortableWrite<W> {
       write: W,
       abort: Option<Box<dyn FnOnce() -> BoxFuture<'static, Result<()>>>>
   }
   ```



##########
datafusion/core/src/datasource/file_format/mod.rs:
##########
@@ -87,6 +98,277 @@ pub trait FileFormat: Send + Sync + fmt::Debug {
         conf: FileScanConfig,
         filters: Option<&Arc<dyn PhysicalExpr>>,
     ) -> Result<Arc<dyn ExecutionPlan>>;
+
+    /// Take a list of files and the configuration to convert it to the
+    /// appropriate writer executor according to this file format.
+    async fn create_writer_physical_plan(
+        &self,
+        _input: Arc<dyn ExecutionPlan>,
+        _state: &SessionState,
+        _conf: FileSinkConfig,
+    ) -> Result<Arc<dyn ExecutionPlan>> {
+        let msg = "Writer not implemented for this format".to_owned();
+        Err(DataFusionError::NotImplemented(msg))
+    }
+}
+
+/// `AsyncPutWriter` is an object that facilitates asynchronous writing to 
object stores.
+/// It is specifically designed for the `object_store` crate's `put` method 
and sends
+/// whole bytes at once when the buffer is flushed.
+pub struct AsyncPutWriter {
+    /// Object metadata
+    object_meta: ObjectMeta,
+    /// A shared reference to the object store
+    store: Arc<dyn ObjectStore>,
+    /// A buffer that stores the bytes to be sent
+    current_buffer: Vec<u8>,
+    /// Used for async handling in flush method
+    inner_state: AsyncPutState,
+}
+
+impl AsyncPutWriter {
+    /// Constructor for the `AsyncPutWriter` object
+    pub fn new(object_meta: ObjectMeta, store: Arc<dyn ObjectStore>) -> Self {
+        Self {
+            object_meta,
+            store,
+            current_buffer: vec![],
+            // The writer starts out in buffering mode
+            inner_state: AsyncPutState::Buffer,
+        }
+    }
+
+    /// Separate implementation function that unpins the [`AsyncPutWriter`] so
+    /// that partial borrows work correctly
+    fn poll_shutdown_inner(
+        &mut self,
+        cx: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        loop {
+            match &mut self.inner_state {
+                AsyncPutState::Buffer => {
+                    // Convert the current buffer to bytes and take ownership 
of it
+                    let bytes = Bytes::from(mem::take(&mut 
self.current_buffer));
+                    // Set the inner state to Put variant with the bytes
+                    self.inner_state = AsyncPutState::Put { bytes }
+                }
+                AsyncPutState::Put { bytes } => {
+                    // Send the bytes to the object store's put method
+                    return Poll::Ready(
+                        ready!(self
+                            .store
+                            .put(&self.object_meta.location, bytes.clone())
+                            .poll_unpin(cx))
+                        .map_err(Error::from),
+                    );
+                }
+            }
+        }
+    }
+}
+
+/// An enum that represents the inner state of AsyncPut
+enum AsyncPutState {
+    /// Building Bytes struct in this state
+    Buffer,
+    /// Data in the buffer is being sent to the object store
+    Put { bytes: Bytes },
+}
+
+impl AsyncWrite for AsyncPutWriter {
+    // Define the implementation of the AsyncWrite trait for the 
`AsyncPutWriter` struct
+    fn poll_write(
+        mut self: Pin<&mut Self>,
+        _: &mut Context<'_>,
+        buf: &[u8],
+    ) -> Poll<std::result::Result<usize, Error>> {
+        // Extend the current buffer with the incoming buffer
+        self.current_buffer.extend_from_slice(buf);
+        // Return a ready poll with the length of the incoming buffer
+        Poll::Ready(Ok(buf.len()))
+    }
+
+    fn poll_flush(
+        self: Pin<&mut Self>,
+        _: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        // Return a ready poll with an empty result
+        Poll::Ready(Ok(()))
+    }
+
+    fn poll_shutdown(
+        mut self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        // Call the poll_shutdown_inner method to handle the actual sending of 
data to the object store
+        self.poll_shutdown_inner(cx)
+    }
+}
+
+/// A simple wrapper around an `AsyncWrite` type that implements 
`FileWriterExt`.
+pub struct AsyncPut<W: AsyncWrite + Unpin + Send> {
+    writer: W,
+}
+
+impl<W: AsyncWrite + Unpin + Send> AsyncPut<W> {
+    /// Create a new `AsyncPut` instance with the given writer.
+    pub fn new(writer: W) -> Self {
+        Self { writer }
+    }
+}
+
+impl<W: AsyncWrite + Unpin + Send> AsyncWrite for AsyncPut<W> {
+    fn poll_write(
+        self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+        buf: &[u8],
+    ) -> Poll<std::result::Result<usize, Error>> {
+        Pin::new(&mut self.get_mut().writer).poll_write(cx, buf)
+    }
+
+    fn poll_flush(
+        self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        Pin::new(&mut self.get_mut().writer).poll_flush(cx)
+    }
+
+    fn poll_shutdown(
+        self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        Pin::new(&mut self.get_mut().writer).poll_shutdown(cx)
+    }
+}
+
+impl<W: AsyncWrite + Unpin + Send> FileWriterExt for AsyncPut<W> {}
+
+/// An extension trait for `AsyncWrite` types that adds an `abort_writer` 
method.
+pub trait FileWriterExt: AsyncWrite + Unpin + Send {
+    /// Aborts the writer and returns a boxed future with the result.
+    /// The default implementation returns an immediately resolved future with 
`Ok(())`.
+    fn abort_writer(&self) -> Result<BoxFuture<'static, Result<()>>> {
+        Ok(async { Ok(()) }.boxed())
+    }
+}
+
+/// A wrapper around an `AsyncWrite` type that provides multipart upload 
functionality.
+pub struct AsyncPutMultipart<W: AsyncWrite + Unpin + Send> {
+    writer: W,
+    /// A shared reference to the object store
+    store: Arc<dyn ObjectStore>,
+    multipart_id: MultipartId,
+    location: Path,
+}
+
+impl<W: AsyncWrite + Unpin + Send> AsyncPutMultipart<W> {
+    /// Create a new `AsyncPutMultipart` instance with the given writer, 
object store, multipart ID, and location.
+    pub fn new(
+        writer: W,
+        store: Arc<dyn ObjectStore>,
+        multipart_id: MultipartId,
+        location: Path,
+    ) -> Self {
+        Self {
+            writer,
+            store,
+            multipart_id,
+            location,
+        }
+    }
+}
+
+impl<W: AsyncWrite + Unpin + Send> AsyncWrite for AsyncPutMultipart<W> {
+    fn poll_write(
+        self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+        buf: &[u8],
+    ) -> Poll<std::result::Result<usize, Error>> {
+        Pin::new(&mut self.get_mut().writer).poll_write(cx, buf)
+    }
+
+    fn poll_flush(
+        self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        Pin::new(&mut self.get_mut().writer).poll_flush(cx)
+    }
+
+    fn poll_shutdown(
+        self: Pin<&mut Self>,
+        cx: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        Pin::new(&mut self.get_mut().writer).poll_shutdown(cx)
+    }
+}
+
+impl<W: AsyncWrite + Unpin + Send> FileWriterExt for AsyncPutMultipart<W> {
+    fn abort_writer(&self) -> Result<BoxFuture<'static, Result<()>>> {
+        let location = self.location.clone();
+        let multipart_id = self.multipart_id.clone();
+        let store = self.store.clone();
+        Ok(Box::pin(async move {
+            store
+                .abort_multipart(&location, &multipart_id)
+                .await
+                .map_err(DataFusionError::ObjectStore)
+        }))
+    }
+}
+
+/// A wrapper around an `AsyncWrite` type that provides append functionality.
+pub struct AsyncAppend<W: AsyncWrite + Unpin + Send> {

Review Comment:
   I can't help feeling `abort` for `AsyncAppend` should return an error, you 
can't abort an append. Currently I think this will return `Ok` for an abort, 
silently leaving the file in a potentially undefined state



##########
datafusion/core/src/datasource/file_format/csv.rs:
##########
@@ -324,6 +349,196 @@ fn build_schema_helper(names: Vec<String>, types: 
&[HashSet<DataType>]) -> Schem
     Schema::new(fields)
 }
 
+impl Default for CsvSerializer {
+    fn default() -> Self {
+        Self::new()
+    }
+}
+
+/// Define a struct for serializing CSV records to a stream
+pub struct CsvSerializer {
+    // CSV writer builder
+    builder: WriterBuilder,
+    // Inner buffer for avoiding reallocation
+    buffer: Vec<u8>,
+    // Flag to indicate whether there will be a header
+    header: bool,
+}
+
+impl CsvSerializer {
+    /// Constructor for the CsvSerializer object
+    pub fn new() -> Self {
+        Self {
+            builder: WriterBuilder::new(),
+            header: true,
+            buffer: Vec::with_capacity(4096),
+        }
+    }
+
+    /// Method for setting the CSV writer builder
+    pub fn with_builder(mut self, builder: WriterBuilder) -> Self {
+        self.builder = builder;
+        self
+    }
+
+    /// Method for setting the CSV writer header status
+    pub fn with_header(mut self, header: bool) -> Self {
+        self.header = header;
+        self
+    }
+}
+
+#[async_trait]
+impl BatchSerializer for CsvSerializer {
+    async fn serialize(&mut self, batch: RecordBatch) -> Result<Bytes> {
+        let builder = self.builder.clone();
+        let mut writer = builder.has_headers(self.header).build(&mut 
self.buffer);
+        writer.write(&batch)?;
+        drop(writer);
+        self.header = false;
+        Ok(Bytes::from(self.buffer.drain(..).collect::<Vec<u8>>()))
+    }
+}
+
+async fn check_for_errors<T>(
+    result: Result<T>,
+    writers: &mut [Box<dyn FileWriterExt>],
+) -> Result<T> {
+    match result {
+        Ok(value) => Ok(value),
+        Err(e) => {
+            // Abort all writers before returning the error:
+            for writer in writers {
+                let mut abort_future = writer.abort_writer();
+                if let Ok(abort_future) = &mut abort_future {
+                    let _ = abort_future.await;
+                }
+                // Ignore errors that occur during abortion,
+                // We do try to abort all writers before returning error.
+            }
+            // After aborting writers return original error.
+            Err(e)
+        }
+    }
+}
+
+/// Implements [`DataSink`] for writing to a CSV file.
+struct CsvSink {
+    /// Config options for writing data
+    config: FileSinkConfig,
+    has_header: bool,
+    delimiter: u8,
+    file_compression_type: FileCompressionType,
+}
+
+impl Debug for CsvSink {
+    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+        f.debug_struct("CsvSink")
+            .field("has_header", &self.has_header)
+            .field("delimiter", &self.delimiter)
+            .field("file_compression_type", &self.file_compression_type)
+            .finish()
+    }
+}
+
+impl Display for CsvSink {
+    fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
+        write!(
+            f,
+            "CsvSink(writer_mode={:?}, file_groups={})",
+            self.config.writer_mode,
+            FileGroupDisplay(&self.config.file_groups),
+        )
+    }
+}
+
+impl CsvSink {
+    fn new(
+        config: FileSinkConfig,
+        has_header: bool,
+        delimiter: u8,
+        file_compression_type: FileCompressionType,
+    ) -> Self {
+        Self {
+            config,
+            has_header,
+            delimiter,
+            file_compression_type,
+        }
+    }
+}
+
+#[async_trait]
+impl DataSink for CsvSink {
+    async fn write_all(
+        &self,
+        mut data: SendableRecordBatchStream,
+        context: &Arc<TaskContext>,
+    ) -> Result<u64> {
+        let num_partitions = self.config.file_groups.len();
+
+        let object_store = context
+            .runtime_env()
+            .object_store(&self.config.object_store_url)?;
+        let opener = CsvWriterOpener::new(
+            self.config.writer_mode,
+            object_store,
+            self.file_compression_type.to_owned(),
+        );
+
+        // Construct serializer and writer for each file group
+        let mut serializers = vec![];
+        let mut writers = vec![];
+        for file_group in &self.config.file_groups {
+            let header = self.has_header
+                && (!matches!(&self.config.writer_mode, FileWriterMode::Append)
+                    || file_group.object_meta.size == 0);

Review Comment:
   For put and put multipart we should write the header regardless of any 
existing contents (as they will be overwritten)?



##########
datafusion/core/src/physical_plan/file_format/csv.rs:
##########
@@ -323,6 +328,78 @@ impl FileOpener for CsvOpener {
     }
 }
 
+/// A [`FileWriterExt`] that opens a CSV file and yields a [`Box<dyn 
AsyncWrite + Unpin + Send>`]
+pub struct CsvWriterOpener {
+    writer_mode: FileWriterMode,
+    object_store: Arc<dyn ObjectStore>,
+    file_compression_type: FileCompressionType,
+}
+
+impl CsvWriterOpener {
+    /// Creates writer/opener for CSV files.
+    pub fn new(
+        writer_mode: FileWriterMode,
+        object_store: Arc<dyn ObjectStore>,
+        file_compression_type: FileCompressionType,
+    ) -> Self {
+        Self {
+            writer_mode,
+            object_store,
+            file_compression_type,
+        }
+    }
+}
+
+#[async_trait]
+impl FileWriterFactory for CsvWriterOpener {
+    async fn create_writer(&self, file_meta: FileMeta) -> Result<Box<dyn 
FileWriterExt>> {
+        let object = &file_meta.object_meta;
+        match self.writer_mode {
+            // If the mode is append, call the store's append method and 
return a ready poll
+            // with the result wrapped in a custom error type if it fails

Review Comment:
   The docs appear to be out of date here



##########
datafusion/core/src/physical_plan/file_format/mod.rs:
##########
@@ -238,6 +239,30 @@ impl FileScanConfig {
     }
 }
 
+/// The base configurations to provide when creating a physical plan for
+/// writing to any given file format.
+#[derive(Debug, Clone)]
+pub struct FileSinkConfig {

Review Comment:
   :+1:



##########
datafusion/core/src/datasource/file_format/mod.rs:
##########
@@ -87,6 +98,277 @@ pub trait FileFormat: Send + Sync + fmt::Debug {
         conf: FileScanConfig,
         filters: Option<&Arc<dyn PhysicalExpr>>,
     ) -> Result<Arc<dyn ExecutionPlan>>;
+
+    /// Take a list of files and the configuration to convert it to the
+    /// appropriate writer executor according to this file format.
+    async fn create_writer_physical_plan(
+        &self,
+        _input: Arc<dyn ExecutionPlan>,
+        _state: &SessionState,
+        _conf: FileSinkConfig,
+    ) -> Result<Arc<dyn ExecutionPlan>> {
+        let msg = "Writer not implemented for this format".to_owned();
+        Err(DataFusionError::NotImplemented(msg))
+    }
+}
+
+/// `AsyncPutWriter` is an object that facilitates asynchronous writing to 
object stores.
+/// It is specifically designed for the `object_store` crate's `put` method 
and sends
+/// whole bytes at once when the buffer is flushed.
+pub struct AsyncPutWriter {
+    /// Object metadata
+    object_meta: ObjectMeta,
+    /// A shared reference to the object store
+    store: Arc<dyn ObjectStore>,
+    /// A buffer that stores the bytes to be sent
+    current_buffer: Vec<u8>,
+    /// Used for async handling in flush method
+    inner_state: AsyncPutState,
+}
+
+impl AsyncPutWriter {
+    /// Constructor for the `AsyncPutWriter` object
+    pub fn new(object_meta: ObjectMeta, store: Arc<dyn ObjectStore>) -> Self {
+        Self {
+            object_meta,
+            store,
+            current_buffer: vec![],
+            // The writer starts out in buffering mode
+            inner_state: AsyncPutState::Buffer,
+        }
+    }
+
+    /// Separate implementation function that unpins the [`AsyncPutWriter`] so
+    /// that partial borrows work correctly
+    fn poll_shutdown_inner(
+        &mut self,
+        cx: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        loop {
+            match &mut self.inner_state {
+                AsyncPutState::Buffer => {
+                    // Convert the current buffer to bytes and take ownership 
of it
+                    let bytes = Bytes::from(mem::take(&mut 
self.current_buffer));
+                    // Set the inner state to Put variant with the bytes
+                    self.inner_state = AsyncPutState::Put { bytes }
+                }
+                AsyncPutState::Put { bytes } => {
+                    // Send the bytes to the object store's put method
+                    return Poll::Ready(
+                        ready!(self
+                            .store
+                            .put(&self.object_meta.location, bytes.clone())
+                            .poll_unpin(cx))
+                        .map_err(Error::from),
+                    );
+                }
+            }
+        }
+    }
+}
+
+/// An enum that represents the inner state of AsyncPut
+enum AsyncPutState {
+    /// Building Bytes struct in this state
+    Buffer,
+    /// Data in the buffer is being sent to the object store
+    Put { bytes: Bytes },
+}
+
+impl AsyncWrite for AsyncPutWriter {
+    // Define the implementation of the AsyncWrite trait for the 
`AsyncPutWriter` struct
+    fn poll_write(
+        mut self: Pin<&mut Self>,
+        _: &mut Context<'_>,
+        buf: &[u8],
+    ) -> Poll<std::result::Result<usize, Error>> {
+        // Extend the current buffer with the incoming buffer
+        self.current_buffer.extend_from_slice(buf);
+        // Return a ready poll with the length of the incoming buffer
+        Poll::Ready(Ok(buf.len()))
+    }
+
+    fn poll_flush(
+        self: Pin<&mut Self>,
+        _: &mut Context<'_>,
+    ) -> Poll<std::result::Result<(), Error>> {
+        // Return a ready poll with an empty result

Review Comment:
   This is the same slightly dubious thing we have to do in `ObjectStore` to 
map multipart upload to these traits, as the traits make the problematic 
assumption that you can always flush. We've been doing this for a while and it 
has never caused any issues, so I think it is fine.



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to