1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
// 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.
//! Apache Iceberg Official Native Rust Implementation
//!
//! # Examples
//!
//! ## Scan A Table
//!
//! ```rust, no_run
//! use std::collections::HashMap;
//!
//! use futures::TryStreamExt;
//! use iceberg::io::{FileIO, FileIOBuilder};
//! use iceberg::memory::MemoryCatalogBuilder;
//! use iceberg::{Catalog, CatalogBuilder, MemoryCatalog, Result, TableIdent};
//!
//! #[tokio::main]
//! async fn main() -> Result<()> {
//! // Connect to a catalog.
//! use iceberg::memory::MEMORY_CATALOG_WAREHOUSE;
//! let catalog = MemoryCatalogBuilder::default()
//! .load(
//! "memory",
//! HashMap::from([(
//! MEMORY_CATALOG_WAREHOUSE.to_string(),
//! "file:///path/to/warehouse".to_string(),
//! )]),
//! )
//! .await?;
//! // Load table from catalog.
//! let table = catalog
//! .load_table(&TableIdent::from_strs(["hello", "world"])?)
//! .await?;
//! // Build table scan.
//! let stream = table
//! .scan()
//! .select(["name", "id"])
//! .build()?
//! .to_arrow()
//! .await?;
//!
//! // Consume this stream like arrow record batch stream.
//! let _data: Vec<_> = stream.try_collect().await?;
//! Ok(())
//! }
//! ```
extern crate derive_builder;
extern crate core;
pub use ;
pub use *;
pub