Fokko commented on code in PR #349:
URL: https://github.com/apache/iceberg-rust/pull/349#discussion_r1847225254


##########
crates/e2e_test/tests/conflict_commit_test.rs:
##########
@@ -0,0 +1,196 @@
+// 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.
+
+//! Integration tests for rest catalog.
+
+use std::collections::HashMap;
+use std::sync::Arc;
+
+use arrow_array::{ArrayRef, BooleanArray, Int32Array, RecordBatch, 
StringArray};
+use futures::TryStreamExt;
+use iceberg::io::{S3_ACCESS_KEY_ID, S3_ENDPOINT, S3_REGION, 
S3_SECRET_ACCESS_KEY};
+use iceberg::spec::{NestedField, PrimitiveType, Schema, Type};
+use iceberg::transaction::Transaction;
+use iceberg::writer::base_writer::data_file_writer::{DataFileWriterBuilder, 
DataFileWriterConfig};
+use iceberg::writer::file_writer::location_generator::{
+    DefaultFileNameGenerator, DefaultLocationGenerator,
+};
+use iceberg::writer::file_writer::ParquetWriterBuilder;
+use iceberg::writer::{IcebergWriter, IcebergWriterBuilder};
+use iceberg::{Catalog, Namespace, NamespaceIdent, TableCreation};
+use iceberg_catalog_rest::{RestCatalog, RestCatalogConfig};
+use iceberg_test_utils::docker::DockerCompose;
+use iceberg_test_utils::{normalize_test_name, set_up};
+use parquet::file::properties::WriterProperties;
+use port_scanner::scan_port_addr;
+use tokio::time::sleep;
+
+const REST_CATALOG_PORT: u16 = 8181;
+
+struct TestFixture {
+    _docker_compose: DockerCompose,
+    rest_catalog: RestCatalog,
+}
+
+async fn set_test_fixture(func: &str) -> TestFixture {
+    set_up();
+    let docker_compose = DockerCompose::new(
+        normalize_test_name(format!("{}_{func}", module_path!())),
+        format!("{}/testdata", env!("CARGO_MANIFEST_DIR")),
+    );
+
+    // Start docker compose
+    docker_compose.run();
+
+    let rest_catalog_ip = docker_compose.get_container_ip("rest");
+
+    let read_port = format!("{}:{}", rest_catalog_ip, REST_CATALOG_PORT);
+    loop {
+        if !scan_port_addr(&read_port) {
+            log::info!("Waiting for 1s rest catalog to ready...");
+            sleep(std::time::Duration::from_millis(1000)).await;
+        } else {
+            break;
+        }
+    }
+
+    let container_ip = docker_compose.get_container_ip("minio");
+    let read_port = format!("{}:{}", container_ip, 9000);
+
+    let config = RestCatalogConfig::builder()
+        .uri(format!("http://{}:{}";, rest_catalog_ip, REST_CATALOG_PORT))
+        .props(HashMap::from([
+            (S3_ENDPOINT.to_string(), format!("http://{}";, read_port)),
+            (S3_ACCESS_KEY_ID.to_string(), "admin".to_string()),
+            (S3_SECRET_ACCESS_KEY.to_string(), "password".to_string()),
+            (S3_REGION.to_string(), "us-east-1".to_string()),
+        ]))
+        .build();
+    let rest_catalog = RestCatalog::new(config);
+
+    TestFixture {
+        _docker_compose: docker_compose,
+        rest_catalog,
+    }
+}
+
+#[tokio::test]
+async fn test_append_data_file_conflict() {
+    let fixture = set_test_fixture("test_create_table").await;
+
+    let ns = Namespace::with_properties(
+        NamespaceIdent::from_strs(["apple", "ios"]).unwrap(),
+        HashMap::from([
+            ("owner".to_string(), "ray".to_string()),
+            ("community".to_string(), "apache".to_string()),
+        ]),
+    );
+
+    fixture
+        .rest_catalog
+        .create_namespace(ns.name(), ns.properties().clone())
+        .await
+        .unwrap();
+
+    let schema = Schema::builder()
+        .with_schema_id(1)
+        .with_identifier_field_ids(vec![2])
+        .with_fields(vec![
+            NestedField::optional(1, "foo", 
Type::Primitive(PrimitiveType::String)).into(),
+            NestedField::required(2, "bar", 
Type::Primitive(PrimitiveType::Int)).into(),
+            NestedField::optional(3, "baz", 
Type::Primitive(PrimitiveType::Boolean)).into(),
+        ])
+        .build()
+        .unwrap();
+
+    let table_creation = TableCreation::builder()
+        .name("t1".to_string())
+        .schema(schema.clone())
+        .build();
+
+    let table = fixture
+        .rest_catalog
+        .create_table(ns.name(), table_creation)
+        .await
+        .unwrap();
+
+    // Create the writer and write the data
+    let schema: Arc<arrow_schema::Schema> = Arc::new(
+        table
+            .metadata()
+            .current_schema()
+            .as_ref()
+            .try_into()
+            .unwrap(),
+    );
+    let location_generator = 
DefaultLocationGenerator::new(table.metadata().clone()).unwrap();
+    let file_name_generator = DefaultFileNameGenerator::new(
+        "test".to_string(),
+        None,
+        iceberg::spec::DataFileFormat::Parquet,
+    );
+    let parquet_writer_builder = ParquetWriterBuilder::new(
+        WriterProperties::default(),
+        table.metadata().current_schema().clone(),
+        table.file_io().clone(),
+        location_generator.clone(),
+        file_name_generator.clone(),
+    );
+    let data_file_writer_builder = 
DataFileWriterBuilder::new(parquet_writer_builder);
+    let mut data_file_writer = data_file_writer_builder
+        .build(DataFileWriterConfig::new(None))
+        .await
+        .unwrap();
+    let col1 = StringArray::from(vec![Some("foo"), Some("bar"), None, 
Some("baz")]);
+    let col2 = Int32Array::from(vec![Some(1), Some(2), Some(3), Some(4)]);
+    let col3 = BooleanArray::from(vec![Some(true), Some(false), None, 
Some(false)]);
+    let batch = RecordBatch::try_new(schema.clone(), vec![
+        Arc::new(col1) as ArrayRef,
+        Arc::new(col2) as ArrayRef,
+        Arc::new(col3) as ArrayRef,
+    ])
+    .unwrap();
+    data_file_writer.write(batch.clone()).await.unwrap();
+    let data_file = data_file_writer.close().await.unwrap();
+
+    // start two transaction and commit one of them
+    let tx1 = Transaction::new(&table);
+    let mut append_action = tx1.fast_append(None, vec![]).unwrap();
+    append_action.add_data_files(data_file.clone()).unwrap();
+    let tx1 = append_action.apply().await.unwrap();
+    let tx2 = Transaction::new(&table);
+    let mut append_action = tx2.fast_append(None, vec![]).unwrap();
+    append_action.add_data_files(data_file.clone()).unwrap();
+    let tx2 = append_action.apply().await.unwrap();
+    let table = tx2.commit(&fixture.rest_catalog).await.unwrap();

Review Comment:
   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.

To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org

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


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

Reply via email to