Fix tow bugs:
1. Enabling file caching requires both `FE session` and `BE` configurations(enable_file_cache=true) to be enabled.
2. `ParquetReader` has not used `IOContext` previously, but `CachedRemoteFileReader::read_at` needs `IOContext` after PR(#17586).
Co-authored-by: ByteYue <[yj976240184@gmail.com](mailto:yj976240184@gmail.com)>
This PR is an optimization for https://github.com/apache/doris/pull/17478:
1. Change the buffer size of `LineReader` to 4MB to align with the size of prefetch buffer.
2. Lazily prefetch data in the first read to prevent wasted reading.
3. S3 block size is 32MB only, which is too small for a file split. Set 128MB as default file split size.
4. Add `_end_offset` for prefetch buffer to prevent wasted reading.
The query performance of reading data on object storage is improved by more than 3x+.
1. Fix value idx in bool rle decoder
2. Iceberg table support datetimev2(3). In the previous version, we converted hive timestamp to datetimev2(0) default.
1. Introduce hadoop libhdfs
2. For Linux-X86 platform, use the hadoop libhdfs
3. For other platform, use libhdfs3, because currently we don't have hadoop libhdfs binary for other platform
Co-authored-by: adonis0147 <adonis0147@gmail.com>
when loading big file with multi bytes line delimiter, some line record maybe incomplete because of _output_buf_limit, so this incomplete data will move to the beginning of the output buf and read more data into output buf. In this case, find line delimiter should start with no offset to avoid a bug that spilt two lines as one line.
PR(#17330) has changed the column type of kay and value from array to normal column, but orc&parquet reader still cast to array column, resulting in cast error.
Problem:
1. FE will split the parquet file into split. So a file can have several splits.
2. BE will scan each split, read the footer of the parquet file.
3. If 2 splits belongs to a same parquet file, the footer of this file will be read twice.
This PR mainly changes:
1. Use kv cache to cache the footer of parquet file.
2. The kv cache is belong to a scan node, so all parquet reader belong to this scan node will share same kv cache.
3. In cache, the key is "meta_file_path", the value is parsed thrift footer.
The KV Cache is sharded into mutlti sub cache.
So that different file can use different sub cache, avoid blocking each other
In my test, a query with 26 splits can reduce the footer parse time from 4s -> 1s
See #17764 for details
I have tested:
- Unit test for local/s3/hdfs/broker file system: be/test/io/fs/file_system_test.cpp
- Outfile to local/s3/hdfs/broker.
- Load from local/s3/hdfs/broker.
- Query file on local/s3/hdfs/broker file system, with table value function and catalog.
- Backup/Restore with local/s3/hdfs/broker file system
Not test:
- cold & host data separation case.
Before this PR when encountering null values with some columns which is specified as `NOT NULL`, null values will not be filtered,thi behavior does not match with the original load behavior.
Second column alignment logic has bug :
```
template <typename ColumnInserterFn>
void align_variant_by_name_and_type(ColumnObject& dst, const ColumnObject& src, size_t row_cnt,
ColumnInserterFn inserter) {
CHECK(dst.is_finalized() && src.is_finalized());
// Use rows() here instead of size(), since size() will check_consistency
// but we could not check_consistency since num_rows will be upgraded even
// if src and dst is empty, we just increase the num_rows of dst and fill
// num_rows of default values when meet new data
size_t num_rows = dst.rows();
```
1. introduce a new type `VARIANT` to encapsulate dynamic generated columns for hidding the detail of types and names of newly generated columns
2. introduce a new expression `SchemaChangeExpr` for doing schema change for extensibility
Support delta encoding and rle(bool) to read Glue data
add delta bit pack decoder,
add delta length byte array decoder,
add delta byte array decoder.
add rle bool decoder.
We find some data type is read with delta encoding on AWS Glue, so it should be supported.
The definition of delta encoding can refer to the delta encoding in parquet.
Fix three bugs:
1. `repeated_parent_def_level ` should be the definition of its repeated parent.
2. Failed to parse schema like `decimal(p, s)`
3. Fill wrong offsets for array type
remove duplicate type definition in function context
remove unused method in function context
not need stale state in vexpr context because vexpr is stateless and function context saves state and they are cloned.
remove useless slot_size in all tuple or slot descriptor.
remove doris_udf namespace, it is useless.
remove some unused macro definitions.
init v_conjuncts in vscanner, not need write the same code in every scanner.
using unique ptr to manage function context since it could only belong to a single expr context.
Issue Number: close #xxx
---------
Co-authored-by: yiguolei <yiguolei@gmail.com>
fix heap-use-after-free
The OrcReader has a internal FileInputStream, If the file is empty, the memory of FileInputStream will leak.
Besides, there is a Statistics instance in FileInputStream. FileInputStream maybe delete if the orc reader
is inited failed, but Statistics maybe used when orc reader is closed, causing heap-use-after-free error.
Potential memory leak
When init file scanner in file scan node, the file scanner prepare failed, the memory of file scanner will leak.
* [Optimize](simd json reader) Cached search results for previous row (keyed as index in JSON object) - used as a hint.
`_simdjson_set_column_value` could become a hot spot while parsing json in simdjson mode,
introduce `_prev_positions` to cache results for previous row (keyed as index in JSON object) due to the json name field order,
should be quite the same between each lines
* fix case
A const reference member variables as class member stores a temporary object, which cannot be got after the temporary object being destroyed, cause be core dump while enable debug level log
_broker_addr has been destroyed in BrokerFileReader
1. Enhencement:
For single-charset column separator,csv_reader use another method of `split value`.
2. BugFix
Set `json` file format loading to be sensitive.
Support parsing map&struct type in parquet&orc reader.
## Remaining Problems
1. Doris use array type to build the key and value column of a `map`, but doesn't fill the offsets in value column, so the offsets in value column is wasted.
2. Parquet support reading only key or value column in `map`, this PR hasn't supported yet.
3. Parquet support reading partial columns in `struct`, this PR hasn't supported yet.
Orc doesn't fill null values in new batch, but the former batch has been release.
Other types like int/long/timestamp... are flat types without pointer in them,
so other types do not need to be handled separately like string.
Support to decode nested array column in parquet reader:
1. FE should generate the right nested column type. FE doesn't check the nesting depth and legality, like map\<array\<int\>, int\>.
2. `ParquetColumnReader` has removed the filtering of page index to support nested array type.
It's too difficult to skip values in nested complex types. Maybe we should support the filtering of page index and lazy read in later PR.
3. `ExternalFileScanNode` has a bug in creating default value expression.
4. Maybe it's slow to read repetition levels in a while loop. I'll optimize this in next PR.
5. Array column has temporary `SchemaElement` in its thrift definition,
we have removed them and keep its parent in former implementation.
The remaining parent should inherit the repetition and definition level of its child.
Improve performance of parquet reader filter calculation.
- Use `filter_data` instead of `(*filter_ptr)` to merge filter to improve performance.
- Use mutable column filter func instead of original new column filter func which introduced by #16850.
- Avoid column ref-count increasing which caused unnecessary copying by passing column pointer ref.
Issue Number: close#17003
## Problem summary
The linker couldn't find some symbols because the implementation of a template member function doris::vectorized::Decoder::init_decimal_converter is missing in the header file in which the corresponding declaration is placed.
In previous implementation, when querying tvf, FE will get schema from BE.
And BE will try to open the first file to get its schema info, but for orc or parquet format,
if the file is empty, it will return error.
But even for an empty file, we can still get schema info from file's footer.
So we should handle the empty file to get schema info correctly.
Also modify the catalog doc to add some FAQ.
Set column names from path to lower case in case-insensitive case.
This is for Iceberg columns from path. Iceberg columns are case sensitive,
which may cause error for table with partitions.
To support schema evolution, Iceberg add schema information to Parquet file metadata.
But for early iceberg version, it doesn't write any schema information to Parquet file.
This PR is to support read parquet without schema information.
Hive 1.x may write orc file with internal column name (_col0, _col1, _col2...).
This will cause query result be NULL because column name in orc file doesn't match
with column name in Doris table schema. This pr is to support query Hive orc files with internal column names.
For now, we haven't see any problem in Parquet file, will send new pr to fix parquet if any problem show up in the future.
Issue Number: close#16351
Dynamic schema table is a special type of table, it's schema change with loading procedure.Now we implemented this feature mainly for semi-structure data such as JSON, since JSON is schema self-described we could extract schema info from the original documents and inference the final type infomation.This speical table could reduce manual schema change operation and easily import semi-structure data and extends it's schema automatically.