I've worked quite a bit with tensor data recently and `arrow.Tensor` (or
just the underlying FixedSizeList) has worked well for me for an in-memory
representation.
> If you compress it, you have no means to decompress individual chunks,
from what I can tell from prototyping within Python.
> Can someone confirm it?
I can confirm that the current join implementation will potentially reorder
input. The larger the input the more likely the chance of reordering.
> I think that ordering is only guaranteed if it has been sorted.
Close enough probably. I think there is an implicit
doing with the raw_values, you don't have to care.
> >> Compute functions commonly have two different loops: one that handles
> >> nulls and a faster one (without checks in the loop body) that you can
> >> use when data.GetNullCount()==0.
> >>
> >&g
There is no advantage to using a Datum here. The Datum class is mainly
intended for representing something that might be a Scalar or might be an
Array.
> Also, is there an efficient way to loop through a slice perhaps by
incrementing a pointer?
You will want to cast the Array and avoid Scalar
p to date but some time
> ago I used Azure SDK directly and downloaded small files to memory buffer:
>
> content = container.download_blob(path).readall() # container is
> azure.storage.blob.ContainerClient
> buffer = pa.BufferReader(content)
> pyarrow.parquet.read_table(buffer, col
Those files are quite small. For every single file pyarrow is going to
need to read the metadata, determine which columns to read (column
filtering), determine if any of the rows need to be read (using row
filtering) and then actually issue the read. If you combined all those
files into one file
You are correct that there is no existing capability to create an IPC
reader with precomputed metadata. I don't think anyone would be opposed to
this feature, it just hasn't been a priority.
If you wanted to avoid changing arrow then you could create your own
implementation of `RandomAccessFile`
I'm not entirely sure what kinds of operations you need.
Arrow arrays all (with the exception of RLE) support constant time (O(1))
random access. So generally if you want to keep a pointer to a particular
element or a row of data then that is ok.
On the other hand, you mentioned sorting. One
How many rows match your timestamp criteria? In other words, how many rows
are you applying the function to? If there is an earlier exact match
filter on a timestamp that only matches 1 (or a few rows) then I are you
sure the expression evaluation (and not the filtering) is the costly spot?
>
Arrow is a great in-memory representation to use for this kind of problem.
However, keep in mind that Arrow, the format, does not actually include
compute capabilities. Much of your question is asking about things that
would normally be considered "compute operations" and the capabilities and
Note that there are a few aggregates that cannot be distributed (e.g.
median). Currently, the way aggregates are implemented in Acero, we
distribute across threads, and so we don't actually have an implementation
for these aggregates anyways so probably a bit of a moot point (although
there is
ws and
{my_table.num_columns} columns in {end - start} seconds")
```
On Mon, Jul 3, 2023 at 7:54 AM Weston Pace wrote:
> Those files are a bit smaller than ideal but not small enough that I would
> expect anything like the performance you are getting.
>
> I can't speak for the Java
tta wrote:
> Each file has 1-2MB with 1 row group each, around 2000 rows per file and
> 61 columns - a total of 7697842 rows. Is this performance expected for this
> dataset or is there any suggestion to optimize?
>
> Thanks!
>
> On Sat, Jul 1, 2023 at 10:44 PM Weston Pace wr
What size are the row groups in your parquet files? How many columns and
rows in the files?
On Sat, Jul 1, 2023, 6:08 PM Paulo Motta wrote:
> Hi,
>
> I'm trying to read 4096 parquet files with a total size of 6GB using this
> cookbook:
>
I'm not familiar with the "parquet block size". However, you can use row
groups to accomplish this task. You could write a single 10GB file with 5
row groups. Then, when reading, the arrow readers allow you to specify
which row groups you would like to read.
On Wed, Jun 7, 2023 at 6:13 AM
>
> So my question is : why does it decompresses the data here ? For me it is
> doing a unnecessary copy by transforming a compressed record batch into a
> uncompressed record batch.
> Could not a table (or a record batch) be composed of array with compressed
> buffer ?
>
> Thank you
with memory mapping and
avoiding access to the data they don't need.
On Mon, May 22, 2023 at 12:51 PM Weston Pace wrote:
> I'm a little bit confused on the benchmark. The benchmark is labeled
> "open file" and yet "read_table" will read the entire file into memory.
I'm a little bit confused on the benchmark. The benchmark is labeled "open
file" and yet "read_table" will read the entire file into memory. I don't
think your other benchmarks are doing this (e.g. they are not reading data
into memory).
As for the questions on memory mapping, I have a few
Arrow can also represent dictionary encoding. If you like StringBuilder
then there is also a StringDictionaryBuilder which should be more or less
compatible:
TEST(TestStringDictionaryBuilder, Basic) {
// Build the dictionary Array
StringDictionaryBuilder builder;
Spencer's answer is good. I'll just add one minor clarification. A
MapArray is what you want, but it is a List> and not
just Struct. This allows you to have many rows of metadata
key/pairs for each row of your structured data.
On Thu, May 18, 2023 at 5:41 PM Spencer Nelson wrote:
> You could
> Can anyone guide what's the best practice here and if my
below understandings are correct:
Here's an example:
https://gist.github.com/westonpace/6f7fdbdc0399501418101851d75091c4
> I receive streaming data via a callback function which gives me data row
by row. To my best knowledge, Subclassing
The page that Aldrin linked is possible but it requires that you use the
same toolchain and version as pyarrow. I would probably advise using the C
data API first. By using the C data API you don't have to couple yourself
so tightly with the pyarrow build. For example, your C++ extension can
reciated.
>
> Thanks,
> Surya
>
> On Sat, Apr 22, 2023 at 2:27 AM Weston Pace wrote:
>
>> No, there's no build-time configuration settings to enable TBB
>> specifically.
>>
>> You can, at runtime, specify a custom executor to use for most
>> oper
No, there's no build-time configuration settings to enable TBB specifically.
You can, at runtime, specify a custom executor to use for most operations.
We use one thread pool for CPU tasks and one for I/O tasks. You could
replace either or both with a TBB-based executor.
For example, the method
It's unlikely there will be much benefit in using dictionary encoding. How
are you doing the filtering? It sounds like you might have many filters.
`arrow::compute::Filter` will not get great performance if you want to
apply many filters. No one has added support for selection vectors but
that
Over a series of PRs (with [1] being the overall PR with most of the
discussion) we have done some internal refactoring of the streaming
execution engine (Acero) to put it into its own namespace and library.
This was a significant move of files and there are two breaking changes
(the affected code
I suspect this is a combination of [1] and [2]/[3].
We do not currently allow you to specify a filter during discovery. We
could, and that should allow us to reduce the amount of reading we need to
do.
Also, when no filter is supplied, we can be more efficient with our usage
of S3.
[1]
No. There is no environment variable for setting the endpoint URL.
In addition to the FileSystemFromUri method you can also construct the
filesystem directly:
```
arrow::fs::S3Options options = arrow::fs::S3Options::Defaults();
options.endpoint_override = "localhost:9000";
Result filesystem =
Thow following headers are supported:
ACL*
Cache-Control
Content-Type
Content-Language
Expires
*ACL must be one of the "canned ACL" strings[1]
> In the docs of open_append_stream() it reads 'Unsupported metadata keys
will be ignored.'
> What does 'unsupported' mean here - unsupported by
kimmo.li...@gmail.com
> +358 40 590 1074
>
>
>
> On 8. Mar 2023, at 18.10, Weston Pace wrote:
>
> I agree with Aldrin. `arrow_schema` has type ArrowSchema*. You are
> passing in `_schema` which has type `ArrowSchema**` but you are
> casting it to `duckdb_arrow_schema*` so it at
I agree with Aldrin. `arrow_schema` has type ArrowSchema*. You are passing
in `_schema` which has type `ArrowSchema**` but you are casting it to
`duckdb_arrow_schema*` so it at least compiles. You can also simplify
things by getting rid of some of this allocation.
```
ArrowSchema
with
> RecordBatch from ImportRecordBatch but It didn’t work.
>
> K.
> --
> Kimmo Linna
> Nihtisalontie 3 as 1
> 02630 ESPOO
> kimmo.li...@gmail.com
> +358 40 590 1074
>
>
>
> On 2. Mar 2023, at 21.05, Weston Pace wrote:
>
> I believe you would need
If you are working in C++ there are a few interfaces you might be
interested in.
The simplest high level API for this would be to use Acero and create a
write node. This is what pyarrow uses (though a little indirectly at the
moment). There is a brief example here[1]. I'd be happy to answer
I believe you would need a Javascript version of the C data interface.
This should be doable with bun:ffi but I'm not aware of anyone that has
done this before. I also wonder if there is a way to create a C data
interface based on TypedArray that would be usable in both bun and node.
I'm also not
> Question to others - is there option to read rows from 1000th to
> 2000th in current parquet interface?
I'm pretty sure the answer today is "no". In theory, however, you
should be able to narrow down which pages to load using a skip. I
believe there is some active work in this area[1].
>
You could store it a List column:
```
>>> x = pa.array([[1.2, 2.3], [3.4]])
>>> x
[
[
1.2,
2.3
],
[
3.4
]
]
>>> x[0]
>>> x[0][1]
>>> x[0].values.slice(0, 1)
[
1.2
]
```
This will be stored in
> 2. Some of the files end up being larger than memory when unzipped. In this
> case I’m using the file size to switch over and use open_csv instead of
> read_csv. Is there any plan for open_csv to be multithreaded in a future
> release (didn’t see anything on Jira, but I’m not great at
ng Custom Rule C:/Temp/arrow/minimal_build/CMakeLists.txt
>> example.cc
>> arrow-example.vcxproj ->
>> C:\Temp\arrow\minimal_build\Debug\arrow-example.exe
>> Building Custom Rule C:/Temp/arrow/minimal_build/CMakeLists.txt
>> Placed test.csv in the
7FF68F936C3C in arrow-example.exe: 0xC005:
> Access violation reading location 0x.
>
> Regards
> Prasad
>
>
> On Mon, Nov 7, 2022 at 12:43 AM Weston Pace wrote:
>>
>> Can you try res.status().ToString() ?
>>
>> On Sun, Nov 6, 2022, 5:31 AM Ra
Can you try res.status().ToString() ?
On Sun, Nov 6, 2022, 5:31 AM Raghavendra Prasad wrote:
> Hi,
>
> Hope you are all well!
>
> I am struggling to get the minimal sample app to work & am hoping for tips
> to move forward
>
> *Setup*: Windows 10, Visual Studio 2019, Arrow 9.0.0 installed via
>
To check for null you can use the `is_null` function:
```
import pyarrow as pa
import pyarrow.compute as pc
import pyarrow.dataset as ds
tab = pa.Table.from_pydict({"x": [1, 2, 3, None], "y": ["a", "b", "c",
"d"]})
filtered = ds.dataset(tab).to_table(filter=pc.is_null(pc.field("x")))
It's not in the footer metadata but each record batch should have its
own metadata the batch's metadata should contain the # of rows. So
you should be able to do it without reading any data. In pyarrow,
this *should* be what count_rows is doing but it has been a while
since I've really dove into
but memory mapping is related
> to the ipc format only, formats such as parquet cannot take advantage of this?
>
> Kind regards
> Nikhil Makan
>
>
> On Tue, Sep 20, 2022 at 5:12 AM Weston Pace wrote:
>>
>> Sorry for the slow reply.
>>
>> > This could
Thanks for the detailed reproducer. I've added a few notes on the JIRA
that I hope will help.
On Tue, Sep 20, 2022, 5:10 AM 1057445597 <1057445...@qq.com> wrote:
> I re-uploaded a copy of the code that can be compiled and run in
> join_test.zip, including cmakelists.txt, the test data files and
> However, I'm wondering if there's a better path to integrating that more
> "natively" into Arrow. Happy to make contributions if that's an option.
I'm not quite sure I understand where Arrow integration comes into
play here. Would that scanner use Arrow internally? Or would you
only convert
Sorry for the slow reply.
> This could be something on the Azure side but I find I am being bottlenecked
> on the download speed and have noticed if I spin up multiple Python sessions
> (or in my case interactive windows) I can increase my throughput. Hence I can
> download each year of the
Within Arrow-C++ that is the only way I am aware of. You might be able to
use DuckDb. It should be able to scan parquet files.
Is this the same program that you shared before? Were you able to figure
out threading? Can you create a JIRA with some sample input files and a
reproducible example?
Setting the batch size will not have too much of an impact on the
amount of memory used. That is mostly controlled by I/O readahead
(e.g. how many record batches to read at once). The readahead
settings are not currently exposed to pyarrow although a PR was
recently merged[1] that should make
question. Since column data
> types can be different data types, and data pages are a fixed size (e.g.
> default 1MB), how do mixed-size tables work w.r.t written data pages? Would
> an int32 rowgroup have less data pages than an int64 rowgroup?
>
> Thank You,
> Arun
>
in keeping my memory usage down, so I'm trying
> to get a better sense of the memory "landscape" that parquet utilizes. Once
> the row group is persisted to disk, the space that the row group previously
> utilized in memory should be freed for use once more right?
>
> Tha
The constant DEFAULT_MAX_ROW_GROUP_LENGTH is for
parquet::WriterProperties::max_row_group_length and the unit here is #
of rows. This is used by parquet::ParquetFileWriter. The
parquet::StreamWriter class wraps an instance of a file writer and
adds the property MaxRowGroupSize. This units for
This PR is what I was looking for and helps alleviate the learning
> curve.
>
> Thanks
>
>
>
> On Thu, Aug 25, 2022, 8:45 PM Weston Pace wrote:
>>
>> > Is there a way to get value_counts of a given column after doing table
>> > group_by?
>>
>
> Is there a way to get value_counts of a given column after doing table
> group_by?
Is your goal to group by some key and then get the value counts of an
entirely different non-key column? If so, then no, not today, at
least not directly. The only group by node we have is a hash-group-by
and
Well, all code has potential bugs :). We generally do not lower our
quality standards for experimental code. So it should still be unit
tested, reviewed, etc. The term is primarily meant to indicate that the
interface could be modified later. Acero, in particular, has quite a few
tests in
I think it is a case of "and" vs. "and_kleene". Also note that `ToString`
has special cases for the _kleene variants which is why you see "(a and b)"
vs. "and(a, b)".
`and_(field_ref("a"), field_ref("b"))` yields `call("and_kleene",
{field_ref("a"), field_ref("b")})`
On Thu, Aug 18, 2022 at
ter [2].
>
> [1]:
> https://github.com/apache/arrow/blob/master/cpp/src/arrow/compute/exec/subtree_internal.h
> [2]:
> https://github.com/apache/arrow/blob/60a1919527003a55a39785633bdbbbeef412c362/cpp/src/arrow/dataset/file_base.cc#L207-L268
>
> On Thu, Aug 4, 2022, at 17:30, West
ts(predicate) could walk the tree (in DFS
order), skipping entire nodes that fail the predicate.
On Thu, Aug 4, 2022 at 9:54 AM Tomaz Maia Suller
wrote:
> Weston, I'm interested in following up.
>
>
> --
> *De:* Weston Pace
> *Enviado:* quinta-feira, 4 de agos
There is a lot of room for improvement here. In the datasets API the call
that you have described (read_parquet) is broken into two steps:
* dataset discovery
During dataset discovery we don't use any partition filter. The goal is to
create the "total dataset" of all the files. So in your
No. There is no text format for expressions or related parser. I believe
the Substrait project still would like to invent a text format someday and
expressions would need to be a part of that so I think there is some
interest.
In the meantime a PR here (or at the Substrait project to align on
[1] https://issues.apache.org/jira/browse/ARROW-17042
On Mon, Jul 11, 2022 at 6:58 AM Wes McKinney wrote:
>
> Would someone like to open a Jira issue about this? This seems like an
> easy rough edge to fix
>
> On Wed, Jul 6, 2022 at 12:44 PM Weston Pace wrote:
> >
> >
If precision is not important you can cast the column to float64 first.
>>> x = pa.array([1, 2, 3], type=pa.decimal128(6, 1))
>>> x.cast(pa.float64()).cast(pa.string())
[
"1",
"2",
"3"
]
If precision is important you could use python or pandas to do the
> I was wondering if it is possible to add a C++ Function to the Compute
> FunctionRegistry and then use the functions in python.
Yes. This is partly handled automatically I believe when functions
are added to the C++ function registry. There are generally two ways
to call the function. For
he main memory, and
>> we can record the exec stats for every batch , and do better local task
>> scheduling based on those stats. Thanks a lot. Morsels is new to me, very
>> interesting ideas
>> Sent from my iPhone
>>
>> > On May 25, 2022, at 7:23 AM, Westo
I'd be more interested in some kind of buffer / array pool plus the
ability to specify an output buffer for a kernel function. I think it
would achieve the same goal (avoiding allocation) with more
flexibility (e.g. you wouldn't have to overwrite your input buffer).
At the moment though I wonder
Arrow C++ depends on a number of 3rd party libraries. Often this is
invisible because, by default, Arrow will do this:
* Check to see if the library is installed on the system
* If it is not installed on the system, download and build the library
So for gflags it appears gflags was not
It appears that it cannot find gflags and that is causing the build to
fail. Is it possible you do not have a 32bit version of gflags
installed?
> CMake Warning at cmake_modules/FindgflagsAlt.cmake:25 (find_package):
> By not providing "Findgflags.cmake" in CMAKE_MODULE_PATH this project has
>
There are a few levels of loops. Two at the moment and three in the
future. Some are fused and some are not. What we have right now is
early stages, is not ideal, and there are people investigating and
working on improvements. I can speak a little bit about where we want
to go. An example may
We do not have the option to do this today. However, it is something
we could do a better job of as long as we aren't reading CSV.
Aldrin's workaround is pretty solid, especially if you are reading
parquet and have a row_index column. Parquet statistics filtering
should ensure we are only
Can you serialize the schema by creating an IPC file with zero record
batches? I apologize, but I do not know the JS API as well. Maybe
you can create a table from just a schema (or a schema and a set of
empty arrays) and then turn that into an IPC file? This shouldn't add
too much overhead.
jira/browse/ARROW-16275
On Thu, Apr 21, 2022 at 6:44 AM Micah Kornfield wrote:
>>
>> If parquet stores statistics for each column of a struct array (don't know
>> offhand if they do) then we should create a JIRA to expose this.
>
>
> It does store statistics per-leaf colum
> However I cannot figure out any easy way to get the mapping
> used to create the dictionary array (vals) easily from the table. Can
> you please let me know the easiest way?
A dictionary is going to be associated with an array and not a table.
So you first need to get the array from the table.
Dutta wrote:
> That works! Thanks. Do you know off hand if this filter would be used in a
> predicate pushdown for a parquet dataset? Or would it be possibly coming in
> version 8.0.0?
>
> On Wed, Apr 20, 2022 at 3:49 PM Weston Pace wrote:
>
>> The second argument t
The second argument to `call_function` should be a list (the args to
the function). Since `arr3` is iterable it is interpreting it as a
list of args and trying to treat each row as an argument to your call
(this is the reason it thinks you have 3 arguments). This should
work:
of
> > parallelism downloading or pipelining technique?
> >
> > On Thu, Apr 14, 2022 at 4:51 AM Weston Pace wrote:
> > >
> > > Yes, that matches my understanding as well. I think, when
> > > pre-buffering is enabled, you might get parallel reads e
>From a pure metadata-only perspective you should be able to get the
size of the column and possibly a null count (for parquet files where
statistics are stored). However, you will not be able to get the
indices of the nulls.
The null count and column size are going to come from the parquet
Yes, that matches my understanding as well. I think, when
pre-buffering is enabled, you might get parallel reads even if you are
using ParquetFile/read_table but I would have to check. I agree that
it would be a good idea to add some documentation to all the readers
going over our parallelism at
I think that would be a reasonable JIRA request. In addition I think
it would also make sense for the `pa.array` function to recognize
scalars:
>>> pa.array([1])
[
1
]
>>> pa.array([pa.scalar(1)])
# --ERR--
On Wed, Mar 30, 2022 at 7:01 PM Wenlei Xie wrote:
>
> Hi,
>
> When play around
Yes and no :) Disclaimer: this answer is a little out of my
wheelhouse as I've learned relational algebra through doing and so my
formal theory may be off. Anyone is welcome to correct me. Also,
this answer turned into a bit of ramble and is a bit scattershot. You
may already be very familiar
ially the
>>>>>>>>>> function being called, and [2] is an example of a wrapper function
>>>>>>>>>> (more in
>>>>>>>>>> that file) I wrote to reduce boilerplate (to make [1] more
>>>>>>>>>> read
Procedurally, you should create a JIRA ticket with your request and
once the fix is complete and merged into master you can request a
release on the mailing list. For an example, see [1].
> Is there a way to request that pyarrow/arrow gets built with the fixed
> version of the AWS SDK for the
As far as I know (and my knowledge here may be dated) the compute
kernels themselves do not do any concurrency. There are certainly
compute kernels that could benefit from concurrency in this manner
(many kernels naively so) and I think things are setup so that, if we
decide to tackle this
nt this category of problem in the
> future?
>
> On Thu, Feb 24, 2022 at 9:48 PM Weston Pace wrote:
> >
> > Thanks for reporting this. It seems a regression crept into 7.0.0
> > that accidentally disabled parallel column decoding when
> > pyarrow.parquet.read_table i
e Static Debug runtime while another is using the Dynamic
> Debug runtime)
>
>
> On Sun., Mar. 6, 2022, 11:46 Weston Pace, wrote:
>>
>> You may need to share some specific error messages or a build log.
>>
>> Or, ideally, setup some kind of CI environment
I don't think you need to do anything differently. This looks like
the correct way to run this. The arrow streaming engine is pretty new
and probably has some good low hanging performance improvements that
could be made. I think most of the dedicated performance work has
been on either
You may need to share some specific error messages or a build log.
Or, ideally, setup some kind of CI environment that mirrors your
environment and reproduce your failure. Github has runners that have
support for visual studio 2022 and I believe there are actions to
install vcpkg. You could
Does setting UseAsync on the C++ end make a difference? It's possible we
switched the default to async in python in 6.0.0 but not in C++.
On Tue, Mar 1, 2022, 11:35 Niranda Perera wrote:
> Oh, I forgot to mention, had to fix LD_LIBRARY_PATH when running the c++
> executable.
>
t; as a parquet file, with compression=snappy and use_dictionary=true. The data
> has ~20M rows and 43 columns. So there is only one row group with
> row_group_size=64M as default. The OS is Ubuntu 20.04 and the file is on
> local disk.
>
> Weston Pace 于2022年2月24日周四 16:45写道:
&
this issue?
On Wed, Feb 23, 2022, 10:34 PM Shawn Zeng wrote:
> use_legacy_dataset=True fixes the problem. Could you explain a little
> about the reason? Thanks!
>
> Weston Pace 于2022年2月24日周四 13:44写道:
>
>> What version of pyarrow are you using? What's your OS? Is the fi
What version of pyarrow are you using? What's your OS? Is the file on a
local disk or S3? How many row groups are in your file?
A difference of that much is not expected. However, they do use different
infrastructure under the hood. Do you also get the faster performance with
I'm pretty sure GCS is similar to S3 in that there is no such thing as
a "directory". Instead a directory is often emulated by an empty
file. Note that the single file being detected is hires-sonde/ (with
a trailing slash). I'm pretty sure this is the convention for
creating mock directories.
These are all great points. A few notes from my own experiments
(mostly confirming what others have said):
1) 1M rows is the minimum safe size for row groups on HDD (and
perhaps a hair too low in some situations) if you are doing any kind
of column selection (i.e. projection pushdown). As that
> 3. Given our limited API sets, what would you recommend?
The filesystem interface is already rather minimal. We generally
don't put a function in there if we aren't using it somewhere. That
being said, you can often get away with a mock implementation. From a
quick rundown:
If you are aiming to speed up processing / consumption of batches then
you can use a queue. For example:
# Once per client
pqf = pq.ParquetFile(f, memory_map=True)
# maxsize will control how much you all buffer in RAM
queue = queue.Queue(maxsize=32)
start_workers(queue)
for batch in
>
> -- Configuring incomplete, errors occurred!
> See also
> "~/build_arrow/arrow/python/build/temp.linux-x86_64-3.7/CMakeFiles/CMakeOutput.log".
> error: command '/usr/bin/cmake' failed with exit code 1
>
> On Tue, Jan 25, 2022 at 12:27 AM Weston Pace wrote:
>
Your problem is probably old hardware, specifically an older CPU. Pip
builds rely on popcnt (which I think is SSE4.1?)
I'm pretty sure you are right that you can compile from source and be ok.
It's a performance / portability tradeoff that has to be made when
packaging prebuilt binaries.
On
he only solution I see is calling the compute function for each row of the
>> RecordBatch (transforming each row to a pyarrow.array somehow). But this
>> would be quite inefficient. On the contrary, pandas can compute the
>> aggregation across columns in a vectorized way
));
> } else {
> RETURN_NOT_OK(
> GetDictionaryPayload(dictionary_id, dictionary, options_,
> ));
> }
> RETURN_NOT_OK(WritePayload(payload));
> ++stats_.num_dictionary_batches;
> if (dict
I'm guessing you mean write_table? Assuming you are passing a
filename / string (and not an open output stream) to write_table I
would expect that any files opened during the call have been closed
before the call returns.
Pedantically, this is not quite the same thing as "finished writing on
Wes' theory seems sound. Perhaps the easiest way to test that theory would
be to put a five second sleep after the to_table call and before you run
show_mem. In theory 1s is long enough but 5s is nice to remove any doubt.
If there is a filter (that cannot be serviced by parquet row group
Bit of a late reply but currently it is not possible.
In theory this should be possible for a list of known IPC files. One
could calculate the total size of every column from the metadata of
all the files. This would require reading in the metadata (not just
the footer chunk at the end but each
1 - 100 of 137 matches
Mail list logo