|
| 1 | +// Copyright 2022 Datafuse Labs. |
| 2 | +// |
| 3 | +// Licensed under the Apache License, Version 2.0 (the "License"); |
| 4 | +// you may not use this file except in compliance with the License. |
| 5 | +// You may obtain a copy of the License at |
| 6 | +// |
| 7 | +// http://www.apache.org/licenses/LICENSE-2.0 |
| 8 | +// |
| 9 | +// Unless required by applicable law or agreed to in writing, software |
| 10 | +// distributed under the License is distributed on an "AS IS" BASIS, |
| 11 | +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 12 | +// See the License for the specific language governing permissions and |
| 13 | +// limitations under the License. |
| 14 | + |
| 15 | +use std::sync::Arc; |
| 16 | + |
| 17 | +use common_datablocks::DataBlock; |
| 18 | +use common_datavalues::prelude::*; |
| 19 | +use common_exception::Result; |
| 20 | +use futures_util::TryStreamExt; |
| 21 | + |
| 22 | +use crate::io::MetaReaders; |
| 23 | +use crate::io::SnapshotHistoryReader; |
| 24 | +use crate::sessions::TableContext; |
| 25 | +use crate::FuseTable; |
| 26 | + |
| 27 | +pub struct FuseBlock<'a> { |
| 28 | + pub ctx: Arc<dyn TableContext>, |
| 29 | + pub table: &'a FuseTable, |
| 30 | + pub snapshot_id: String, |
| 31 | +} |
| 32 | + |
| 33 | +impl<'a> FuseBlock<'a> { |
| 34 | + pub fn new(ctx: Arc<dyn TableContext>, table: &'a FuseTable, snapshot_id: String) -> Self { |
| 35 | + Self { |
| 36 | + ctx, |
| 37 | + table, |
| 38 | + snapshot_id, |
| 39 | + } |
| 40 | + } |
| 41 | + |
| 42 | + pub async fn get_blocks(&self) -> Result<DataBlock> { |
| 43 | + let tbl = self.table; |
| 44 | + let maybe_snapshot = tbl.read_table_snapshot(self.ctx.clone()).await?; |
| 45 | + if let Some(snapshot) = maybe_snapshot { |
| 46 | + // prepare the stream of snapshot |
| 47 | + let snapshot_version = tbl.snapshot_format_version(); |
| 48 | + let snapshot_location = tbl |
| 49 | + .meta_location_generator |
| 50 | + .snapshot_location_from_uuid(&snapshot.snapshot_id, snapshot_version)?; |
| 51 | + let reader = MetaReaders::table_snapshot_reader(self.ctx.clone()); |
| 52 | + let mut snapshot_stream = reader.snapshot_history( |
| 53 | + snapshot_location, |
| 54 | + snapshot_version, |
| 55 | + tbl.meta_location_generator().clone(), |
| 56 | + ); |
| 57 | + |
| 58 | + // find the element by snapshot_id in stream |
| 59 | + while let Some(snapshot) = snapshot_stream.try_next().await? { |
| 60 | + if snapshot.snapshot_id.simple().to_string() == self.snapshot_id { |
| 61 | + let len = snapshot.summary.block_count as usize; |
| 62 | + let snapshot_id = vec![self.snapshot_id.clone().into_bytes()]; |
| 63 | + let timestamp = |
| 64 | + vec![snapshot.timestamp.map(|dt| (dt.timestamp_micros()) as i64)]; |
| 65 | + let mut block_location: Vec<Vec<u8>> = Vec::with_capacity(len); |
| 66 | + let mut block_size: Vec<u64> = Vec::with_capacity(len); |
| 67 | + let mut bloom_filter_location: Vec<Option<Vec<u8>>> = Vec::with_capacity(len); |
| 68 | + let mut bloom_filter_size: Vec<u64> = Vec::with_capacity(len); |
| 69 | + |
| 70 | + let reader = MetaReaders::segment_info_reader(self.ctx.as_ref()); |
| 71 | + for (x, ver) in &snapshot.segments { |
| 72 | + let segment = reader.read(x, None, *ver).await?; |
| 73 | + segment.blocks.clone().into_iter().for_each(|block| { |
| 74 | + block_location.push(block.location.0.into_bytes()); |
| 75 | + block_size.push(block.block_size); |
| 76 | + bloom_filter_location.push( |
| 77 | + block |
| 78 | + .bloom_filter_index_location |
| 79 | + .map(|(s, _)| s.into_bytes()), |
| 80 | + ); |
| 81 | + bloom_filter_size.push(block.bloom_filter_index_size); |
| 82 | + }); |
| 83 | + } |
| 84 | + |
| 85 | + return Ok(DataBlock::create(FuseBlock::schema(), vec![ |
| 86 | + Arc::new(ConstColumn::new(Series::from_data(snapshot_id), len)), |
| 87 | + Arc::new(ConstColumn::new(Series::from_data(timestamp), len)), |
| 88 | + Series::from_data(block_location), |
| 89 | + Series::from_data(block_size), |
| 90 | + Series::from_data(bloom_filter_location), |
| 91 | + Series::from_data(bloom_filter_size), |
| 92 | + ])); |
| 93 | + } |
| 94 | + } |
| 95 | + } |
| 96 | + |
| 97 | + Ok(DataBlock::empty_with_schema(Self::schema())) |
| 98 | + } |
| 99 | + |
| 100 | + pub fn schema() -> Arc<DataSchema> { |
| 101 | + DataSchemaRefExt::create(vec![ |
| 102 | + DataField::new("snapshot_id", Vu8::to_data_type()), |
| 103 | + DataField::new_nullable("timestamp", TimestampType::new_impl(6)), |
| 104 | + DataField::new("block_location", Vu8::to_data_type()), |
| 105 | + DataField::new("block_size", u64::to_data_type()), |
| 106 | + DataField::new_nullable("bloom_filter_location", Vu8::to_data_type()), |
| 107 | + DataField::new("bloom_filter_size", u64::to_data_type()), |
| 108 | + ]) |
| 109 | + } |
| 110 | +} |
0 commit comments