This is an automated email from the ASF dual-hosted git repository.

xuanwo pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/opendal.git


The following commit(s) were added to refs/heads/main by this push:
     new 1b47d42b2 feat(bindings/nodejs): Add WriteOptions support for new 
options API (#6322)
1b47d42b2 is described below

commit 1b47d42b28f652b3a72712735b8008f0465f3074
Author: Kingsword <kingswor...@gmail.com>
AuthorDate: Fri Jul 11 16:33:59 2025 +0800

    feat(bindings/nodejs): Add WriteOptions support for new options API (#6322)
    
    * feat(bindings/nodejs): Add WriteOptions support for new options API
    
    * chore: merge
---
 bindings/nodejs/generated.d.ts                     |  90 +++++++--
 bindings/nodejs/src/capability.rs                  |  30 +++
 bindings/nodejs/src/lib.rs                         | 118 +++++------
 bindings/nodejs/src/options.rs                     |  95 +++++++++
 .../tests/suites/asyncWriteOptions.suite.mjs       | 215 +++++++++++++++++++++
 bindings/nodejs/tests/suites/index.mjs             |   4 +
 .../nodejs/tests/suites/syncWriteOptions.suite.mjs | 215 +++++++++++++++++++++
 7 files changed, 679 insertions(+), 88 deletions(-)

diff --git a/bindings/nodejs/generated.d.ts b/bindings/nodejs/generated.d.ts
index 82d4ad1e5..5fb411099 100644
--- a/bindings/nodejs/generated.d.ts
+++ b/bindings/nodejs/generated.d.ts
@@ -269,17 +269,6 @@ export interface ListOptions {
   */
   deleted?: boolean
 }
-export interface DeleteOptions {
-  version?: string
-}
-export const enum EntryMode {
-  /** FILE means the path has data to read. */
-  FILE = 0,
-  /** DIR means the path can be listed. */
-  DIR = 1,
-  /** Unknown means we don't know what we can do on this path. */
-  Unknown = 2
-}
 export interface WriteOptions {
   /**
    * Append bytes into a path.
@@ -308,6 +297,65 @@ export interface WriteOptions {
   contentDisposition?: string
   /** Set the 
[Cache-Control](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Cache-Control)
 of op. */
   cacheControl?: string
+  /** Set the [Content-Encoding] 
https://developer.mozilla.org/en-US/docs/Web/HTTP/Reference/Headers/Content-Encoding
 of op. */
+  contentEncoding?: string
+  /**
+   * Sets user metadata of op.
+   *
+   * If chunk is set, the user metadata will be attached to the object during 
write.
+   *
+   * ## NOTE
+   *
+   * - Services may have limitations for user metadata, for example:
+   *   - Key length is typically limited (e.g., 1024 bytes)
+   *   - Value length is typically limited (e.g., 4096 bytes)
+   *   - Total metadata size might be limited
+   *   - Some characters might be forbidden in keys
+   */
+  userMetadata?: Record<string, string>
+  /**
+   * Sets if-match condition of op.
+   *
+   * This operation provides conditional write functionality based on ETag 
matching,
+   * helping prevent unintended overwrites in concurrent scenarios.
+   */
+  ifMatch?: string
+  /**
+   * Sets if-none-match condition of op.
+   *
+   * This operation provides conditional write functionality based on ETag 
non-matching,
+   * useful for preventing overwriting existing resources or ensuring unique 
writes.
+   */
+  ifNoneMatch?: string
+  /**
+   * Sets if_not_exists condition of op.
+   *
+   * This operation provides a way to ensure write operations only create new 
resources
+   * without overwriting existing ones, useful for implementing "create if not 
exists" logic.
+   */
+  ifNotExists?: boolean
+  /**
+   * Sets concurrent of op.
+   *
+   * - By default, OpenDAL writes files sequentially
+   * - When concurrent is set:
+   *   - Multiple write operations can execute in parallel
+   *   - Write operations return immediately without waiting if tasks space 
are available
+   *   - Close operation ensures all writes complete in order
+   *   - Memory usage increases with concurrency level
+   */
+  concurrent?: number
+}
+export interface DeleteOptions {
+  version?: string
+}
+export const enum EntryMode {
+  /** FILE means the path has data to read. */
+  FILE = 0,
+  /** DIR means the path can be listed. */
+  DIR = 1,
+  /** Unknown means we don't know what we can do on this path. */
+  Unknown = 2
 }
 /** PresignedRequest is a presigned request return by `presign`. */
 export interface PresignedRequest {
@@ -381,6 +429,16 @@ export class Capability {
   get writeWithContentDisposition(): boolean
   /** If operator supports write with cache control. */
   get writeWithCacheControl(): boolean
+  /** If operator supports write with content encoding. */
+  get writeWithContentEncoding(): boolean
+  /** If operator supports write with user metadata. */
+  get writeWithUserMetadata(): boolean
+  /** If operator supports write with if match. */
+  get writeWithIfMatch(): boolean
+  /** If operator supports write with if none match. */
+  get writeWithIfNoneMatch(): boolean
+  /** If operator supports write with if not exists. */
+  get writeWithIfNotExists(): boolean
   /**
    * write_multi_max_size is the max size that services support in write_multi.
    *
@@ -572,7 +630,7 @@ export class Operator {
    * await op.write("path/to/file", Buffer.from("hello world"), { contentType: 
"text/plain" });
    * ```
    */
-  write(path: string, content: Buffer | string, options?: WriteOptions | 
undefined | null): Promise<void>
+  write(path: string, content: Buffer | string, options?: WriteOptions | 
undefined | null): Promise<Metadata>
   /**
    * Write multiple bytes into a path.
    *
@@ -597,7 +655,7 @@ export class Operator {
    * op.writeSync("path/to/file", Buffer.from("hello world"), { contentType: 
"text/plain" });
    * ```
    */
-  writeSync(path: string, content: Buffer | string, options?: WriteOptions | 
undefined | null): void
+  writeSync(path: string, content: Buffer | string, options?: WriteOptions | 
undefined | null): Metadata
   /**
    * Copy file according to given `from` and `to` path.
    *
@@ -845,14 +903,20 @@ export class Metadata {
    * deletion or has been permanently deleted.
    */
   isDeleted(): boolean
+  /** Cache-Control of this object. */
+  get cacheControl(): string | null
   /** Content-Disposition of this object */
   get contentDisposition(): string | null
   /** Content Length of this object */
   get contentLength(): bigint | null
+  /** Content Encoding of this object */
+  get contentEncoding(): string | null
   /** Content MD5 of this object. */
   get contentMd5(): string | null
   /** Content Type of this object. */
   get contentType(): string | null
+  /** User Metadata of this object. */
+  get userMetadata(): Record<string, string> | null
   /** ETag of this object. */
   get etag(): string | null
   /**
diff --git a/bindings/nodejs/src/capability.rs 
b/bindings/nodejs/src/capability.rs
index e77a5787a..742cae94e 100644
--- a/bindings/nodejs/src/capability.rs
+++ b/bindings/nodejs/src/capability.rs
@@ -186,6 +186,36 @@ impl Capability {
         self.0.write_with_cache_control
     }
 
+    /// If operator supports write with content encoding.
+    #[napi(getter)]
+    pub fn write_with_content_encoding(&self) -> bool {
+        self.0.write_with_content_encoding
+    }
+
+    /// If operator supports write with user metadata.
+    #[napi(getter)]
+    pub fn write_with_user_metadata(&self) -> bool {
+        self.0.write_with_user_metadata
+    }
+
+    /// If operator supports write with if match.
+    #[napi(getter)]
+    pub fn write_with_if_match(&self) -> bool {
+        self.0.write_with_if_match
+    }
+
+    /// If operator supports write with if none match.
+    #[napi(getter)]
+    pub fn write_with_if_none_match(&self) -> bool {
+        self.0.write_with_if_none_match
+    }
+
+    /// If operator supports write with if not exists.
+    #[napi(getter)]
+    pub fn write_with_if_not_exists(&self) -> bool {
+        self.0.write_with_if_not_exists
+    }
+
     /// write_multi_max_size is the max size that services support in 
write_multi.
     ///
     /// For example, AWS S3 supports 5GiB as max in write_multi.
diff --git a/bindings/nodejs/src/lib.rs b/bindings/nodejs/src/lib.rs
index cc862194d..d0961ef22 100644
--- a/bindings/nodejs/src/lib.rs
+++ b/bindings/nodejs/src/lib.rs
@@ -27,7 +27,9 @@ use std::time::Duration;
 use futures::AsyncReadExt;
 use futures::TryStreamExt;
 use napi::bindgen_prelude::*;
-use opendal::options::{DeleteOptions, ListOptions, ReadOptions, ReaderOptions, 
StatOptions};
+use opendal::options::{
+    DeleteOptions, ListOptions, ReadOptions, ReaderOptions, StatOptions, 
WriteOptions,
+};
 
 mod capability;
 mod options;
@@ -304,31 +306,19 @@ impl Operator {
         &self,
         path: String,
         content: Either<Buffer, String>,
-        options: Option<WriteOptions>,
-    ) -> Result<()> {
+        options: Option<options::WriteOptions>,
+    ) -> Result<Metadata> {
         let c = match content {
             Either::A(buf) => buf.as_ref().to_owned(),
             Either::B(s) => s.into_bytes(),
         };
-        let mut writer = self.async_op.write_with(&path, c);
-        if let Some(options) = options {
-            if let Some(append) = options.append {
-                writer = writer.append(append);
-            }
-            if let Some(chunk) = options.chunk {
-                writer = writer.chunk(chunk.get_u64().1 as usize);
-            }
-            if let Some(ref content_type) = options.content_type {
-                writer = writer.content_type(content_type);
-            }
-            if let Some(ref content_disposition) = options.content_disposition 
{
-                writer = writer.content_disposition(content_disposition);
-            }
-            if let Some(ref cache_control) = options.cache_control {
-                writer = writer.cache_control(cache_control);
-            }
-        }
-        writer.await.map(|_| ()).map_err(format_napi_error)
+        let options = options.map_or_else(WriteOptions::default, 
WriteOptions::from);
+        let metadata = self
+            .async_op
+            .write_options(&path, c, options)
+            .await
+            .map_err(format_napi_error)?;
+        Ok(Metadata(metadata))
     }
 
     //noinspection DuplicatedCode
@@ -336,7 +326,11 @@ impl Operator {
     ///
     /// It could be used to write large file in a streaming way.
     #[napi]
-    pub async fn writer(&self, path: String, options: Option<WriteOptions>) -> 
Result<Writer> {
+    pub async fn writer(
+        &self,
+        path: String,
+        options: Option<options::WriteOptions>,
+    ) -> Result<Writer> {
         let options = options.unwrap_or_default();
         let writer = self
             .async_op
@@ -353,7 +347,7 @@ impl Operator {
     pub fn writer_sync(
         &self,
         path: String,
-        options: Option<WriteOptions>,
+        options: Option<options::WriteOptions>,
     ) -> Result<BlockingWriter> {
         let options = options.unwrap_or_default();
         let writer = self
@@ -379,17 +373,18 @@ impl Operator {
         &self,
         path: String,
         content: Either<Buffer, String>,
-        options: Option<WriteOptions>,
-    ) -> Result<()> {
+        options: Option<options::WriteOptions>,
+    ) -> Result<Metadata> {
         let c = match content {
             Either::A(buf) => buf.as_ref().to_owned(),
             Either::B(s) => s.into_bytes(),
         };
-        let options = options.unwrap_or_default();
-        self.blocking_op
-            .write_options(&path, c, options.into())
+        let options = options.map_or_else(WriteOptions::default, 
WriteOptions::from);
+        let metadata = self
+            .blocking_op
+            .write_options(&path, c, options)
             .map_err(format_napi_error)?;
-        Ok(())
+        Ok(Metadata(metadata))
     }
 
     /// Copy file according to given `from` and `to` path.
@@ -778,6 +773,12 @@ impl Metadata {
         self.0.is_deleted()
     }
 
+    /// Cache-Control of this object.
+    #[napi(getter)]
+    pub fn cache_control(&self) -> Option<String> {
+        self.0.cache_control().map(|s| s.to_string())
+    }
+
     /// Content-Disposition of this object
     #[napi(getter)]
     pub fn content_disposition(&self) -> Option<String> {
@@ -790,6 +791,12 @@ impl Metadata {
         self.0.content_length().into()
     }
 
+    /// Content Encoding of this object
+    #[napi(getter)]
+    pub fn content_encoding(&self) -> Option<String> {
+        self.0.content_encoding().map(|s| s.to_string())
+    }
+
     /// Content MD5 of this object.
     #[napi(getter)]
     pub fn content_md5(&self) -> Option<String> {
@@ -802,6 +809,12 @@ impl Metadata {
         self.0.content_type().map(|s| s.to_string())
     }
 
+    /// User Metadata of this object.
+    #[napi(getter)]
+    pub fn user_metadata(&self) -> Option<HashMap<String, String>> {
+        self.0.user_metadata().cloned()
+    }
+
     /// ETag of this object.
     #[napi(getter)]
     pub fn etag(&self) -> Option<String> {
@@ -961,51 +974,6 @@ impl Writer {
     }
 }
 
-#[napi(object)]
-#[derive(Default)]
-pub struct WriteOptions {
-    /// Append bytes into a path.
-    ///
-    /// ### Notes
-    ///
-    /// - It always appends content to the end of the file.
-    /// - It will create file if the path does not exist.
-    pub append: Option<bool>,
-
-    /// Set the chunk of op.
-    ///
-    /// If chunk is set, the data will be chunked by the underlying writer.
-    ///
-    /// ## NOTE
-    ///
-    /// A service could have their own minimum chunk size while perform write
-    /// operations like multipart uploads. So the chunk size may be larger than
-    /// the given buffer size.
-    pub chunk: Option<BigInt>,
-
-    /// Set the 
[Content-Type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type)
 of op.
-    pub content_type: Option<String>,
-
-    /// Set the 
[Content-Disposition](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Disposition)
 of op.
-    pub content_disposition: Option<String>,
-
-    /// Set the 
[Cache-Control](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Cache-Control)
 of op.
-    pub cache_control: Option<String>,
-}
-
-impl From<WriteOptions> for opendal::options::WriteOptions {
-    fn from(value: WriteOptions) -> Self {
-        Self {
-            append: value.append.unwrap_or_default(),
-            chunk: value.chunk.map(|v| v.get_u64().1 as usize),
-            content_type: value.content_type,
-            content_disposition: value.content_disposition,
-            cache_control: value.cache_control,
-            ..Default::default()
-        }
-    }
-}
-
 /// Lister is designed to list entries at a given path in an asynchronous
 /// manner.
 #[napi]
diff --git a/bindings/nodejs/src/options.rs b/bindings/nodejs/src/options.rs
index 994f0d74b..f7dd5d149 100644
--- a/bindings/nodejs/src/options.rs
+++ b/bindings/nodejs/src/options.rs
@@ -17,6 +17,7 @@
 
 use napi::bindgen_prelude::BigInt;
 use opendal::raw::{parse_datetime_from_rfc3339, BytesRange};
+use std::collections::HashMap;
 
 #[napi(object)]
 #[derive(Debug)]
@@ -392,6 +393,100 @@ impl From<ListOptions> for opendal::options::ListOptions {
     }
 }
 
+#[napi(object)]
+#[derive(Default, Debug)]
+pub struct WriteOptions {
+    /// Append bytes into a path.
+    ///
+    /// ### Notes
+    ///
+    /// - It always appends content to the end of the file.
+    /// - It will create file if the path does not exist.
+    pub append: Option<bool>,
+
+    /// Set the chunk of op.
+    ///
+    /// If chunk is set, the data will be chunked by the underlying writer.
+    ///
+    /// ## NOTE
+    ///
+    /// A service could have their own minimum chunk size while perform write
+    /// operations like multipart uploads. So the chunk size may be larger than
+    /// the given buffer size.
+    pub chunk: Option<BigInt>,
+
+    /// Set the 
[Content-Type](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Type)
 of op.
+    pub content_type: Option<String>,
+
+    /// Set the 
[Content-Disposition](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Content-Disposition)
 of op.
+    pub content_disposition: Option<String>,
+
+    /// Set the 
[Cache-Control](https://developer.mozilla.org/en-US/docs/Web/HTTP/Headers/Cache-Control)
 of op.
+    pub cache_control: Option<String>,
+
+    /// Set the [Content-Encoding] 
https://developer.mozilla.org/en-US/docs/Web/HTTP/Reference/Headers/Content-Encoding
 of op.
+    pub content_encoding: Option<String>,
+
+    /// Sets user metadata of op.
+    ///
+    /// If chunk is set, the user metadata will be attached to the object 
during write.
+    ///
+    /// ## NOTE
+    ///
+    /// - Services may have limitations for user metadata, for example:
+    ///   - Key length is typically limited (e.g., 1024 bytes)
+    ///   - Value length is typically limited (e.g., 4096 bytes)
+    ///   - Total metadata size might be limited
+    ///   - Some characters might be forbidden in keys
+    pub user_metadata: Option<HashMap<String, String>>,
+
+    /// Sets if-match condition of op.
+    ///
+    /// This operation provides conditional write functionality based on ETag 
matching,
+    /// helping prevent unintended overwrites in concurrent scenarios.
+    pub if_match: Option<String>,
+
+    /// Sets if-none-match condition of op.
+    ///
+    /// This operation provides conditional write functionality based on ETag 
non-matching,
+    /// useful for preventing overwriting existing resources or ensuring 
unique writes.
+    pub if_none_match: Option<String>,
+
+    /// Sets if_not_exists condition of op.
+    ///
+    /// This operation provides a way to ensure write operations only create 
new resources
+    /// without overwriting existing ones, useful for implementing "create if 
not exists" logic.
+    pub if_not_exists: Option<bool>,
+
+    /// Sets concurrent of op.
+    ///
+    /// - By default, OpenDAL writes files sequentially
+    /// - When concurrent is set:
+    ///   - Multiple write operations can execute in parallel
+    ///   - Write operations return immediately without waiting if tasks space 
are available
+    ///   - Close operation ensures all writes complete in order
+    ///   - Memory usage increases with concurrency level
+    pub concurrent: Option<u32>,
+}
+
+impl From<WriteOptions> for opendal::options::WriteOptions {
+    fn from(value: WriteOptions) -> Self {
+        Self {
+            append: value.append.unwrap_or_default(),
+            chunk: value.chunk.map(|v| v.get_u64().1 as usize),
+            content_type: value.content_type,
+            content_disposition: value.content_disposition,
+            cache_control: value.cache_control,
+            content_encoding: value.content_encoding,
+            user_metadata: value.user_metadata,
+            if_match: value.if_match,
+            if_none_match: value.if_none_match,
+            if_not_exists: value.if_not_exists.unwrap_or_default(),
+            concurrent: value.concurrent.unwrap_or_default() as usize,
+        }
+    }
+}
+
 #[napi(object)]
 #[derive(Default)]
 pub struct DeleteOptions {
diff --git a/bindings/nodejs/tests/suites/asyncWriteOptions.suite.mjs 
b/bindings/nodejs/tests/suites/asyncWriteOptions.suite.mjs
new file mode 100644
index 000000000..730e536cc
--- /dev/null
+++ b/bindings/nodejs/tests/suites/asyncWriteOptions.suite.mjs
@@ -0,0 +1,215 @@
+/*
+ * 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.
+ */
+
+import { randomUUID } from 'node:crypto'
+import { test, describe, expect, assert } from 'vitest'
+
+import { EntryMode, Metadata } from '../../index.mjs'
+import { generateBytes, generateFixedBytes } from '../utils.mjs'
+
+/**
+ * @param {import("../../index").Operator} op
+ */
+export function run(op) {
+  const capability = op.capability()
+
+  describe.runIf(capability.read && capability.write && 
capability.stat)('async writeOptions test', () => {
+    test.runIf(capability.writeCanMulti)('write with concurrent', async () => {
+      const size = 3 * 1024 * 1024
+      const filename = `random_file_${randomUUID()}`
+      const content = generateFixedBytes(size)
+
+      await op.write(filename, content, {
+        concurrent: 2,
+        chunk: BigInt(1024 * 1024),
+      })
+
+      const bs = await op.read(filename)
+      assert.equal(Buffer.compare(bs, content), 0)
+
+      await op.delete(filename)
+    })
+
+    test.runIf(capability.writeWithIfNotExists)('write with if not exists', 
async () => {
+      const size = 3 * 1024 * 1024
+      const filename = `random_file_${randomUUID()}`
+      const content = generateFixedBytes(size)
+
+      const meta = await op.write(filename, content, {
+        ifNotExists: true,
+      })
+      assert.instanceOf(meta, Metadata)
+
+      await expect(op.write(filename, content, { ifNotExists: true 
})).rejects.toThrowError('ConditionNotMatch')
+
+      await op.delete(filename)
+    })
+
+    test.runIf(capability.writeWithCacheControl)('write with cache control', 
async () => {
+      const filename = `random_file_${randomUUID()}`
+      const content = generateBytes()
+
+      const target_cache_control = 'no-cache, no-store, max-age=300'
+      await op.write(filename, content, {
+        cacheControl: target_cache_control,
+      })
+
+      const meta = await op.stat(filename)
+      expect(meta.mode).toBe(EntryMode.FILE)
+      expect(meta.cacheControl).toBe(target_cache_control)
+
+      await op.delete(filename)
+    })
+
+    test.runIf(capability.writeWithContentType)('write with content type', 
async () => {
+      const filename = `random_file_${randomUUID()}`
+      const content = generateBytes()
+
+      const target_content_type = 'application/json'
+      await op.write(filename, content, {
+        contentType: target_content_type,
+      })
+
+      const meta = await op.stat(filename)
+      expect(meta.mode).toBe(EntryMode.FILE)
+      expect(meta.contentLength).toBe(BigInt(content.length))
+
+      await op.delete(filename)
+    })
+
+    test.runIf(capability.writeWithContentDisposition)('write with content 
disposition', async () => {
+      const filename = `random_file_${randomUUID()}`
+      const content = generateBytes()
+
+      const target_content_disposition = 'attachment; filename="filename.jpg"'
+      await op.write(filename, content, {
+        contentDisposition: target_content_disposition,
+      })
+
+      const meta = await op.stat(filename)
+      expect(meta.mode).toBe(EntryMode.FILE)
+      expect(meta.contentDisposition).toBe(target_content_disposition)
+      expect(meta.contentLength).toBe(BigInt(content.length))
+
+      await op.delete(filename)
+    })
+
+    test.runIf(capability.writeWithContentEncoding)('write with content 
encoding', async () => {
+      const filename = `random_file_${randomUUID()}`
+      const content = generateBytes()
+
+      const target_content_encoding = 'gzip'
+      await op.write(filename, content, {
+        contentEncoding: target_content_encoding,
+      })
+
+      const meta = await op.stat(filename)
+      expect(meta.mode).toBe(EntryMode.FILE)
+      expect(meta.contentEncoding).toBe(target_content_encoding)
+
+      await op.delete(filename)
+    })
+
+    test.runIf(capability.writeWithUserMetadata)('write with user metadata', 
async () => {
+      const filename = `random_file_${randomUUID()}`
+      const content = generateBytes()
+
+      const target_user_metadata = {
+        location: 'everywhere',
+      }
+      await op.write(filename, content, {
+        userMetadata: target_user_metadata,
+      })
+
+      const meta = await op.stat(filename)
+      expect(meta.mode).toBe(EntryMode.FILE)
+      expect(meta.userMetadata).toStrictEqual(target_user_metadata)
+
+      await op.delete(filename)
+    })
+
+    test.runIf(capability.writeWithIfMatch)('write with if match', async () => 
{
+      const filenameA = `random_file_${randomUUID()}`
+      const filenameB = `random_file_${randomUUID()}`
+      const contentA = generateBytes()
+      const contentB = generateBytes()
+
+      await op.write(filenameA, contentA)
+      await op.write(filenameB, contentB)
+
+      const metaA = await op.stat(filenameA)
+      const etagA = metaA.etag
+      const metaB = await op.stat(filenameB)
+      const etagB = metaB.etag
+
+      const meta = await op.write(filenameA, contentA, { ifMatch: etagA })
+      assert.instanceOf(meta, Metadata)
+
+      await expect(op.write(filenameA, contentA, { ifMatch: etagB 
})).rejects.toThrowError('ConditionNotMatch')
+
+      await op.delete(filenameA)
+      await op.delete(filenameB)
+    })
+
+    test.runIf(capability.writeWithIfNoneMatch)('write with if none match', 
async () => {
+      const filename = `random_file_${randomUUID()}`
+      const content = generateBytes()
+
+      await op.write(filename, content)
+      const meta = await op.stat(filename)
+
+      await expect(op.write(filename, content, { ifNoneMatch: meta.etag 
})).rejects.toThrowError('ConditionNotMatch')
+
+      await op.delete(filename)
+    })
+
+    test.runIf(capability.writeCanAppend)('write with append', async () => {
+      const filename = `random_file_${randomUUID()}`
+      const contentOne = generateBytes()
+      const contentTwo = generateBytes()
+
+      await op.write(filename, contentOne, { append: true })
+      const meta = await op.stat(filename)
+
+      expect(meta.contentLength).toBe(BigInt(contentOne.length))
+
+      await op.write(filename, contentTwo, { append: true })
+
+      const ds = await op.read(filename)
+      expect(contentOne.length + contentTwo.length).toBe(ds.length)
+      expect(contentOne.length).toEqual(ds.subarray(0, 
contentOne.length).length)
+      expect(contentTwo.length).toEqual(ds.subarray(contentOne.length).length)
+
+      await op.delete(filename)
+    })
+
+    test.runIf(capability.writeCanAppend)('write with append returns 
metadata', async () => {
+      const filename = `random_file_${randomUUID()}`
+      const contentOne = generateBytes()
+      const contentTwo = generateBytes()
+
+      await op.write(filename, contentOne, { append: true })
+      const meta = await op.write(filename, contentTwo, { append: true })
+      const statMeta = await op.stat(filename)
+      expect(meta).toStrictEqual(statMeta)
+
+      await op.delete(filename)
+    })
+  })
+}
diff --git a/bindings/nodejs/tests/suites/index.mjs 
b/bindings/nodejs/tests/suites/index.mjs
index 258a749ae..f7848c2dd 100644
--- a/bindings/nodejs/tests/suites/index.mjs
+++ b/bindings/nodejs/tests/suites/index.mjs
@@ -32,6 +32,8 @@ import { run as AsyncListOptionsTestRun } from 
'./asyncListOptions.suite.mjs'
 import { run as SyncListOptionsTestRun } from './syncListOptions.suite.mjs'
 import { run as AsyncDeleteOptionsTestRun } from 
'./asyncDeleteOptions.suite.mjs'
 import { run as SyncDeleteOptionsTestRun } from './syncDeleteOptions.suite.mjs'
+import { run as AsyncWriteOptionsTestRun } from './asyncWriteOptions.suite.mjs'
+import { run as SyncWriteOptionsTestRun } from './syncWriteOptions.suite.mjs'
 
 export function runner(testName, scheme) {
   if (!scheme) {
@@ -69,5 +71,7 @@ export function runner(testName, scheme) {
     SyncListOptionsTestRun(operator)
     AsyncDeleteOptionsTestRun(operator)
     SyncDeleteOptionsTestRun(operator)
+    AsyncWriteOptionsTestRun(operator)
+    SyncWriteOptionsTestRun(operator)
   })
 }
diff --git a/bindings/nodejs/tests/suites/syncWriteOptions.suite.mjs 
b/bindings/nodejs/tests/suites/syncWriteOptions.suite.mjs
new file mode 100644
index 000000000..f777ac55a
--- /dev/null
+++ b/bindings/nodejs/tests/suites/syncWriteOptions.suite.mjs
@@ -0,0 +1,215 @@
+/*
+ * 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.
+ */
+
+import { randomUUID } from 'node:crypto'
+import { test, describe, expect, assert } from 'vitest'
+
+import { EntryMode, Metadata } from '../../index.mjs'
+import { generateBytes, generateFixedBytes } from '../utils.mjs'
+
+/**
+ * @param {import("../../index").Operator} op
+ */
+export function run(op) {
+  const capability = op.capability()
+
+  describe.runIf(capability.read && capability.write && capability.stat)('sync 
writeOptions test', () => {
+    test.runIf(capability.writeCanMulti)('write with concurrent', () => {
+      const size = 3 * 1024 * 1024
+      const filename = `random_file_${randomUUID()}`
+      const content = generateFixedBytes(size)
+
+      op.writeSync(filename, content, {
+        concurrent: 2,
+        chunk: BigInt(1024 * 1024),
+      })
+
+      const bs = op.readSync(filename)
+      assert.equal(Buffer.compare(bs, content), 0)
+
+      op.deleteSync(filename)
+    })
+
+    test.runIf(capability.writeWithIfNotExists)('write with if not exists', () 
=> {
+      const size = 3 * 1024 * 1024
+      const filename = `random_file_${randomUUID()}`
+      const content = generateFixedBytes(size)
+
+      const meta = op.writeSync(filename, content, {
+        ifNotExists: true,
+      })
+      assert.instanceOf(meta, Metadata)
+
+      expect(() => op.writeSync(filename, content, { ifNotExists: true 
})).toThrowError('ConditionNotMatch')
+
+      op.deleteSync(filename)
+    })
+
+    test.runIf(capability.writeWithCacheControl)('write with cache control', 
() => {
+      const filename = `random_file_${randomUUID()}`
+      const content = generateBytes()
+
+      const target_cache_control = 'no-cache, no-store, max-age=300'
+      op.writeSync(filename, content, {
+        cacheControl: target_cache_control,
+      })
+
+      const meta = op.statSync(filename)
+      expect(meta.mode).toBe(EntryMode.FILE)
+      expect(meta.cacheControl).toBe(target_cache_control)
+
+      op.deleteSync(filename)
+    })
+
+    test.runIf(capability.writeWithContentType)('write with content type', () 
=> {
+      const filename = `random_file_${randomUUID()}`
+      const content = generateBytes()
+
+      const target_content_type = 'application/json'
+      op.writeSync(filename, content, {
+        contentType: target_content_type,
+      })
+
+      const meta = op.statSync(filename)
+      expect(meta.mode).toBe(EntryMode.FILE)
+      expect(meta.contentLength).toBe(BigInt(content.length))
+
+      op.deleteSync(filename)
+    })
+
+    test.runIf(capability.writeWithContentDisposition)('write with content 
disposition', () => {
+      const filename = `random_file_${randomUUID()}`
+      const content = generateBytes()
+
+      const target_content_disposition = 'attachment; filename="filename.jpg"'
+      op.writeSync(filename, content, {
+        contentDisposition: target_content_disposition,
+      })
+
+      const meta = op.statSync(filename)
+      expect(meta.mode).toBe(EntryMode.FILE)
+      expect(meta.contentDisposition).toBe(target_content_disposition)
+      expect(meta.contentLength).toBe(BigInt(content.length))
+
+      op.deleteSync(filename)
+    })
+
+    test.runIf(capability.writeWithContentEncoding)('write with content 
encoding', () => {
+      const filename = `random_file_${randomUUID()}`
+      const content = generateBytes()
+
+      const target_content_encoding = 'gzip'
+      op.writeSync(filename, content, {
+        contentEncoding: target_content_encoding,
+      })
+
+      const meta = op.statSync(filename)
+      expect(meta.mode).toBe(EntryMode.FILE)
+      expect(meta.contentEncoding).toBe(target_content_encoding)
+
+      op.deleteSync(filename)
+    })
+
+    test.runIf(capability.writeWithUserMetadata)('write with user metadata', 
() => {
+      const filename = `random_file_${randomUUID()}`
+      const content = generateBytes()
+
+      const target_user_metadata = {
+        location: 'everywhere',
+      }
+      op.writeSync(filename, content, {
+        userMetadata: target_user_metadata,
+      })
+
+      const meta = op.statSync(filename)
+      expect(meta.mode).toBe(EntryMode.FILE)
+      expect(meta.userMetadata).toStrictEqual(target_user_metadata)
+
+      op.deleteSync(filename)
+    })
+
+    test.runIf(capability.writeWithIfMatch)('write with if match', () => {
+      const filenameA = `random_file_${randomUUID()}`
+      const filenameB = `random_file_${randomUUID()}`
+      const contentA = generateBytes()
+      const contentB = generateBytes()
+
+      op.writeSync(filenameA, contentA)
+      op.writeSync(filenameB, contentB)
+
+      const metaA = op.statSync(filenameA)
+      const etagA = metaA.etag
+      const metaB = op.statSync(filenameB)
+      const etagB = metaB.etag
+
+      const meta = op.writeSync(filenameA, contentA, { ifMatch: etagA })
+      assert.instanceOf(meta, Metadata)
+
+      expect(() => op.writeSync(filenameA, contentA, { ifMatch: etagB 
})).toThrowError('ConditionNotMatch')
+
+      op.deleteSync(filenameA)
+      op.deleteSync(filenameB)
+    })
+
+    test.runIf(capability.writeWithIfNoneMatch)('write with if none match', () 
=> {
+      const filename = `random_file_${randomUUID()}`
+      const content = generateBytes()
+
+      op.writeSync(filename, content)
+      const meta = op.statSync(filename)
+
+      expect(() => op.writeSync(filename, content, { ifNoneMatch: meta.etag 
})).toThrowError('ConditionNotMatch')
+
+      op.deleteSync(filename)
+    })
+
+    test.runIf(capability.writeCanAppend)('write with append', () => {
+      const filename = `random_file_${randomUUID()}`
+      const contentOne = generateBytes()
+      const contentTwo = generateBytes()
+
+      op.writeSync(filename, contentOne, { append: true })
+      const meta = op.statSync(filename)
+
+      expect(meta.contentLength).toBe(BigInt(contentOne.length))
+
+      op.writeSync(filename, contentTwo, { append: true })
+
+      const ds = op.readSync(filename)
+      expect(contentOne.length + contentTwo.length).toBe(ds.length)
+      expect(contentOne.length).toEqual(ds.subarray(0, 
contentOne.length).length)
+      expect(contentTwo.length).toEqual(ds.subarray(contentOne.length).length)
+
+      op.deleteSync(filename)
+    })
+
+    test.runIf(capability.writeCanAppend)('write with append returns 
metadata', () => {
+      const filename = `random_file_${randomUUID()}`
+      const contentOne = generateBytes()
+      const contentTwo = generateBytes()
+
+      op.writeSync(filename, contentOne, { append: true })
+      const meta = op.writeSync(filename, contentTwo, { append: true })
+      const statMeta = op.statSync(filename)
+      expect(meta).toStrictEqual(statMeta)
+
+      op.deleteSync(filename)
+    })
+  })
+}


Reply via email to