Coder Social home page Coder Social logo

materializeinc / materialize Goto Github PK

View Code? Open in Web Editor NEW
5.6K 75.0 452.0 190.61 MB

The data warehouse for operational workloads.

Home Page: https://materialize.com

License: Other

Rust 82.52% Shell 0.82% Awk 0.01% Dockerfile 0.17% HTML 0.04% Python 14.63% JavaScript 0.26% TypeScript 0.02% Java 0.03% CSS 0.01% C# 0.02% Emacs Lisp 0.01% jq 0.02% C 0.08% Nix 0.01% CMake 0.01% C++ 1.17% Handlebars 0.01% Ruby 0.11% PLpgSQL 0.04%
rust database sql streaming kafka distributed-systems postgresql-dialect materialized-view stream-processing postgresql

materialize's Introduction

Build status Doc reference Chat on Slack

Materialize is a cloud-native data warehouse purpose-built for operational workloads where an analytical data warehouse would be too slow, and a stream processor would be too complicated.

Using SQL and common tools in the wider data ecosystem, Materialize allows you to build real-time automation, engaging customer experiences, and interactive data products that drive value for your business while reducing the cost of data freshness.

Get started

Ready to try out Materialize? Sign up to get started! 🚀

Have questions? We'd love to hear from you:

About

Materialize is designed to help you interactively explore your streaming data, perform analytics against live relational data, or increase data freshness while reducing the load of your dashboard and monitoring tasks. The moment you need a refreshed answer, you can get it in milliseconds.

It focuses on providing correct and consistent answers with minimal latency, and does not ask you to accept either approximate answers or eventual consistency. Whenever Materialize answers a query, that answer is the correct result on some specific (and recent) version of your data. Materialize does all of this by recasting your SQL queries as dataflows, which can react efficiently to changes in your data as they happen.

Our fully managed service is cloud native, featuring high availability, via multi-active replication, horizontal scalability, by seamlessly scaling dataflows across multiple machines, and near infinite storage, by leveraging cloud object storage (e.g., Amazon S3).

We support a large fraction of PostgreSQL, and are actively working on supporting more built-in PostgreSQL functions. Please file an issue if something doesn't work as expected!

Get data in

Materialize can read data from Kafka (and other Kafka API-compatible systems like Redpanda), directly from a PostgreSQL replication stream, or from SaaS applications via webhooks. It also supports regular database tables to which you can insert, update, and delete rows.

Transform, manipulate, and read your data

Once you've got the data in, define views and perform reads via the PostgreSQL protocol. Use your favorite SQL client, including the psql you probably already have on your system.

Materialize supports a comprehensive variety of SQL features, all using the PostgreSQL dialect and protocol:

  • Joins, joins, joins! Materialize supports multi-column join conditions, multi-way joins, self-joins, cross-joins, inner joins, outer joins, etc.
  • Delta-joins avoid intermediate state blowup compared to systems that can only plan nested binary joins - tested on joins of up to 64 relations.
  • Support for subqueries. Materialize's SQL optimizer performs subquery decorrelation out-of-the-box, avoiding the need to manually rewrite subqueries into joins.
  • Materialize can incrementally maintain views in the presence of arbitrary inserts, updates, and deletes. No asterisks.
  • All the aggregations: min, max, count, sum, stddev, etc.
  • HAVING
  • ORDER BY
  • LIMIT
  • DISTINCT
  • JSON support in the PostgreSQL dialect including operators and functions like ->, ->>, @>, ?, jsonb_array_element, jsonb_each. Materialize automatically plans lateral joins for efficient jsonb_each support.
  • Nest views on views on views!
  • Multiple views that have overlapping subplans can share underlying indices for space and compute efficiency, so just declaratively define what you want, and we'll worry about how to efficiently maintain them.

Just show us what it can do!

Here's an example join query that works fine in Materialize, TPC-H query 15:

-- Views define commonly reused subqueries.
CREATE VIEW revenue (supplier_no, total_revenue) AS
    SELECT
        l_suppkey,
        SUM(l_extendedprice * (1 - l_discount))
    FROM
        lineitem
    WHERE
        l_shipdate >= DATE '1996-01-01'
        AND l_shipdate < DATE '1996-01-01' + INTERVAL '3' month
    GROUP BY
        l_suppkey;

-- The MATERIALIZED keyword is the trigger to begin
-- eagerly, consistently, and incrementally maintaining
-- results that are stored directly in durable storage.
CREATE MATERIALIZED VIEW tpch_q15 AS
  SELECT
    s_suppkey,
    s_name,
    s_address,
    s_phone,
    total_revenue
FROM
    supplier,
    revenue
WHERE
    s_suppkey = supplier_no
    AND total_revenue = (
        SELECT
            max(total_revenue)
        FROM
            revenue
    )
ORDER BY
    s_suppkey;

-- Creating an index keeps results always up to date and in memory.
-- In this example, the index will allow for fast point lookups of
-- individual supply keys.
CREATE INDEX tpch_q15_idx ON tpch_q15 (s_suppkey);

Stream inserts, updates, and deletes on the underlying tables (lineitem and supplier), and Materialize keeps the materialized view incrementally updated. You can type SELECT * FROM tpch_q15 and expect to see the current results immediately!

Get data out

Pull based: Use any PostgreSQL-compatible driver in any language/environment to make SELECT queries against your views. Tell them they're talking to a PostgreSQL database, they don't ever need to know otherwise.

Push based: Listen to changes directly using SUBSCRIBE or configure Materialize to stream results to a Kafka topic as soon as the views change.

Documentation

Check out our documentation.

License

Materialize is provided primarily as a fully managed cloud service with credit-based pricing. Included in the price are proprietary cloud-native features like horizontal scalability, high availability, and a web management console.

However, we're big believers in advancing the frontier of human knowledge. To that end, the source code of the standalone database engine is publicly available, in this repository, and licensed under the BSL 1.1, converting to the open-source Apache 2.0 license after 4 years. As stated in the BSL, use of the standalone database engine on a single node is free forever. Please be warned that this deployment model is not suitable for production use and we cannot offer support for it.

Materialize depends upon many open source Rust crates. We maintain a list of these crates and their licenses, including links to their source repositories.

For developers

Materialize is primarily written in Rust.

Developers can find docs at doc/developer, and Rust API documentation is hosted at https://dev.materialize.com/api/rust/. The Materialize development roadmap is divided up into roughly month-long milestones, and managed in GitHub.

Contributions are welcome. Prospective code contributors might find the D-good for external contributors label useful. See CONTRIBUTING.md for additional guidance.

Credits

Materialize is lovingly crafted by a team of developers and one bot. Join us.

materialize's People

Contributors

aalexandrov avatar aljoscha avatar andrioni avatar antiguru avatar benesch avatar bkirwi avatar danhhz avatar def- avatar dependabot-preview[bot] avatar dependabot[bot] avatar frankmcsherry avatar ggevay avatar guswynn avatar jamii avatar jkosh44 avatar jldlaughlin avatar maddyblue avatar materialize-bot avatar morsapaes avatar nrainer-materialize avatar parkmycar avatar petrosagg avatar philip-stoev avatar quodlibetor avatar rjobanp avatar ruchirk avatar sploiselle avatar teskje avatar umanwizard avatar vmarcos avatar

Stargazers

 avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar

Watchers

 avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar

materialize's Issues

Timely worker synchronization for new dataflow creation

All timely workers need to create new dataflows in the exact same order. This is arguably an implementation flaw - it is possible that timely workers could have a namespace and new dataflows could register against this namespace. But today, there is a single global counter from which channels are assigned for dataflows. Thus, every timely worker must create new dataflows in the same order.

In order to facilitate this ordering, the interactive crate has a meta-dataflow in which workers insert their intents to create new dataflows. This meta-dataflow assigns an ordering, which workers use to coordinate.

One disadvantage of this hack is that dataflows are not persistent. if we want to be able to recover from crashes, we want to be able to see the history of dataflows we created (i.e. the views). Thus, replacing this meta-dataflow framework with a consensus-replicated durable log fulfills this other goal, as well as satisfying the original desideratum of having a consistent shared ordering.

We should rip out the meta-dataflow, and instead put all dataflow creation commands through a shared durable log (i.e. on top of zookeeper/etcd).

metastore: maintain consistency

Metastore will currently allow you delete a dataflow that's depended upon by other dataflows. There's a more subtle problem when creating a dataflow: you might create a dataflow that depends on a dataflow that was just deleted. The create dataflow operation needs to be part of a transaction that verifies that all dependent dataflows still exist.

There are TODOs in the relevant metastore functions to this effect.

Data plane consistency

Problem

#33 deals with control plan consistency. Regardless of which solution we adopt, we still have a related issue on the data plane - we don't have a notion of data plane time and so we don't have any kind of consistency for PEEK. Consequences:

  • tests have no way to know if PEEK results are wrong or just not caught up yet, so they have to set a long timeout before failing
  • in a replicated setting, PEEK might go back in time if executed against different servers
  • multiple timely workers might resolve PEEK at different timestamps? (I'm not sure of this, but I think @benesch mentioned it)

Solution A

@frankmcsherry suggested rather than allowing workers to read freely from Kafka, add an Advance command which contains a Kafka timestamp and allows workers to read up to the timestamp. This ties data plane times to control plane times, so we can now define PEEK consistency by saying that it must return results that are correct as of the last Advance before the PEEK.

Initial questions:

  • Who sends the Advance?
  • How much will this impact latency?
  • Can our control plane take the throughput?

sql: improve min/max aggregations

The nascent support for aggregations uses the simplest possible approach, which involves using differential's comparatively inefficient reduce operator. Here's some pseudocode for how SELECT sum(total), min(order_date) FROM ... GROUP BY <some col> is currently implemented:

.reduce(|key, input, output| {
    let mut min_order_date = +Inf;
    let mut sum_total = 0;
    for record in input {
        if input.order_date < min_order_date {
            min_order_date = input.order_date
        }
        sum_total += input.total;
    }
    output.push((min_order_date, sum_total))
})

Computing counts and sums (and averages, I think?) can be far more efficient, if you stuff the aggregatable column into differential's "diff" field and use the count operator. Mins/maxs can also be more efficient, if you use this hierarchical min/max technique that Frank manually coded here: https://github.com/TimelyDataflow/differential-dataflow/blob/c2fe19dffb007cbc2720e6df79b0bebaae9f2097/tpchlike/src/queries/query15.rs#L70-L91.

The Zulip conversation where this was fleshed out is here: https://timely.zulipchat.com/#narrow/stream/181610-.5Barchived.5D-differential-dataflow/topic/reduce/near/162364355

GROUP BY is not grouping?

Materialize:

jamie=> create table foo(a integer);
CREATE TABLE
jamie=> insert into foo values (42);
INSERT 0 1
jamie=> insert into foo values (37);
INSERT 0 1
jamie=> insert into foo values (42);
INSERT 0 1
jamie=> select * from foo;
 a  
----
 37
 42
 42
(3 rows)

jamie=> select * from foo group by a;
 a  
----
 37
 42
 42
(3 rows)

Sqlite:

sqlite> create table foo( a integer);
sqlite> insert into foo values(42);
sqlite> insert into foo values(42);
sqlite> insert into foo values(37);
sqlite> select * from foo;
42
42
37
sqlite> select * from foo group by a;
37
42

Types and unwrapping panics

Our use of dynamic typing (or .. no typing?) leads to some awkward moments where types are not right and it is a bit hard to immediately see why.

From Zulip:


Frank McSherry: @jamie Brandon I've not fully debugged this, but I'm seeing a panic in unwrap_int32() in

match (aggregates[0].0).func {
    AggregateFunc::SumInt32 => {
        data.explode(|(key,val)| Some((key,val.unwrap_int32() as isize)))
            .count()
            .map(|(mut key,sum)| { key.push(Datum::Int32(sum as i32)); key })
}

where it claims we have a Datum::Int64 as input. It could easily be some screw-up on my part, but is it expected that all the types are correct all the way through?

Frank McSherry: Wait, NO! My fault sorry. It wasn't there, but rather that (elsewhere) I failed to wrap the output of Count as Datum::Int64 and had instead wrapped it as Datum::Int32.


This may just be an unavoidable pain of a language that needs to support a mix of numeric, boolean, and other (string) data in the same positions. However, anything we can think to do to detect these issues earlier might help.

IR design

Preliminary thoughts

IR wants:

  • Narrow waist - small number of orthogonal operators with simple semantics and no surprising edge cases
  • Simple rules for transformations eg predicate pushdown
  • Easy to map SQL -> dumb IR -> smart IR -> differential
  • Represent names/scope in a way that doesn't require constant fixing during transformations (maybe UUIDs per unique column?)
  • Be able to represent multi-way joins

Things to do in SQL -> dumb IR:

  • Name resolution - handle all of SQLs weird scoping rules
  • Common subexpression elimination - to handle queries like select (a+1)+1 from foo group by a+1
  • Similarly, name aggregates by UUID to avoid the double lookup we do at the moment.
  • Express joins as product + filter to accurately capture semantics - optimize in later pass
  • TODO find nice IR for:
    • aggregates without group by - have to produce output when the input is empty
    • outer joins - requires some kind of lookup operator, but not clear how flexible it needs to be

Do we want to change our type representation while we're here?

  • We've talked about splitting Type into RowType and ScalarType so that we don't have .unwrap_tuple littered around the code. Similarly for Datum into Row and Scalar.
  • The column name in type doesn't carry enough information - SQLPlan has it's own Name type instead to track table and function names. The IR might want to use UUIDs instead and just keep column names around for printing output.

materialized server skeleton

Forked from #1:

Rust's lack of a good threading story has been holding me up here. I think the right answer is probably to fork a thread for every incoming connection. That should hold us for a while, since I can't imagine the average application has more than hundreds of (producers + consumers + users holding open SQL shells).

We can start with a heavyweight threading architecture and explicitly tell users that they should be connection-pooling aggressively, that seems fine to me. SQL wire protocols in particular are heavyweight w.r.t. connections in any case - you don't typically open a new connection for each query. This would be more of an issue to solve soon if we started with a raw REST API, but that doesn't seem like the direction we're taking at the moment.

Roadmap I

Working roadmap for MVP, good enough for controlled demo to early clients.

  • Testing
    • Testdrive for (slow) end-to-end testing (@benesch)
    • Sqllogictest for SQL semantics (@jamii)
      • Parse test files
      • Support CREATE TABLE, INSERT and non-view queries
      • Figure out how to wait until input has been processed (#24)
      • Provide some way to categorize errors
      • Climb the mountain of test failures - #1129
    • Fuzzing for crashes - server never dies! (@jamii)
      • Basic integration with libFuzzer
      • Target testdrive and sqllogictest
      • Setup a fuzzing server
    • Chaos testing for distributed setups?
  • Benchmarking
    • Benchmarking / profiling tool (@benesch)
      • Measure end-to-end latency
      • Measure memory usage
      • Track time spent in each dataflow operator
    • TPC-H, because everyone will want to see it (#11)
      • Compare vs hand-coded differential dataflow (measures overhead of sql layer)
      • Compare vs eg memsql (measures how often you need to look at the results before materialize is worthwhile)
      • Compare vs ksql (most obvious competition)
    • Some benchmark with realistic skew...
  • SQL support
    • Check that we cover everything in the SQL-92 spec
    • Implement subqueries
    • Implement orderby/limit/offset
  • Performance - decide after benchmarking / profiling, but likely options are:
    • Store type tags per dataflow instead of per datum
    • Push filters down through joins
    • Vectorize - execute Expr per batch instead of per row
    • Incrementalize aggregates
    • Switch to delta queries for joins (#779)
    • Switch to worst-case optimal joins
    • Reuse dataflows, especially indexes
  • Architecture
    • Figure out control plane consistency / durability (#33) (@jamii)
    • Figure out data plane consistency (#34)
    • Support Kakfa sinks from views
      • Might also want an option to return a key-update/delete stream instead of a row-diff stream
  • Platform
    • Figure out packaging / deployment
    • Configuration
      • Mostly hardcoded at the moment
    • Documentation
    • Logging
      • Probably want logs as a dataflow, which we can query in SQL or sink into Kakfa
    • Introspection
      • At minimum, want to be able to query available dataflows ala pg_catalog

Probably not in MVP:

  • Distributed materialized
  • Active / active replication (some discussion in #33) (@jamii ?)

Split off into linked issues as needed.

DROP <THING> should verify it targets a <THING>

To match Postgres (and perhaps the SQL spec?) DROP TABLE should not be able to drop a view, DROP VIEW should not be able to drop a table, etc. At the moment we allow a DROP to target any <THING>.

Intermediate representation

Our intermediate representation, the dataflow::Plan type, was not chosen with any particular eye towards plan transformations, optimization, or potentially anything other than getting the dataflow built. We should probably be more intentional about our choice here, to make sure that we preserve as much information about the query as possible, and present it in an easy-to-use format.

Here are some thoughts, which are not yet actionably structured. We may need to try some of these out to see if they work for us.

There are several things we could discuss, but to summarize:

  1. We could decompose outer joins into more elemental plan stages.
  2. We could represent multiway joins as single plan stages.
  3. We could decompose delta queries into more elemental plan stages.
  4. We could move from Datum records to [Datum] records.

None of these are "should" choices. I am not sure we have enough experience with the tradeoffs yet.

Relational joins vs outer joins

We currently use the Join plan stage for both inner and outer joins. We may want to reduce outer joins down to more elemental operators, for example as

fn left_outer_join(left, right) {
    left.semijoin(y.keys().distinct())
        .negate()
        .concat(x)
        .map(|x| (x, None))
        .concat(left.join(y))
}

This might give us more insight into the shareable arrangements available to use (e.g. y.keys().distinct() and left).

Downsides include the loss of relational information. If we could have realized something structure from x.left_outer_join(y) that is no longer visible from the reduced representation, that is a problem.

Multiway joins

It seems like multiway joins are a useful abstraction, as at some point in the analysis process we will probably want to explore different join orders, or plans like delta queries and worst-case optimal joins. It seems appealing to have all of this information co-located in a single operator, when possible.

Downsides include:

  1. Outer joins are not obviously natural here, and we might need to break them down into their non-relational base operators.
  2. We can probably always reconstruct multiway joins as connected components of joins. If we are planning on decomposing outer joins we may need to do this anyhow, at which point forcing things into a multiway blob may not have added much value.

Delta joins

The implementation of delta joins uses some weird operators from the dogsdogsdogs differential project, but they actually each have fairly crisp semantics. For example, the propose operator (which is probably the most important one) takes two input update streams, left and right, and produces what I've been calling a "half-join"

for_each ((l_key, l_val), l_time, l_diff) in left : 
    for each ((l_key, r_val), r_accum) in right@l_time :
        output ((l_key, l_val, r_val), l_time, l_diff * r_accum)

which is essentially look-ups into right for each change in left. It is like an incremental join that does not respond to changes in right (and blocks on changes to left until it has the correct answer).

We could introduce this type of operator in to our plans, once we understand it a bit better. This would have the advantage that we could push projections and selections backwards through the operator. We could also plausibly find more opportunities for re-use: for a join on relations A, B, C, D, .. Z the delta rules often look like

dA x B x C x D .. x Z
dB x A x C x D .. x Z
...

which we should be able to reformulate as

((dA x B) + (dB x A)) x C x D .. x Z

which .. probably gives better performance as larger batches move through less dataflow.

Tuple vs Datum record types

We currently have a Datum type that can be a tuple, but need not be a tuple. This makes it a little less natural to discuss certain relational idioms, like the provenance of particular columns (e.g. from which source relations does a particular field of a tuple derive, so that we can push predicates closer to the sources).

Ideally we prefer a record type that allows us to maintain information about the mapping of "columns" as we cross plan stages, so that any per-record analyses can be propagated backwards through the dataflow graph.

There are several options, which include but are not limited to:

  1. We could leave things how they are, which passes a bit of a burden on to the analysis stages, but avoids pre-optimizing anything before we understand the pain points better.
  2. We could pivot to a [Datum] record type, and discuss support and provenance at the column level.
  3. We could go "whole PL" and retain information about the relations between arbitrary input and output fields in tree-structure Datum types.

One additional bonus for [Datum] records is that there is that much less interpretation for the common case of relational equijoins, in the special case where we only plan to pull out fields from the input records and act on them. In addition, it seems easier to specialize trace implementations to store contiguous [Datum] slices if we know the records look like that, and harder if there are flavors we might have to decode.

TODO

A lightweight list of things that need to be built. At some point keeping all of these tasks in one issue will fail to scale, but for now it's better than scattering knowledge across multiple issues.

  • materialized(1) server skeleton (#3)
    Rust's lack of a good threading story has been holding me up here. I think the right answer is probably to fork a thread for every incoming connection. That should hold us for a while, since I can't imagine the average application has more than hundreds of (producers + consumers + users holding open SQL shells).
  • SQL translation layer (#4)
    • Unit testing framework
      I think it makes the most sense to test the SQL translation layer as a pure function from SQL query to differential/timely plan (rather than something logictests style). We can have a higher level acceptance test that actually funnels data through a full Material server and cross-checks the result with another system.
  • Postgres wire implementation
  • Kafka wire implementation

My rough timeline is to have a working end-to-end prototype by the end of this week (Mar 1). To be clear, I expect this prototype to be broken in almost every way. My acceptance criteria are:

  • Ingest TPC-H data using the Kafka file source. This will require a Kafka wire implementation that is correct enough that the upstream Kafka connectors think we're a legit Kafka instance.
  • Create a materialized view TPC-H query 1 using the Rust Postgres adapter. This will require a Postgres wire implementation that supports connection handshaking and query execution.
  • Read from the materialized view and see the correct answer. This will require a SQL translation layer that can handle basic aggregations and filters.

sql: build it

Forked from #1.

Unit testing framework: I think it makes the most sense to test the SQL translation layer as a pure function from SQL query to differential/timely plan (rather than something logictests style). We can have a higher level acceptance test that actually funnels data through a full Material server and cross-checks the result with another system.

A little pushback here: it's not clear to me that we know what Timely plans we want. At Cockroach we did a bunch of these tests in distributed SQL planning, and it wasn't useful for anything more than manual inspection. It certainly wasn't a 'unit test' in any sense.

It's pretty easy to hook Timely up to read inputs from a file. So that should enable us to do SQL logictests easily. I'd push to hook up that path in order to do end-to-end logictests.

Timely assert failure

On repeated runs on RUST_BACKTRACE=full cargo run --bin=sqllogictest sqllogictest/test/random/aggregates/slt_good_23.test -vv we sometimes get:

thread 'worker thread 1' panicked at 'assertion failed: lower.len() > 0', /home/jamie/.cargo/git/checkouts/differential-dataflow-d065d23d797aa027/3773f17/src/trace/description.rs:78:3
stack backtrace:
   0:     0x55acdf821063 - std::sys::unix::backtrace::tracing::imp::unwind_backtrace::hc7b76c57e25c7d91
                               at src/libstd/sys/unix/backtrace/tracing/gcc_s.rs:39
   1:     0x55acdf81bcbb - std::sys_common::backtrace::_print::h203699a8f1b8b948
                               at src/libstd/sys_common/backtrace.rs:70
   2:     0x55acdf81edc6 - std::panicking::default_hook::{{closure}}::h74fcb67c61dc8b92
                               at src/libstd/sys_common/backtrace.rs:58
                               at src/libstd/panicking.rs:200
   3:     0x55acdf81eb44 - std::panicking::default_hook::hb2e06978a2102930
                               at src/libstd/panicking.rs:215
   4:     0x55acdf81f4cf - std::panicking::rust_panic_with_hook::h4ac0b487eb7e54da
                               at src/libstd/panicking.rs:478
   5:     0x55acdeea5e57 - std::panicking::begin_panic::hadbb0d8580e19fad
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:412
   6:     0x55acdeccfe24 - <differential_dataflow::trace::description::Description<Time>>::new::hcd2fd3535d443b4e
                               at /home/jamie/materialize/<::std::macros::panic macros>:3
   7:     0x55acdeba39f1 - <differential_dataflow::trace::implementations::ord::OrdKeyBuilder<K, T, R> as differential_dataflow::trace::Builder<K, (), T, R, differential_dataflow::trace::implementations::ord::OrdKeyBatch<K, T, R>>>::done::h53bad8180371116c
                               at /home/jamie/.cargo/git/checkouts/differential-dataflow-d065d23d797aa027/3773f17/src/trace/implementations/ord.rs:569
   8:     0x55acde8026a8 - <differential_dataflow::trace::implementations::merge_batcher::MergeBatcher<K, V, T, R, B> as differential_dataflow::trace::Batcher<K, V, T, R, B>>::seal::h3605435a70df1b94
                               at /home/jamie/.cargo/git/checkouts/differential-dataflow-d065d23d797aa027/3773f17/src/trace/implementations/merge_batcher.rs:99
   9:     0x55acde7ac08c - <differential_dataflow::trace::rc_blanket_impls::RcBatcher<K, V, T, R, B> as differential_dataflow::trace::Batcher<K, V, T, R, alloc::rc::Rc<B>>>::seal::hfe01f172593f863b
                               at /home/jamie/.cargo/git/checkouts/differential-dataflow-d065d23d797aa027/3773f17/src/trace/mod.rs:338
  10:     0x55acdecfb771 - <differential_dataflow::collection::Collection<G, (K, V), R> as differential_dataflow::operators::arrange::arrangement::Arrange<G, K, V, R>>::arrange_core::{{closure}}::{{closure}}::h98e9ca28a9e5b61b
                               at /home/jamie/.cargo/git/checkouts/differential-dataflow-d065d23d797aa027/3773f17/src/operators/arrange/arrangement.rs:594
  11:     0x55acde9b4288 - <timely::dataflow::stream::Stream<G, D1> as timely::dataflow::operators::generic::operator::Operator<G, D1>>::unary_frontier::{{closure}}::{{closure}}::h5fd152839f9fdb8c
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/generic/operator.rs:352
  12:     0x55acdeb84589 - <timely::dataflow::operators::generic::builder_rc::OperatorBuilder<G>>::build::{{closure}}::he5ec5b0fa21d3801
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/generic/builder_rc.rs:136
  13:     0x55acdea0d7e2 - <timely::dataflow::operators::generic::builder_raw::OperatorCore<T, PEP, PIP> as timely::scheduling::Schedule>::schedule::hffce1a16f8690a23
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/generic/builder_raw.rs:219
  14:     0x55acdecb503e - <timely::progress::subgraph::PerOperatorState<T>>::schedule::hebfc4f0fde4b8d41
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:646
  15:     0x55acdecb5977 - <timely::progress::subgraph::Subgraph<TOuter, TInner>>::activate_child::h9ae5d7849b54e217
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:329
  16:     0x55acdecb0b2b - <timely::progress::subgraph::Subgraph<TOuter, TInner> as timely::scheduling::Schedule>::schedule::hcacdec1bef1082b3
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:295
  17:     0x55acdf4a7454 - timely::worker::Wrapper::step::{{closure}}::h971ad292bc084c03
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:489
  18:     0x55acdf4a634d - <core::option::Option<T>>::map::hcd877f91cd6d63f8
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/option.rs:414
  19:     0x55acdf4a7272 - timely::worker::Wrapper::step::h079de6881be60958
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:489
  20:     0x55acde9fa3e0 - <timely::worker::Worker<A>>::step_or_park::ha3cf7ccb3a665639
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:227
  21:     0x55acde9fd0e2 - <timely::worker::Worker<A>>::step::hb691f6f1f3a3f589
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:144
  22:     0x55acde99f270 - <materialize::dataflow::server::Worker<'w, A>>::run::hda8e00cce02a56ae
                               at src/materialize/dataflow/server.rs:129
  23:     0x55acde99ebea - materialize::dataflow::server::serve::{{closure}}::h6bc7e17ac31d0bce
                               at src/materialize/dataflow/server.rs:47
  24:     0x55acdecef060 - timely::execute::execute::{{closure}}::h1f93346fe3e1bc0d
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/execute.rs:201
  25:     0x55acde9bcf29 - timely_communication::initialize::initialize_from::{{closure}}::h1252acab1e6178d6
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/communication/src/initialize.rs:269
  26:     0x55acdeaea803 - std::sys_common::backtrace::__rust_begin_short_backtrace::h12d7320851696d2d
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/sys_common/backtrace.rs:135
  27:     0x55acdeae7205 - std::thread::Builder::spawn_unchecked::{{closure}}::{{closure}}::hdb80b6cb28504035
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/thread/mod.rs:469
  28:     0x55acded1c055 - <std::panic::AssertUnwindSafe<F> as core::ops::function::FnOnce<()>>::call_once::h6a208203b39e18a3
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panic.rs:309
  29:     0x55acdecf5259 - std::panicking::try::do_call::hc0a910baaa414789
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:297
  30:     0x55acdf824689 - __rust_maybe_catch_panic
                               at src/libpanic_unwind/lib.rs:87
  31:     0x55acdecf513f - std::panicking::try::h2f37f341d87162d4
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:276
  32:     0x55acded1c095 - std::panic::catch_unwind::hc732feefc72cf84c
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panic.rs:388
  33:     0x55acdeae6fcf - std::thread::Builder::spawn_unchecked::{{closure}}::h6e0ecf038c3a03fe
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/thread/mod.rs:468
  34:     0x55acdeae72d7 - <F as alloc::boxed::FnBox<A>>::call_box::ha04220c2aedddf29
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/boxed.rs:749
  35:     0x55acdf823a2d - std::sys::unix::thread::Thread::new::thread_start::h7fc2d82a3b91bc88
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/boxed.rs:759
                               at src/libstd/sys_common/thread.rs:14
                               at src/libstd/sys/unix/thread.rs:81
  36:     0x7f2065be5ef6 - start_thread
  37:     0x7f206590522e - __GI___clone
  38:                0x0 - <unknown>
thread 'worker thread 0' panicked at 'called `Result::unwrap()` on an `Err` value: "SendError(..)"', src/libcore/result.rs:997:5
stack backtrace:
   0:     0xthread '55acdf821063worker thread 2' panicked at 'called `Result::unwrap()` on an `Err` value: "SendError(..)"', src/libcore/result.rs - :997:5
std::sys::unix::backtrace::tracing::imp::unwind_backtrace::hc7b76c57e25c7d91
                               at src/libstd/sys/unix/backtrace/tracing/gcc_s.rs:39
   1:     0x55acdf81bcbb - std::sys_common::backtrace::_print::h203699a8f1b8b948
                               at src/libstd/sys_common/backtrace.rs:70
   2:     0x55acdf81edc6 - std::panicking::default_hook::{{closure}}::h74fcb67c61dc8b92
                               at src/libstd/sys_common/backtrace.rs:58
                               at src/libstd/panicking.rs:200
   3:     0x55acdf81eb44 - std::panicking::default_hook::hb2e06978a2102930
                               at src/libstd/panicking.rs:215
   4:     0x55acdf81f4cf - std::panicking::rust_panic_with_hook::h4ac0b487eb7e54da
                               at src/libstd/panicking.rs:478
   5:     0x55acdf81f051 - std::panicking::continue_panic_fmt::h23ca3ed77b25f368
                               at src/libstd/panicking.rs:385
   6:     0x55acdf81ef35 - rust_begin_unwind
                               at src/libstd/panicking.rs:312
   7:     0x55acdf83872c - core::panicking::panic_fmt::hfa7141b9630aa10b
                               at src/libcore/panicking.rs:85
   8:     0x55acdea62647 - core::result::unwrap_failed::h838d4a9b229ac1a1
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/macros.rs:17
   9:     0x55acdea5210a - <core::result::Result<T, E>>::unwrap::h16acc825006b558b
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/result.rs:798
  10:     0x55acdeb4fbf4 - <timely_communication::allocator::process::Pusher<T> as timely_communication::Push<T>>::push::h4da557c4e6183be3
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/communication/src/allocator/process.rs:211
  11:     0x55acdece0280 - <timely_communication::allocator::counters::ArcPusher<T, P> as timely_communication::Push<T>>::push::h85a6c988ed825be6
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/communication/src/allocator/counters.rs:100
  12:     0x55acde6d701c - <alloc::boxed::Box<P> as timely_communication::Push<T>>::push::ha57c881827fe8054
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/communication/src/lib.rs:145
  13:     0x55acdeae651e - <timely::progress::broadcast::Progcaster<T>>::send::hc0869ed658a60e56
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/broadcast.rs:91
  14:     0x55acdecb5866 - <timely::progress::subgraph::Subgraph<TOuter, TInner>>::send_progress::h7cdecadd6c55079d
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:495
  15:     0x55acdecb0b4c - <timely::progress::subgraph::Subgraph<TOuter, TInner> as timely::scheduling::Schedule>::schedule::hcacdec1bef1082b3
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:301
  16:     0x55acdf4a7454 - timely::worker::Wrapper::step::{{closure}}::h971ad292bc084c03
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:489
  17:     0x55acdf4a634d - <core::option::Option<T>>::map::hcd877f91cd6d63f8
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/option.rs:414
  18:     0x55acdf4a7272 - timely::worker::Wrapper::step::h079de6881be60958
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:489
  19:     0x55acde9fa3e0 - <timely::worker::Worker<A>>::step_or_park::ha3cf7ccb3a665639
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:227
  20:     0x55acde9fd0e2 - <timely::worker::Worker<A>>::step::hb691f6f1f3a3f589
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:144
  21:     0x55acde99f270 - <materialize::dataflow::server::Worker<'w, A>>::run::hda8e00cce02a56ae
                               at src/materialize/dataflow/server.rs:129
  22:     0x55acde99ebea - materialize::dataflow::server::serve::{{closure}}::h6bc7e17ac31d0bce
                               at src/materialize/dataflow/server.rs:47
  23:     0x55acdecef060 - timely::execute::execute::{{closure}}::h1f93346fe3e1bc0d
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/execute.rs:201
  24:     0x55acde9bcf29 - timely_communication::initialize::initialize_from::{{closure}}::h1252acab1e6178d6
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/communication/src/initialize.rs:269
  25:     0x55acdeaea803 - std::sys_common::backtrace::__rust_begin_short_backtrace::h12d7320851696d2d
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/sys_common/backtrace.rs:135
  26:     0x55acdeae7205 - std::thread::Builder::spawn_unchecked::{{closure}}::{{closure}}::hdb80b6cb28504035
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/thread/mod.rs:469
  27:     0x55acded1c055 - <std::panic::AssertUnwindSafe<F> as core::ops::function::FnOnce<()>>::call_once::h6a208203b39e18a3
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panic.rs:309
  28:     0x55acdecf5259 - std::panicking::try::do_call::hc0a910baaa414789
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:297
  29:     0x55acdf824689 - __rust_maybe_catch_panic
                               at src/libpanic_unwind/lib.rs:87
  30:     0x55acdecf513f - std::panicking::try::h2f37f341d87162d4
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:276
  31:     0x55acded1c095 - std::panic::catch_unwind::hc732feefc72cf84c
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panic.rs:388
  32:     0x55acdeae6fcf - std::thread::Builder::spawn_unchecked::{{closure}}::h6e0ecf038c3a03fe
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/thread/mod.rs:468
  33:     0x55acdeae72d7 - <F as alloc::boxed::FnBox<A>>::call_box::ha04220c2aedddf29
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/boxed.rs:749
  34:     0x55acdf823a2d - std::sys::unix::thread::Thread::new::thread_start::h7fc2d82a3b91bc88
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/boxed.rs:759
                               at src/libstd/sys_common/thread.rs:14
                               at src/libstd/sys/unix/thread.rs:81
  35:     0x7f2065be5ef6 - start_thread
  36:     0x7f206590522e - __GI___clone
  37:                0x0 - <unknown>
stack backtrace:
   0:     0x55acdf821063 - std::sys::unix::backtrace::tracing::imp::unwind_backtrace::hc7b76c57e25c7d91
                               at src/libstd/sys/unix/backtrace/tracing/gcc_s.rs:39
   1:     0x55acdf81bcbb - std::sys_common::backtrace::_print::h203699a8f1b8b948
                               at src/libstd/sys_common/backtrace.rs:70
   2:     0x55acdf81edc6 - std::panicking::default_hook::{{closure}}::h74fcb67c61dc8b92
                               at src/libstd/sys_common/backtrace.rs:58
                               at src/libstd/panicking.rs:200
   3:     0x55acdf81eb44 - std::panicking::default_hook::hb2e06978a2102930
                               at src/libstd/panicking.rs:215
   4:     0x55acdf81f4cf - std::panicking::rust_panic_with_hook::h4ac0b487eb7e54da
                               at src/libstd/panicking.rs:478
   5:     0x55acdf81f051 - std::panicking::continue_panic_fmt::h23ca3ed77b25f368
                               at src/libstd/panicking.rs:385
   6:     0x55acdf81ef35 - rust_begin_unwind
                               at src/libstd/panicking.rs:312
   7:     0x55acdf83872c - core::panicking::panic_fmt::hfa7141b9630aa10b
                               at src/libcore/panicking.rs:85
   8:     0x55acdea62647 - core::result::unwrap_failed::h838d4a9b229ac1a1
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/macros.rs:17
   9:     0x55acdea5210a - <core::result::Result<T, E>>::unwrap::h16acc825006b558b
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/result.rs:798
  10:     0x55acdeb4fbf4 - <timely_communication::allocator::process::Pusher<T> as timely_communication::Push<T>>::push::h4da557c4e6183be3
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/communication/src/allocator/process.rs:211
  11:     0x55acdece0280 - <timely_communication::allocator::counters::ArcPusher<T, P> as timely_communication::Push<T>>::push::h85a6c988ed825be6
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/communication/src/allocator/counters.rs:100
  12:     0x55acde6d701c - <alloc::boxed::Box<P> as timely_communication::Push<T>>::push::ha57c881827fe8054
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/communication/src/lib.rs:145
  13:     0x55acdeae651e - <timely::progress::broadcast::Progcaster<T>>::send::hc0869ed658a60e56
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/broadcast.rs:91
  14:     0x55acdecb5866 - <timely::progress::subgraph::Subgraph<TOuter, TInner>>::send_progress::h7cdecadd6c55079d
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:495
  15:     0x55acdecb0b4c - <timely::progress::subgraph::Subgraph<TOuter, TInner> as timely::scheduling::Schedule>::schedule::hcacdec1bef1082b3
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:301
  16:     0x55acdf4a7454 - timely::worker::Wrapper::step::{{closure}}::h971ad292bc084c03
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:489
  17:     0x55acdf4a634d - <core::option::Option<T>>::map::hcd877f91cd6d63f8
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/option.rs:414
  18:     0x55acdf4a7272 - timely::worker::Wrapper::step::h079de6881be60958
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:489
  19:     0x55acde9fa3e0 - <timely::worker::Worker<A>>::step_or_park::ha3cf7ccb3a665639
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:227
  20:     0x55acde9fd0e2 - <timely::worker::Worker<A>>::step::hb691f6f1f3a3f589
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:144
  21:     0x55acde99f270 - <materialize::dataflow::server::Worker<'w, A>>::run::hda8e00cce02a56ae
                               at src/materialize/dataflow/server.rs:129
  22:     0x55acde99ebea - materialize::dataflow::server::serve::{{closure}}::h6bc7e17ac31d0bce
                               at src/materialize/dataflow/server.rs:47
  23:     0x55acdecef060 - timely::execute::execute::{{closure}}::h1f93346fe3e1bc0d
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/execute.rs:201
  24:     0x55acde9bcf29 - timely_communication::initialize::initialize_from::{{closure}}::h1252acab1e6178d6
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/communication/src/initialize.rs:269
  25:     0x55acdeaea803 - std::sys_common::backtrace::__rust_begin_short_backtrace::h12d7320851696d2d
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/sys_common/backtrace.rs:135
  26:     0x55acdeae7205 - std::thread::Builder::spawn_unchecked::{{closure}}::{{closure}}::hdb80b6cb28504035
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/thread/mod.rs:469
  27:     0x55acded1c055 - <std::panic::AssertUnwindSafe<F> as core::ops::function::FnOnce<()>>::call_once::h6a208203b39e18a3
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panic.rs:309
  28:     0x55acdecf5259 - std::panicking::try::do_call::hc0a910baaa414789
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:297
  29:     0x55acdf824689 - __rust_maybe_catch_panic
                               at src/libpanic_unwind/lib.rs:87
  30:     0x55acdecf513f - std::panicking::try::h2f37f341d87162d4
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:276
  31:     0x55acded1c095 - std::panic::catch_unwind::hc732feefc72cf84c
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panic.rs:388
  32:     0x55acdeae6fcf - std::thread::Builder::spawn_unchecked::{{closure}}::h6e0ecf038c3a03fe
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/thread/mod.rs:468
  33:     0x55acdeae72d7 - <F as alloc::boxed::FnBox<A>>::call_box::ha04220c2aedddf29
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/boxed.rs:749
  34:     0x55acdf823a2d - std::sys::unix::thread::Thread::new::thread_start::h7fc2d82a3b91bc88
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/boxed.rs:759
                               at src/libstd/sys_common/thread.rs:14
                               at src/libstd/sys/unix/thread.rs:81
  35:     0x7f2065be5ef6 - start_thread
  36:     0x7f206590522e - __GI___clone
  37:                0x0 - <unknown>

Possibly fixed already, needs testing.

server: support distributed clusters

The system has been designed with an eye towards supporting distributed operation—e.g., metastore stores data in ZooKeeper, and not a local SQLite database—but the plumbing for multiple nodes isn't quite hooked up. Basically, there's no way to specify on the materialize command line how many nodes you want, and what their addresses are.

slt: test failure unwrapping float as int

jamie@machine:~/materialize$ cat | RUST_BACKTRACE=full cargo run --bin=sqllogictest - -vv
statement ok
CREATE TABLE tab0(col0 INTEGER, col1 INTEGER, col2 INTEGER)

statement ok
CREATE TABLE tab1(col0 INTEGER, col1 INTEGER, col2 INTEGER)

statement ok
CREATE TABLE tab2(col0 INTEGER, col1 INTEGER, col2 INTEGER)

statement ok
INSERT INTO tab0 VALUES(83,0,38)

statement ok
INSERT INTO tab0 VALUES(26,0,79)

statement ok
INSERT INTO tab0 VALUES(43,81,24)

query I rowsort
SELECT DISTINCT - - col0 + - - col2 col0 FROM tab0 AS cor0 GROUP BY col2, col0 HAVING NOT - AVG ( + col1 ) IS NOT NULL
----
^D

thread 'thread 'worker thread 2worker thread 0' panicked at '' panicked at 'Datum::unwrap_int64 called on Float64(OrderedFloat(0.0))Datum::unwrap_int64 called on Float64(OrderedFloat(81.0))', ', src/materialize/repr.rssrc/materialize/repr.rs::8181::1818

stack backtrace:
thread 'worker thread 1' panicked at 'Datum::unwrap_int64 called on Float64(OrderedFloat(0.0))', src/materialize/repr.rs:81:18
   0:     0x560ddbac4103 - std::sys::unix::backtrace::tracing::imp::unwind_backtrace::hc7b76c57e25c7d91
                               at src/libstd/sys/unix/backtrace/tracing/gcc_s.rs:39
   1:     0x560ddbabed5b - std::sys_common::backtrace::_print::h203699a8f1b8b948
                               at src/libstd/sys_common/backtrace.rs:70
   2:     0x560ddbac1e66 - std::panicking::default_hook::{{closure}}::h74fcb67c61dc8b92
                               at src/libstd/sys_common/backtrace.rs:58
                               at src/libstd/panicking.rs:200
   3:     0x560ddbac1be4 - std::panicking::default_hook::hb2e06978a2102930
                               at src/libstd/panicking.rs:215
   4:     0x560ddbac256f - std::panicking::rust_panic_with_hook::h4ac0b487eb7e54da
                               at src/libstd/panicking.rs:478
   5:     0x560ddbac20f1 - std::panicking::continue_panic_fmt::h23ca3ed77b25f368
                               at src/libstd/panicking.rs:385
   6:     0x560ddbac203e - std::panicking::begin_panic_fmt::he14137966dec33cc
                               at src/libstd/panicking.rs:340
   7:     0x560ddad7ffa7 - materialize::repr::Datum::unwrap_int64::h2b6534eeb9a58cd3
                               at src/materialize/repr.rs:81
   8:     0x560ddac3ee60 - materialize::dataflow::func::neg_int64::h0ba04d8a0f387e89
                               at src/materialize/dataflow/func.rs:197
   9:     0x560ddaf8ec02 - materialize::dataflow::render::eval_expr::h7328560a17a0733e
                               at src/materialize/dataflow/render.rs:260
  10:     0x560ddaf8ebab - materialize::dataflow::render::eval_expr::h7328560a17a0733e
                               at src/materialize/dataflow/render.rs:259
  11:     0x560ddaf8ebab - materialize::dataflow::render::eval_expr::h7328560a17a0733e
                               at src/materialize/dataflow/render.rs:259
  12:     0x560ddaf8ebab - materialize::dataflow::render::eval_expr::h7328560a17a0733e
                               at src/materialize/dataflow/render.rs:259
  13:     0x560ddaf7df01 - materialize::dataflow::render::build_plan::{{closure}}::he8a195e76bf33a29
                               at src/materialize/dataflow/render.rs:106
  14:     0x560ddad0854d - <differential_dataflow::collection::Collection<G, D, R>>::filter::{{closure}}::h32b4b24bf54d24af
                               at /home/jamie/.cargo/git/checkouts/differential-dataflow-d065d23d797aa027/3773f17/src/collection.rs:167
  15:     0x560ddadaea6a - <timely::dataflow::stream::Stream<G, D> as timely::dataflow::operators::filter::Filter<D>>::filter::{{closure}}::{{closure}}::{{closure}}::{{closure}}::h728b2a7e40db965b
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/filter.rs:31
  16:     0x560ddab8bbfa - <alloc::vec::Vec<T>>::retain::{{closure}}::h513af9f10c208c5e
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/vec.rs:954
  17:     0x560ddab5e6a0 - <alloc::vec::DrainFilter<'_, T, F> as core::iter::traits::iterator::Iterator>::next::h4e6c14627243f903
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/vec.rs:2675
  18:     0x560ddabbc45e - <&mut I as core::iter::traits::iterator::Iterator>::next::h12b2f31f9e4e4f2f
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/iter/traits/iterator.rs:2730
  19:     0x560ddab9e8de - core::iter::traits::iterator::Iterator::try_fold::h7ca163ec09b21faa
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/iter/traits/iterator.rs:1571
  20:     0x560ddab96f01 - core::iter::traits::iterator::Iterator::fold::h28d728a035079bd0
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/iter/traits/iterator.rs:1684
  21:     0x560ddab98ab3 - core::iter::traits::iterator::Iterator::for_each::h1aa0ef23958bed8f
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/iter/traits/iterator.rs:604
  22:     0x560ddabe01ce - <alloc::vec::DrainFilter<'_, T, F> as core::ops::drop::Drop>::drop::hfabd4db84b17481d
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/vec.rs:2702
  23:     0x560ddac8dcde - core::ptr::real_drop_in_place::he2e72b0b2553f6c9
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/ptr.rs:193
  24:     0x560ddab8b986 - <alloc::vec::Vec<T>>::retain::he37ccda9cb53fdc0
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/vec.rs:954
  25:     0x560ddadae9a4 - <timely::dataflow::stream::Stream<G, D> as timely::dataflow::operators::filter::Filter<D>>::filter::{{closure}}::{{closure}}::{{closure}}::h3c1d5b4357d5550e
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/filter.rs:31
  26:     0x560ddad6b8c6 - <timely::dataflow::operators::generic::handles::InputHandle<T, D, P>>::for_each::h0bd3fe734fd754e1
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/generic/handles.rs:82
  27:     0x560ddadae934 - <timely::dataflow::stream::Stream<G, D> as timely::dataflow::operators::filter::Filter<D>>::filter::{{closure}}::{{closure}}::h9df491dcdace634b
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/filter.rs:29
  28:     0x560ddac5b57c - <timely::dataflow::stream::Stream<G, D1> as timely::dataflow::operators::generic::operator::Operator<G, D1>>::unary::{{closure}}::{{closure}}::hc7afecfeea477078
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/generic/operator.rs:402
  29:     0x560ddae0df89 - <timely::dataflow::operators::generic::builder_rc::OperatorBuilder<G>>::build::{{closure}}::h0276b991e8453537
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/generic/builder_rc.rs:136
  30:     0x560ddacae022 - <timely::dataflow::operators::generic::builder_raw::OperatorCore<T, PEP, PIP> as timely::scheduling::Schedule>::schedule::h962a2b586bd2d33a
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/generic/builder_raw.rs:219
  31:     0x560ddaf5781e - <timely::progress::subgraph::PerOperatorState<T>>::schedule::hebfc4f0fde4b8d41
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:646
  32:     0x560ddaf58157 - <timely::progress::subgraph::Subgraph<TOuter, TInner>>::activate_child::h9ae5d7849b54e217
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:329
  33:     0x560ddaf5330b - <timely::progress::subgraph::Subgraph<TOuter, TInner> as timely::scheduling::Schedule>::schedule::hcacdec1bef1082b3
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:295
  34:     0x560ddb749ed4 - timely::worker::Wrapper::step::{{closure}}::h971ad292bc084c03
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:489
  35:     0x560ddb748dcd - <core::option::Option<T>>::map::hcd877f91cd6d63f8
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/option.rs:414
  36:     0x560ddb749cf2 - timely::worker::Wrapper::step::h079de6881be60958
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:489
  37:     0x560ddac9cbc0 - <timely::worker::Worker<A>>::step_or_park::ha3cf7ccb3a665639
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:227
  38:     0x560ddac9f8c2 - <timely::worker::Worker<A>>::step::hb691f6f1f3a3f589
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:144
  39:     0x560ddac41a30 - <materialize::dataflow::server::Worker<'w, A>>::run::hda8e00cce02a56ae
                               at src/materialize/dataflow/server.rs:129
  40:     0x560ddac413aa - materialize::dataflow::server::serve::{{closure}}::h6bc7e17ac31d0bce
                               at src/materialize/dataflow/server.rs:47
  41:     0x560ddaf91840 - timely::execute::execute::{{closure}}::h1f93346fe3e1bc0d
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/execute.rs:201
  42:     0x560ddac5f709 - timely_communication::initialize::initialize_from::{{closure}}::h1252acab1e6178d6
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/communication/src/initialize.rs:269
  43:     0x560ddad8cfe3 - std::sys_common::backtrace::__rust_begin_short_backtrace::h12d7320851696d2d
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/sys_common/backtrace.rs:135
  44:     0x560ddad899e5 - std::thread::Builder::spawn_unchecked::{{closure}}::{{closure}}::hdb80b6cb28504035
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/thread/mod.rs:469
  45:     0x560ddafbe835 - <std::panic::AssertUnwindSafe<F> as core::ops::function::FnOnce<()>>::call_once::h6a208203b39e18a3
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panic.rs:309
  46:     0x560ddaf97a39 - std::panicking::try::do_call::hc0a910baaa414789
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:297
  47:     0x560ddbac7729 - __rust_maybe_catch_panic
                               at src/libpanic_unwind/lib.rs:87
  48:     0x560ddaf9791f - std::panicking::try::h2f37f341d87162d4
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:276
  49:     0x560ddafbe875 - std::panic::catch_unwind::hc732feefc72cf84c
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panic.rs:388
  50:     0x560ddad897af - std::thread::Builder::spawn_unchecked::{{closure}}::h6e0ecf038c3a03fe
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/thread/mod.rs:468
  51:     0x560ddad89ab7 - <F as alloc::boxed::FnBox<A>>::call_box::ha04220c2aedddf29
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/boxed.rs:749
  52:     0x560ddbac6acd - std::sys::unix::thread::Thread::new::thread_start::h7fc2d82a3b91bc88
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/boxed.rs:759
                               at src/libstd/sys_common/thread.rs:14
                               at src/libstd/sys/unix/thread.rs:81
  53:     0x7fcf1c335ef6 - start_thread
  54:     0x7fcf1c05522e - __GI___clone
  55:                0x0 - <unknown>

pgwire: support extended query protocol

The current pgwire protocol supports only the basic query protocol. This is sufficient to support psql, but won't be sufficient to support arbitrary $LANG drivers.

Dependabot needs permission to see sqlparser

Dependabot couldn't resolve your project's dependencies as it couldn't access sqlparser.

You can grant Dependabot access to sqlparser by adding the repository here. We'll automatically close this issue and kick off another update run once permission is granted.

You can mention @dependabot in the comments below to contact the Dependabot team.

server: improve RPC framework

Rust has no nice RPC frameworks:

  • GRPC is really heavyweight and brings with it a dependency on protobuf. Plus the Rust implementation isn't official; it's maintained by the TiDB guys and therefore doesn't inspire the most confidence.

  • Tarpc requires the nightly compiler, which is a dependency we decided we didn't want to take (#10).

  • capnproto-rust bundles an RPC framework, and it looks semi-official, but... looks lightly utilized and unidiomatic.

So instead our "RPC"s use HTTP. This is actually likely to be fine—HTTP2 is really efficient—but the current implementation is really stupid, and opens a new HTTP connection for every RPC. This needs to be adjusted to use a client pool.

infra: upgrade to async/await

So far I've been adhering to the "thou shalt only use stable" commandment, promulgated by @frankmcsherry, but writing futures code is sufficiently painful that it may be worth using the nightly compiler to get access to async/await.

The right way to evaluate this is probably to pick a small but particularly hairy swath of stable futures code, rewrite it using async/await, and then evaluating how much better it gets.

slt: test failure unwrapping int as float

jamie@machine:~/materialize$ cat | RUST_BACKTRACE=full cargo run --bin=sqllogictest - -vv
statement ok
CREATE TABLE tab0(pk INTEGER PRIMARY KEY, col0 INTEGER, col1 FLOAT, col2 TEXT, col3 INTEGER, col4 FLOAT, col5 TEXT)

statement ok
INSERT INTO tab0 VALUES(0,661,8490.41,'vdluk',6313,3096.77,'yfnkt')

statement ok
INSERT INTO tab0 VALUES(1,1111,7339.24,'htbmw',4665,1640.91,'zpibt')

statement ok
INSERT INTO tab0 VALUES(2,2676,9411.62,'zlnli',5870,224.97,'bfbng')

statement ok
INSERT INTO tab0 VALUES(3,204,9224.50,'hqctd',8301,2791.3,'aahvo')

statement ok
INSERT INTO tab0 VALUES(4,2155,458.4,'uzgyn',7070,3070.26,'noujm')

statement ok
INSERT INTO tab0 VALUES(5,6855,7203.31,'obrlm',275,5824.51,'envgf')

statement ok
INSERT INTO tab0 VALUES(6,8740,5208.84,'zxpnf',5255,8102.95,'qndbm')

statement ok
INSERT INTO tab0 VALUES(7,4612,3134.34,'gsuqw',4837,487.69,'snzov')

statement ok
INSERT INTO tab0 VALUES(8,9884,3801.24,'ensdb',3584,8348.39,'orofo')

statement ok
INSERT INTO tab0 VALUES(9,4834,5439.34,'mqqzq',4049,3090.91,'utvua')

statement ok
INSERT INTO tab0 VALUES(10,853,2938.46,'wnwec',3025,3402.94,'mxvzn')

statement ok
INSERT INTO tab0 VALUES(11,6695,9099.5,'edqpn',7789,6508.57,'mfrgt')

statement ok
INSERT INTO tab0 VALUES(12,4726,5883.79,'numnv',2360,9418.0,'lhuco')

statement ok
INSERT INTO tab0 VALUES(13,7178,5661.25,'itggs',3743,177.9,'wcxyk')

statement ok
INSERT INTO tab0 VALUES(14,2394,2463.52,'mqerb',8184,9114.2,'jhksf')

statement ok
INSERT INTO tab0 VALUES(15,1081,3232.78,'qiecu',7413,94.92,'tbejt')

statement ok
INSERT INTO tab0 VALUES(16,2560,9067.98,'amsqc',4902,1769.46,'cpwpo')

statement ok
INSERT INTO tab0 VALUES(17,1014,6482.46,'vcfyq',2111,8822.53,'zwdmb')

statement ok
INSERT INTO tab0 VALUES(18,1741,8647.55,'gdzxp',4735,3225.81,'ldnbx')

statement ok
INSERT INTO tab0 VALUES(19,7400,6645.20,'ojkct',8962,5483.45,'gquxx')

statement ok
INSERT INTO tab0 VALUES(20,4265,717.61,'hrbor',7714,5888.61,'zfwwd')

statement ok
INSERT INTO tab0 VALUES(21,8473,1516.89,'hxviv',6905,5764.21,'evcrg')

statement ok
INSERT INTO tab0 VALUES(22,2273,1353.60,'alddu',1684,8724.42,'killl')

statement ok
INSERT INTO tab0 VALUES(23,280,5452.48,'mswad',2690,1913.39,'ibjko')

statement ok
INSERT INTO tab0 VALUES(24,5720,3195.97,'kaqvv',2300,7263.82,'bidpk')

statement ok
INSERT INTO tab0 VALUES(25,6682,9375.94,'lzxaf',880,2741.60,'chxhr')

statement ok
INSERT INTO tab0 VALUES(26,2370,5444.22,'fibgc',3033,3598.44,'livtd')

statement ok
INSERT INTO tab0 VALUES(27,9351,767.28,'kiovb',3851,3622.47,'vvgxu')

statement ok
INSERT INTO tab0 VALUES(28,7803,3865.85,'bpcoi',5164,6384.92,'trrwz')

statement ok
INSERT INTO tab0 VALUES(29,7771,3673.54,'noqhd',1790,6492.68,'scbno')

statement ok
INSERT INTO tab0 VALUES(30,5410,784.43,'qfpmq',7787,838.97,'gbxjw')

statement ok
INSERT INTO tab0 VALUES(31,9240,8702.78,'vtayf',2698,9849.93,'zagwa')

statement ok
INSERT INTO tab0 VALUES(32,6394,5090.54,'japhn',7809,8739.29,'zkyhz')

statement ok
INSERT INTO tab0 VALUES(33,8869,3023.9,'hivtn',2389,8754.94,'rosfp')

statement ok
INSERT INTO tab0 VALUES(34,9772,2236.79,'rzetw',9949,8429.62,'azjez')

statement ok
INSERT INTO tab0 VALUES(35,9578,371.63,'tygit',4487,4832.49,'serif')

statement ok
INSERT INTO tab0 VALUES(36,2372,4234.83,'xltdc',6739,4815.76,'ffwbx')

statement ok
INSERT INTO tab0 VALUES(37,2103,1040.53,'zpsrv',431,6267.36,'zllob')

statement ok
INSERT INTO tab0 VALUES(38,8835,9662.34,'hzshw',8524,9362.50,'kzkoh')

statement ok
INSERT INTO tab0 VALUES(39,4992,1887.0,'rlulh',9470,7809.37,'gjluw')

statement ok
INSERT INTO tab0 VALUES(40,1615,3659.1,'wmspk',3459,101.70,'jhyuy')

statement ok
INSERT INTO tab0 VALUES(41,4605,6334.26,'kbdty',8669,768.40,'fchjq')

statement ok
INSERT INTO tab0 VALUES(42,978,7927.61,'tcoze',7515,1624.21,'rntke')

statement ok
INSERT INTO tab0 VALUES(43,5113,4364.56,'pfqga',2902,3018.32,'ldugx')

statement ok
INSERT INTO tab0 VALUES(44,5003,730.88,'wgdve',4472,5085.95,'xgdpq')

statement ok
INSERT INTO tab0 VALUES(45,8502,2521.19,'tzhdg',4858,8995.5,'khwxo')

statement ok
INSERT INTO tab0 VALUES(46,9580,6894.64,'arpdu',5926,8203.85,'lmpcl')

statement ok
INSERT INTO tab0 VALUES(47,4569,3914.48,'bshdq',5036,9698.54,'ppogw')

statement ok
INSERT INTO tab0 VALUES(48,8413,7809.72,'zjejt',1283,5256.88,'sjvdd')

statement ok
INSERT INTO tab0 VALUES(49,923,8214.76,'uczor',1751,2687.78,'fjfwn')

statement ok
INSERT INTO tab0 VALUES(50,3019,1426.98,'kphoh',7762,7087.2,'yjqnn')

statement ok
INSERT INTO tab0 VALUES(51,2987,1652.41,'aaucf',1127,2087.78,'nfdtr')

statement ok
INSERT INTO tab0 VALUES(52,2726,2259.32,'wxlhx',394,1101.32,'hgany')

statement ok
INSERT INTO tab0 VALUES(53,3997,7025.86,'trlep',7102,8583.59,'qbyou')

statement ok
INSERT INTO tab0 VALUES(54,4675,1386.67,'vflpq',2025,1266.84,'vbcae')

statement ok
INSERT INTO tab0 VALUES(55,4999,3690.54,'yorpm',1428,9423.27,'ociiu')

statement ok
INSERT INTO tab0 VALUES(56,3820,6445.98,'rocdx',3311,8110.65,'odekg')

statement ok
INSERT INTO tab0 VALUES(57,4854,7568.92,'llmtz',3666,8419.93,'fudsb')

statement ok
INSERT INTO tab0 VALUES(58,2927,1637.25,'rkmbl',2721,8727.89,'iarot')

statement ok
INSERT INTO tab0 VALUES(59,232,1141.51,'yjyno',1193,5945.72,'mvlxg')

statement ok
INSERT INTO tab0 VALUES(60,2574,7728.49,'akuam',6536,6770.71,'yrgyq')

statement ok
INSERT INTO tab0 VALUES(61,5621,3456.85,'bqike',9196,8936.80,'bgjqn')

statement ok
INSERT INTO tab0 VALUES(62,1153,2044.76,'uoigr',8852,2458.29,'depme')

statement ok
INSERT INTO tab0 VALUES(63,3316,119.51,'kxcjd',676,891.38,'qkhcl')

statement ok
INSERT INTO tab0 VALUES(64,4867,8473.14,'zcqma',1575,2570.85,'karsd')

statement ok
INSERT INTO tab0 VALUES(65,3428,3302.7,'kauxd',6446,1870.67,'svsby')

statement ok
INSERT INTO tab0 VALUES(66,7733,7468.74,'oojax',4423,448.64,'jynrr')

statement ok
INSERT INTO tab0 VALUES(67,3879,1738.18,'keebi',5131,8498.96,'vkhvj')

statement ok
INSERT INTO tab0 VALUES(68,1578,6042.71,'gshqf',4930,5998.93,'jxkxk')

statement ok
INSERT INTO tab0 VALUES(69,404,721.50,'svdkg',3170,3961.52,'hhtfn')

statement ok
INSERT INTO tab0 VALUES(70,6044,7774.23,'wslre',8438,5686.4,'ugrnf')

statement ok
INSERT INTO tab0 VALUES(71,6550,2110.71,'lrrbw',4812,8006.70,'jxeqo')

statement ok
INSERT INTO tab0 VALUES(72,8585,2800.51,'yxxak',8105,2542.30,'unxdm')

statement ok
INSERT INTO tab0 VALUES(73,3677,4324.55,'subql',2121,6946.83,'hzpxz')

statement ok
INSERT INTO tab0 VALUES(74,4467,7824.78,'bxzbz',4901,6159.9,'oouwb')

statement ok
INSERT INTO tab0 VALUES(75,958,2348.76,'mlqhu',4973,3653.32,'blinw')

statement ok
INSERT INTO tab0 VALUES(76,6768,6109.31,'rtdsj',7606,2690.82,'cmenb')

statement ok
INSERT INTO tab0 VALUES(77,5378,7727.13,'rextc',1055,7564.58,'uqtko')

statement ok
INSERT INTO tab0 VALUES(78,6380,7482.42,'boxlk',2814,4855.91,'jtaeb')

statement ok
INSERT INTO tab0 VALUES(79,1733,8567.49,'bxofw',5867,4677.6,'gcoaw')

statement ok
INSERT INTO tab0 VALUES(80,2006,8332.42,'oxchj',4934,1932.83,'fbprd')

statement ok
INSERT INTO tab0 VALUES(81,2633,9540.67,'islmd',2724,7554.24,'jqjkz')

statement ok
INSERT INTO tab0 VALUES(82,162,1611.69,'gnosy',6184,9763.9,'kidhp')

statement ok
INSERT INTO tab0 VALUES(83,8503,1879.56,'dbmuf',2750,5316.64,'tkecm')

statement ok
INSERT INTO tab0 VALUES(84,5177,6834.62,'flkve',6595,754.20,'pvdvi')

statement ok
INSERT INTO tab0 VALUES(85,7573,3378.8,'zqway',4883,3652.79,'btafk')

statement ok
INSERT INTO tab0 VALUES(86,9192,75.64,'lcweq',4013,3872.73,'jdhpn')

statement ok
INSERT INTO tab0 VALUES(87,5895,8397.16,'glxlm',5873,4809.29,'vkbie')

statement ok
INSERT INTO tab0 VALUES(88,2160,3976.21,'ubhwd',4093,4645.28,'ebhnu')

statement ok
INSERT INTO tab0 VALUES(89,7327,7855.77,'itaam',285,7598.72,'mhtpq')

statement ok
INSERT INTO tab0 VALUES(90,1848,7737.81,'bgeqz',304,5169.45,'lrezd')

statement ok
INSERT INTO tab0 VALUES(91,3541,9235.93,'kbwdw',493,7393.73,'jrorz')

statement ok
INSERT INTO tab0 VALUES(92,3835,7123.97,'yvhoy',3930,4095.91,'kkaxb')

statement ok
INSERT INTO tab0 VALUES(93,4022,6583.66,'qfdtc',6684,6130.0,'rcduv')

statement ok
INSERT INTO tab0 VALUES(94,1510,4460.24,'zhlkc',4063,6642.34,'hwbvi')

statement ok
INSERT INTO tab0 VALUES(95,6069,507.5,'civiz',6861,2308.21,'phkqh')

statement ok
INSERT INTO tab0 VALUES(96,6079,8638.20,'tjqjy',7611,4590.45,'fasrg')

statement ok
INSERT INTO tab0 VALUES(97,4789,8176.17,'ruzws',6835,987.10,'fkiot')

statement ok
INSERT INTO tab0 VALUES(98,6557,5107.46,'pcahl',9293,6281.0,'ecpzk')

statement ok
INSERT INTO tab0 VALUES(99,3107,3325.18,'hhwyz',233,3791.44,'kuahc')

statement ok
INSERT INTO tab0 VALUES(100,9773,5884.68,'vuxmx',1641,6620.77,'bqjwy')

statement ok
INSERT INTO tab0 VALUES(101,1756,6247.22,'ruwpu',8491,8906.2,'dbvep')

statement ok
INSERT INTO tab0 VALUES(102,368,9542.53,'sbusc',4998,7073.55,'sjzxg')

statement ok
INSERT INTO tab0 VALUES(103,9284,3802.49,'nmmlo',862,1080.66,'cfebw')

statement ok
INSERT INTO tab0 VALUES(104,795,1223.71,'jshjd',7522,5671.20,'vkmfg')

statement ok
INSERT INTO tab0 VALUES(105,2052,9410.21,'kwmpe',4407,1625.22,'zktjb')

statement ok
INSERT INTO tab0 VALUES(106,1320,2628.98,'uafjo',5182,7007.58,'cbzle')

statement ok
INSERT INTO tab0 VALUES(107,4230,3052.64,'uvstv',4002,8415.59,'ldgaq')

statement ok
INSERT INTO tab0 VALUES(108,23,2020.9,'yhkdj',1759,2234.60,'azxut')

statement ok
INSERT INTO tab0 VALUES(109,7805,1276.7,'cjdch',2335,5885.68,'xerll')

statement ok
INSERT INTO tab0 VALUES(110,2795,4227.85,'ricrw',8848,9071.20,'bctzb')

statement ok
INSERT INTO tab0 VALUES(111,2575,7189.81,'qvain',9907,5033.46,'upbqu')

statement ok
INSERT INTO tab0 VALUES(112,3338,638.9,'sdkkx',5856,1796.34,'opegp')

statement ok
INSERT INTO tab0 VALUES(113,6145,461.21,'oxqtb',5030,3421.89,'ktrdj')

statement ok
INSERT INTO tab0 VALUES(114,3169,9205.68,'sywkp',1,1252.1,'uzakf')

statement ok
INSERT INTO tab0 VALUES(115,3943,7325.65,'znldt',3830,5172.0,'yfxyi')

statement ok
INSERT INTO tab0 VALUES(116,1617,4440.51,'hiyen',6316,4845.25,'wyfth')

statement ok
INSERT INTO tab0 VALUES(117,9201,577.54,'yveuw',246,972.82,'owird')

statement ok
INSERT INTO tab0 VALUES(118,5822,7423.52,'vdkws',7645,6541.11,'yozcn')

statement ok
INSERT INTO tab0 VALUES(119,2968,9574.33,'jjopd',6027,8071.44,'fyqlt')

statement ok
INSERT INTO tab0 VALUES(120,739,5790.65,'undbn',4644,6870.69,'xdnwa')

statement ok
INSERT INTO tab0 VALUES(121,4102,238.49,'cwoyq',737,278.96,'xwswl')

statement ok
INSERT INTO tab0 VALUES(122,9134,3745.90,'jvdyu',5783,2924.12,'shiyh')

statement ok
INSERT INTO tab0 VALUES(123,9276,5050.53,'vjhrx',1750,7968.51,'lkoez')

statement ok
INSERT INTO tab0 VALUES(124,1584,2050.29,'vuqds',7344,8358.53,'cxzuh')

statement ok
INSERT INTO tab0 VALUES(125,2976,3056.97,'fgbiu',4256,8076.27,'opkzm')

statement ok
INSERT INTO tab0 VALUES(126,6482,8034.87,'gjyxw',3420,3390.12,'udhij')

statement ok
INSERT INTO tab0 VALUES(127,7498,9864.40,'vafvr',1254,4101.43,'yropn')

statement ok
INSERT INTO tab0 VALUES(128,31,5585.72,'pvozi',5918,4799.10,'kkcdh')

statement ok
INSERT INTO tab0 VALUES(129,2952,7645.40,'brksg',3558,6105.23,'bhnta')

statement ok
INSERT INTO tab0 VALUES(130,6246,5154.97,'uqepl',4559,5927.73,'kmned')

statement ok
INSERT INTO tab0 VALUES(131,7261,5131.60,'jijzq',9793,709.94,'mgyti')

statement ok
INSERT INTO tab0 VALUES(132,6958,3756.74,'qrppd',6872,6680.27,'fjvao')

statement ok
INSERT INTO tab0 VALUES(133,6160,7618.68,'xmygj',6396,1102.84,'mupkf')

statement ok
INSERT INTO tab0 VALUES(134,1806,4967.23,'xauip',8204,3524.5,'eiiul')

statement ok
INSERT INTO tab0 VALUES(135,3195,6964.32,'yufpk',7619,7030.26,'kgndw')

statement ok
INSERT INTO tab0 VALUES(136,8244,8928.64,'dawqm',1709,9490.39,'nxpce')

statement ok
INSERT INTO tab0 VALUES(137,9728,1589.24,'mijrb',5525,4806.55,'jjttw')

statement ok
INSERT INTO tab0 VALUES(138,1538,2338.20,'yidyv',8359,4169.68,'jaico')

statement ok
INSERT INTO tab0 VALUES(139,4929,6476.21,'ojupd',9989,7302.12,'neqtd')

statement ok
INSERT INTO tab0 VALUES(140,4768,8464.20,'syaed',6933,3717.31,'stjmu')

statement ok
INSERT INTO tab0 VALUES(141,5784,3171.0,'bmzdu',8698,3955.36,'uxoad')

statement ok
INSERT INTO tab0 VALUES(142,8823,7130.93,'vzdtf',4028,850.72,'potnk')

statement ok
INSERT INTO tab0 VALUES(143,7091,9781.44,'topmo',7154,764.75,'dccdp')

statement ok
INSERT INTO tab0 VALUES(144,2409,4726.72,'mbfsi',4048,4848.38,'ubpwm')

statement ok
INSERT INTO tab0 VALUES(145,1459,977.37,'jsnjk',8670,289.75,'iqbat')

statement ok
INSERT INTO tab0 VALUES(146,2040,4205.95,'wwdof',9214,1555.79,'cihwi')

statement ok
INSERT INTO tab0 VALUES(147,9311,7992.67,'nksdt',4369,1653.88,'cphgf')

statement ok
INSERT INTO tab0 VALUES(148,2046,5079.99,'bmajx',9558,3374.46,'eogiu')

statement ok
INSERT INTO tab0 VALUES(149,8881,8620.26,'rghoj',3193,544.68,'bkwyz')

statement ok
INSERT INTO tab0 VALUES(150,4567,3427.29,'ahcsx',2923,9273.97,'oheod')

statement ok
INSERT INTO tab0 VALUES(151,6217,7221.7,'iipal',1889,9209.73,'sszrl')

statement ok
INSERT INTO tab0 VALUES(152,9558,1886.31,'pwrsd',4967,4782.91,'tjunx')

statement ok
INSERT INTO tab0 VALUES(153,5179,4714.9,'twako',5022,6459.64,'mnapm')

statement ok
INSERT INTO tab0 VALUES(154,4783,9659.52,'peiav',9158,1992.89,'ewcne')

statement ok
INSERT INTO tab0 VALUES(155,7438,8583.31,'ejgds',4966,7669.4,'tankh')

statement ok
INSERT INTO tab0 VALUES(156,5823,2510.45,'sitvt',6778,9559.57,'beeoj')

statement ok
INSERT INTO tab0 VALUES(157,2481,2873.78,'utadj',1911,4773.61,'ttegm')

statement ok
INSERT INTO tab0 VALUES(158,6068,2765.65,'nvuua',6922,2701.63,'pzgci')

statement ok
INSERT INTO tab0 VALUES(159,7604,8191.55,'qahwh',3855,6271.4,'pnuwe')

statement ok
INSERT INTO tab0 VALUES(160,6722,9314.71,'nllgc',6496,3196.79,'yfucg')

statement ok
INSERT INTO tab0 VALUES(161,7917,1708.74,'prbjs',6614,7223.6,'ilipt')

statement ok
INSERT INTO tab0 VALUES(162,6800,7608.25,'dfwxy',219,9387.81,'bbjdw')

statement ok
INSERT INTO tab0 VALUES(163,6343,7157.47,'phqfh',3811,9254.44,'zvaec')

statement ok
INSERT INTO tab0 VALUES(164,3131,9661.89,'rjjby',624,6178.45,'wcshf')

statement ok
INSERT INTO tab0 VALUES(165,3615,4834.89,'kwjja',6884,549.98,'jkdym')

statement ok
INSERT INTO tab0 VALUES(166,357,4050.33,'ipkpt',8054,5038.52,'fcqct')

statement ok
INSERT INTO tab0 VALUES(167,8283,1596.21,'rqvoj',5085,5856.49,'haduo')

statement ok
INSERT INTO tab0 VALUES(168,932,5915.28,'hsqwn',667,3996.87,'ubhth')

statement ok
INSERT INTO tab0 VALUES(169,6705,864.65,'zcsvf',5476,6488.84,'kxqbj')

statement ok
INSERT INTO tab0 VALUES(170,5120,6957.46,'nxvuw',4431,7644.38,'hluvf')

statement ok
INSERT INTO tab0 VALUES(171,3054,2620.46,'ubxxs',2933,316.91,'hoksm')

statement ok
INSERT INTO tab0 VALUES(172,1001,5440.66,'zqyts',3128,7137.93,'eunpd')

statement ok
INSERT INTO tab0 VALUES(173,9316,3410.93,'npjxq',144,1778.62,'muker')

statement ok
INSERT INTO tab0 VALUES(174,4466,6190.66,'mnbsk',8352,192.47,'ybzse')

statement ok
INSERT INTO tab0 VALUES(175,5286,6343.37,'mbjqz',7609,1871.92,'ipsmi')

statement ok
INSERT INTO tab0 VALUES(176,9219,597.32,'wrkfs',9511,1090.97,'xkvmu')

statement ok
INSERT INTO tab0 VALUES(177,8575,1088.15,'wddjg',3641,8158.80,'eiocc')

statement ok
INSERT INTO tab0 VALUES(178,6775,6612.49,'shlgv',3823,671.42,'jjyqy')

statement ok
INSERT INTO tab0 VALUES(179,4606,3878.62,'trfui',4366,4761.26,'fqicd')

statement ok
INSERT INTO tab0 VALUES(180,6464,8751.6,'pzrji',6320,6342.85,'lrgox')

statement ok
INSERT INTO tab0 VALUES(181,6613,7509.22,'exgil',3722,2366.36,'pjpax')

statement ok
INSERT INTO tab0 VALUES(182,5426,8088.97,'zowrf',9847,2069.31,'giwaj')

statement ok
INSERT INTO tab0 VALUES(183,5643,1338.37,'ciatj',3460,8222.16,'gsdai')

statement ok
INSERT INTO tab0 VALUES(184,8615,2265.18,'hexuw',5059,8383.91,'tsdkn')

statement ok
INSERT INTO tab0 VALUES(185,557,1286.55,'hrmnd',3139,6141.77,'fests')

statement ok
INSERT INTO tab0 VALUES(186,8797,5481.66,'tguwc',428,2374.76,'ujnrv')

statement ok
INSERT INTO tab0 VALUES(187,8027,9837.28,'natmh',140,6495.34,'hlkjw')

statement ok
INSERT INTO tab0 VALUES(188,6551,8502.45,'pksoc',2233,7699.2,'ffnfg')

statement ok
INSERT INTO tab0 VALUES(189,2538,6034.8,'ksddt',9649,6221.3,'epknv')

statement ok
INSERT INTO tab0 VALUES(190,268,8653.59,'wracm',4569,4575.87,'skaac')

statement ok
INSERT INTO tab0 VALUES(191,6387,5487.52,'mhzlu',4666,8816.3,'owaof')

statement ok
INSERT INTO tab0 VALUES(192,6152,9825.17,'phgfh',9344,9781.53,'vjvpe')

statement ok
INSERT INTO tab0 VALUES(193,9683,366.21,'gsxxd',6002,4343.59,'axtdz')

statement ok
INSERT INTO tab0 VALUES(194,8217,2469.30,'rtafy',6275,6660.24,'dwczq')

statement ok
INSERT INTO tab0 VALUES(195,350,9403.76,'bdroz',2286,9424.96,'zohpk')

statement ok
INSERT INTO tab0 VALUES(196,8294,3598.13,'yxdwg',2897,5115.29,'awasy')

statement ok
INSERT INTO tab0 VALUES(197,4123,7894.85,'ezdzu',9955,7247.45,'uykui')

statement ok
INSERT INTO tab0 VALUES(198,6024,6791.84,'cvqnm',4551,1616.98,'urzxe')

statement ok
INSERT INTO tab0 VALUES(199,4548,7697.16,'saihn',3259,7558.36,'unbbw')

statement ok
INSERT INTO tab0 VALUES(200,8859,2185.63,'txpbg',4395,9191.65,'swodq')

statement ok
INSERT INTO tab0 VALUES(201,2985,7233.43,'vfncn',4261,6670.7,'czjhq')

statement ok
INSERT INTO tab0 VALUES(202,2066,6483.74,'mthwq',3398,7433.68,'whkqc')

statement ok
INSERT INTO tab0 VALUES(203,8607,1483.40,'lxaoo',2836,7455.96,'uiqyo')

statement ok
INSERT INTO tab0 VALUES(204,8724,2947.45,'rjewc',343,2496.47,'jcjbu')

statement ok
INSERT INTO tab0 VALUES(205,8398,1994.85,'nrjlv',6673,9351.45,'yvgxc')

statement ok
INSERT INTO tab0 VALUES(206,9724,1406.5,'syvol',1822,7098.46,'bqmqh')

statement ok
INSERT INTO tab0 VALUES(207,7157,6874.62,'ujgwv',1272,8355.99,'jbwsz')

statement ok
INSERT INTO tab0 VALUES(208,3769,8249.89,'vybxa',2244,520.10,'wrxdo')

statement ok
INSERT INTO tab0 VALUES(209,7492,793.55,'zcmro',9946,7028.5,'yvdil')

statement ok
INSERT INTO tab0 VALUES(210,7375,6427.31,'efskl',3827,8601.91,'ndztb')

statement ok
INSERT INTO tab0 VALUES(211,9827,4984.51,'riadj',3821,4494.17,'igbjb')

statement ok
INSERT INTO tab0 VALUES(212,4872,5454.73,'vuxev',6603,3749.40,'siekn')

statement ok
INSERT INTO tab0 VALUES(213,2396,7241.42,'ltbad',1834,6440.86,'dtpxx')

statement ok
INSERT INTO tab0 VALUES(214,8399,1407.56,'tzutq',8416,4929.14,'zsckg')

statement ok
INSERT INTO tab0 VALUES(215,1246,1924.22,'riwty',736,5112.73,'xzbni')

statement ok
INSERT INTO tab0 VALUES(216,7912,6925.26,'vluhv',4923,7672.71,'vfppu')

statement ok
INSERT INTO tab0 VALUES(217,238,9953.89,'sixfl',5090,9324.36,'cxtif')

statement ok
INSERT INTO tab0 VALUES(218,1176,1210.33,'vcmlp',9593,4070.75,'qvokf')

statement ok
INSERT INTO tab0 VALUES(219,1043,8810.43,'tiutr',7272,4447.40,'atfbs')

statement ok
INSERT INTO tab0 VALUES(220,3770,26.29,'kclit',3708,2778.90,'racuy')

statement ok
INSERT INTO tab0 VALUES(221,3146,2541.88,'zdhky',7469,2419.52,'xccxn')

statement ok
INSERT INTO tab0 VALUES(222,9082,5699.68,'fxqtj',6362,9013.85,'dkneu')

statement ok
INSERT INTO tab0 VALUES(223,3846,1754.30,'lwioe',3682,1737.41,'xsios')

statement ok
INSERT INTO tab0 VALUES(224,8321,7119.9,'thzwy',4462,4555.31,'dyfqb')

statement ok
INSERT INTO tab0 VALUES(225,8252,3619.79,'bxzho',1724,7463.82,'cruiq')

statement ok
INSERT INTO tab0 VALUES(226,499,9737.14,'jhrnk',2727,7746.3,'rvsqj')

statement ok
INSERT INTO tab0 VALUES(227,9285,1869.49,'sxmcr',5220,1221.49,'kisag')

statement ok
INSERT INTO tab0 VALUES(228,2446,5568.47,'lfayd',3689,3491.80,'rqvht')

statement ok
INSERT INTO tab0 VALUES(229,7682,6799.55,'evfbp',5609,1195.24,'tkkyt')

statement ok
INSERT INTO tab0 VALUES(230,8804,3188.75,'drmbv',7692,3257.42,'ycjds')

statement ok
INSERT INTO tab0 VALUES(231,1989,5269.17,'cxtvu',2141,2136.87,'vtgzv')

statement ok
INSERT INTO tab0 VALUES(232,2025,7736.70,'pmser',2893,5075.72,'gikid')

statement ok
INSERT INTO tab0 VALUES(233,8566,697.58,'brulb',1878,5353.26,'kpymo')

statement ok
INSERT INTO tab0 VALUES(234,2377,915.5,'wvmfd',5261,3535.59,'epdog')

statement ok
INSERT INTO tab0 VALUES(235,1586,3067.1,'oyfrr',9697,8721.28,'algjl')

statement ok
INSERT INTO tab0 VALUES(236,7936,987.59,'xespg',8578,9794.93,'hqxyu')

statement ok
INSERT INTO tab0 VALUES(237,3033,8923.78,'zszqi',8286,6675.85,'ebcdi')

statement ok
INSERT INTO tab0 VALUES(238,4595,2781.9,'mfnhj',4438,1568.67,'ahghu')

statement ok
INSERT INTO tab0 VALUES(239,2705,1376.18,'qlida',8144,214.94,'gmnun')

statement ok
INSERT INTO tab0 VALUES(240,6058,7684.41,'kyduf',2248,3138.78,'suvuu')

statement ok
INSERT INTO tab0 VALUES(241,8211,93.57,'srnih',7585,6899.51,'pcwtt')

statement ok
INSERT INTO tab0 VALUES(242,3839,3649.41,'iqzca',9226,5344.25,'qplfr')

statement ok
INSERT INTO tab0 VALUES(243,4975,1309.33,'ikdcz',2734,854.58,'jmleb')

statement ok
INSERT INTO tab0 VALUES(244,7268,5190.98,'vxrvj',406,5593.12,'zmbxt')

statement ok
INSERT INTO tab0 VALUES(245,6231,7819.87,'mvssj',7530,2764.19,'snvoa')

statement ok
INSERT INTO tab0 VALUES(246,3487,3383.4,'invze',8630,7852.44,'ulwtt')

statement ok
INSERT INTO tab0 VALUES(247,3026,8981.64,'cjdbe',3723,9878.43,'rlucn')

statement ok
INSERT INTO tab0 VALUES(248,7448,5868.56,'fqkgx',4557,2650.31,'bvcat')

statement ok
INSERT INTO tab0 VALUES(249,4220,714.10,'awhmm',874,6008.39,'drdix')

statement ok
INSERT INTO tab0 VALUES(250,1918,6846.53,'urkuj',6654,7079.84,'qggsj')

statement ok
INSERT INTO tab0 VALUES(251,6485,5503.77,'zhjyu',6363,1526.22,'yyxnw')

statement ok
INSERT INTO tab0 VALUES(252,7475,7082.97,'ghafs',1204,4795.19,'ipxgl')

statement ok
INSERT INTO tab0 VALUES(253,4307,2392.17,'pbowa',5373,3369.8,'xhhhp')

statement ok
INSERT INTO tab0 VALUES(254,2511,3902.92,'qmpup',2301,6015.55,'vzuoj')

statement ok
INSERT INTO tab0 VALUES(255,9983,6990.88,'msifm',907,5035.18,'oaqyo')

statement ok
INSERT INTO tab0 VALUES(256,3229,1195.48,'anrfz',1851,7517.13,'twxew')

statement ok
INSERT INTO tab0 VALUES(257,3622,1466.50,'adgmx',3779,6585.75,'oslgu')

statement ok
INSERT INTO tab0 VALUES(258,3645,9413.75,'gwvvj',9575,2646.18,'qafdc')

statement ok
INSERT INTO tab0 VALUES(259,3147,4554.65,'oobuk',2612,4643.4,'igilq')

statement ok
INSERT INTO tab0 VALUES(260,5588,5570.89,'djyjo',3003,1305.98,'vlbzq')

statement ok
INSERT INTO tab0 VALUES(261,4293,1549.0,'ljrqn',3846,2395.36,'fucku')

statement ok
INSERT INTO tab0 VALUES(262,8699,3243.6,'ldzfu',7480,9962.83,'rjtvk')

statement ok
INSERT INTO tab0 VALUES(263,607,4309.24,'ujcdf',1730,9590.2,'nstmm')

statement ok
INSERT INTO tab0 VALUES(264,431,2570.48,'unafs',6163,1647.9,'kkvhv')

statement ok
INSERT INTO tab0 VALUES(265,3449,1255.10,'rmoxu',8329,2537.39,'ktkog')

statement ok
INSERT INTO tab0 VALUES(266,6051,255.39,'rekuj',20,5951.11,'dkanq')

statement ok
INSERT INTO tab0 VALUES(267,1991,7369.32,'dckeb',3516,4112.86,'yxhbc')

statement ok
INSERT INTO tab0 VALUES(268,8637,9560.6,'obowf',7834,9030.31,'ymegb')

statement ok
INSERT INTO tab0 VALUES(269,3219,4197.69,'ztepd',4981,3750.83,'elxyk')

statement ok
INSERT INTO tab0 VALUES(270,9081,312.79,'ujbwd',6199,3601.11,'tbhqr')

statement ok
INSERT INTO tab0 VALUES(271,1929,9846.4,'bhnkq',8223,7449.77,'zyusz')

statement ok
INSERT INTO tab0 VALUES(272,6086,8901.91,'pqrvj',9522,8944.0,'kvoll')

statement ok
INSERT INTO tab0 VALUES(273,733,8988.21,'cwhjm',6556,7771.16,'derjz')

statement ok
INSERT INTO tab0 VALUES(274,6167,8278.79,'eohhp',22,901.45,'stgzu')

statement ok
INSERT INTO tab0 VALUES(275,50,2854.9,'qxmtb',1105,3839.50,'jinlb')

statement ok
INSERT INTO tab0 VALUES(276,3655,7958.82,'nizgy',7730,1877.1,'gdxcn')

statement ok
INSERT INTO tab0 VALUES(277,8117,9241.57,'gcfje',808,1339.35,'itibf')

statement ok
INSERT INTO tab0 VALUES(278,689,4016.43,'reziv',491,839.22,'rmcaw')

statement ok
INSERT INTO tab0 VALUES(279,982,2506.61,'iluqa',2416,2520.27,'mdiuw')

statement ok
INSERT INTO tab0 VALUES(280,7224,9093.8,'ltrrf',3961,3316.69,'jprtl')

statement ok
INSERT INTO tab0 VALUES(281,2503,3570.18,'ffrje',492,6192.88,'imxtu')

statement ok
INSERT INTO tab0 VALUES(282,1025,2390.26,'rapwi',1610,1402.3,'vqopg')

statement ok
INSERT INTO tab0 VALUES(283,4354,9916.29,'dylar',7410,7730.81,'dzorh')

statement ok
INSERT INTO tab0 VALUES(284,7671,9671.49,'ackdy',4720,3751.90,'gxwpt')

statement ok
INSERT INTO tab0 VALUES(285,6019,5588.96,'glynw',3359,9069.73,'edztz')

statement ok
INSERT INTO tab0 VALUES(286,554,3262.32,'cmyfe',2385,5579.99,'wdvdu')

statement ok
INSERT INTO tab0 VALUES(287,4796,1196.28,'ovajq',6341,8708.34,'pvasw')

statement ok
INSERT INTO tab0 VALUES(288,877,2053.33,'jxggm',1511,8668.0,'ufxgp')

statement ok
INSERT INTO tab0 VALUES(289,5794,3908.33,'bmtoe',2166,1708.37,'lcrul')

statement ok
INSERT INTO tab0 VALUES(290,7655,1495.14,'gkmdv',6817,1359.69,'alqlv')

statement ok
INSERT INTO tab0 VALUES(291,1935,7459.25,'qdlqd',4463,9692.59,'rhcvn')

statement ok
INSERT INTO tab0 VALUES(292,7719,3604.40,'ciyco',837,8558.83,'lgtvo')

statement ok
INSERT INTO tab0 VALUES(293,3955,2144.20,'zqlqn',4667,846.80,'fcodk')

statement ok
INSERT INTO tab0 VALUES(294,5708,6954.78,'cfbak',3913,1730.23,'xldzf')

statement ok
INSERT INTO tab0 VALUES(295,402,4820.37,'viift',2028,9920.80,'tkopu')

statement ok
INSERT INTO tab0 VALUES(296,4076,6687.98,'iznfm',4202,8907.74,'edvas')

statement ok
INSERT INTO tab0 VALUES(297,7634,8908.99,'sqtmt',3612,8223.48,'alsoq')

statement ok
INSERT INTO tab0 VALUES(298,3090,4502.76,'xtcqj',7674,4681.23,'upbph')

statement ok
INSERT INTO tab0 VALUES(299,9824,4392.30,'dlqcs',4293,3630.70,'ptuzo')

statement ok
INSERT INTO tab0 VALUES(300,828,8664.88,'tzssa',8111,205.60,'jodxr')

statement ok
INSERT INTO tab0 VALUES(301,8957,7815.61,'mtyhi',6759,576.39,'yasij')

statement ok
INSERT INTO tab0 VALUES(302,3181,4001.14,'qkmju',7106,3198.79,'iwprx')

statement ok
INSERT INTO tab0 VALUES(303,4831,2287.22,'cxxtq',1510,61.50,'nioni')

statement ok
INSERT INTO tab0 VALUES(304,3853,1900.37,'kzpcl',4067,7939.76,'ztcdn')

statement ok
INSERT INTO tab0 VALUES(305,5055,7477.48,'cqusd',6860,9644.1,'pbpdg')

statement ok
INSERT INTO tab0 VALUES(306,4360,315.44,'qaoze',6212,5271.51,'rqmck')

statement ok
INSERT INTO tab0 VALUES(307,5052,9619.78,'rudbr',8934,3451.94,'jugfd')

statement ok
INSERT INTO tab0 VALUES(308,4030,3758.25,'zbwno',8574,2598.6,'nirww')

statement ok
INSERT INTO tab0 VALUES(309,8056,1508.52,'uqoap',1670,2166.33,'ykain')

statement ok
INSERT INTO tab0 VALUES(310,5,3307.72,'vyhvp',3996,3888.49,'owxhp')

statement ok
INSERT INTO tab0 VALUES(311,5115,2502.15,'jowmn',6818,3038.25,'kcqpj')

statement ok
INSERT INTO tab0 VALUES(312,7238,6600.81,'ovrnr',3136,9671.87,'vnjsu')

statement ok
INSERT INTO tab0 VALUES(313,5089,8521.14,'hkzwp',3941,9879.81,'qizpy')

statement ok
INSERT INTO tab0 VALUES(314,8496,7840.52,'mcaso',4464,7038.34,'eyocx')

statement ok
INSERT INTO tab0 VALUES(315,4650,2048.99,'gfbai',416,3061.15,'mwcpa')

statement ok
INSERT INTO tab0 VALUES(316,4356,7791.85,'ulpdn',4257,9645.3,'kabvx')

statement ok
INSERT INTO tab0 VALUES(317,8284,2129.25,'whjlp',1429,9521.44,'srdgs')

statement ok
INSERT INTO tab0 VALUES(318,3211,3779.87,'pvrzc',6615,7708.15,'fwqxc')

statement ok
INSERT INTO tab0 VALUES(319,8286,4371.48,'tfics',9982,2757.88,'ycmvo')

statement ok
INSERT INTO tab0 VALUES(320,1264,9693.65,'vmvak',3441,8008.91,'ezgvh')

statement ok
INSERT INTO tab0 VALUES(321,4767,945.28,'hymml',6887,2831.70,'stobn')

statement ok
INSERT INTO tab0 VALUES(322,2530,5986.31,'lzqlt',4456,3886.54,'vaxpp')

statement ok
INSERT INTO tab0 VALUES(323,5650,584.28,'wpqqw',770,6366.16,'nkafz')

statement ok
INSERT INTO tab0 VALUES(324,8839,9670.0,'hczdc',7191,5249.34,'obwvo')

statement ok
INSERT INTO tab0 VALUES(325,6670,8584.14,'uqpdo',8069,6548.56,'sitfq')

statement ok
INSERT INTO tab0 VALUES(326,9831,4325.89,'zyulp',6991,8562.61,'tjnpe')

statement ok
INSERT INTO tab0 VALUES(327,1373,6801.56,'evmcm',3034,3125.23,'pvris')

statement ok
INSERT INTO tab0 VALUES(328,2960,85.20,'gxfex',7012,6856.70,'omknu')

statement ok
INSERT INTO tab0 VALUES(329,1120,3656.25,'cdldx',7729,2409.51,'hswnu')

statement ok
INSERT INTO tab0 VALUES(330,3493,3657.15,'opntb',8224,9723.96,'dhftf')

statement ok
INSERT INTO tab0 VALUES(331,5083,4134.98,'pxxlq',2029,7273.80,'yyreu')

statement ok
INSERT INTO tab0 VALUES(332,4508,8301.3,'jaecp',9408,3318.82,'nxuzp')

statement ok
INSERT INTO tab0 VALUES(333,247,9778.52,'nrirv',3620,7153.52,'ntmfd')

statement ok
INSERT INTO tab0 VALUES(334,6895,3847.13,'aitsh',7159,358.70,'rtdln')

statement ok
INSERT INTO tab0 VALUES(335,1409,8697.70,'rikca',8005,8048.11,'xtwfq')

statement ok
INSERT INTO tab0 VALUES(336,9360,6682.79,'uaaol',5882,123.7,'lomwb')

statement ok
INSERT INTO tab0 VALUES(337,1180,5562.37,'tazlg',8497,2518.6,'ytqcl')

statement ok
INSERT INTO tab0 VALUES(338,8417,9758.37,'uyves',7798,8455.42,'pfbon')

statement ok
INSERT INTO tab0 VALUES(339,9980,1604.40,'jcbav',6221,9688.89,'hvqmf')

statement ok
INSERT INTO tab0 VALUES(340,8395,2590.27,'dcipg',2939,7759.29,'tblit')

statement ok
INSERT INTO tab0 VALUES(341,6046,4620.71,'cjsqp',114,5243.13,'plykx')

statement ok
INSERT INTO tab0 VALUES(342,1435,1254.92,'twslm',9147,7002.23,'ooenz')

statement ok
INSERT INTO tab0 VALUES(343,5425,8967.24,'gsiqr',6928,1432.36,'khtma')

statement ok
INSERT INTO tab0 VALUES(344,7493,8390.76,'nviuh',2081,1714.84,'kpczv')

statement ok
INSERT INTO tab0 VALUES(345,1976,5566.96,'gmhyt',1224,7065.19,'dgmxh')

statement ok
INSERT INTO tab0 VALUES(346,4331,3198.30,'fysia',5210,3987.74,'tgdeh')

statement ok
INSERT INTO tab0 VALUES(347,7373,1901.88,'hdfyt',5380,7518.26,'cppga')

statement ok
INSERT INTO tab0 VALUES(348,7531,1444.5,'oipfj',9814,2477.58,'vwlzw')

statement ok
INSERT INTO tab0 VALUES(349,1099,2014.4,'npypd',9582,4723.23,'nwuvu')

statement ok
INSERT INTO tab0 VALUES(350,483,5453.57,'dnuek',4989,2399.5,'wefxh')

statement ok
INSERT INTO tab0 VALUES(351,452,6516.50,'zxhjr',1831,7182.81,'pwyhr')

statement ok
INSERT INTO tab0 VALUES(352,4390,4116.43,'kghls',7641,5535.86,'ycxxx')

statement ok
INSERT INTO tab0 VALUES(353,542,2577.36,'qmlgr',7941,9956.73,'jrayc')

statement ok
INSERT INTO tab0 VALUES(354,8269,2883.26,'vnhue',2490,227.76,'cvcpi')

statement ok
INSERT INTO tab0 VALUES(355,4292,3929.75,'izour',1536,4666.74,'vrdpy')

statement ok
INSERT INTO tab0 VALUES(356,8948,3768.69,'rrcne',2066,1378.53,'fxmzp')

statement ok
INSERT INTO tab0 VALUES(357,2436,867.83,'vjbcs',4055,9501.65,'tlrom')

statement ok
INSERT INTO tab0 VALUES(358,1069,5350.41,'fhdmj',6715,457.97,'ccyrg')

statement ok
INSERT INTO tab0 VALUES(359,3522,251.81,'vwnti',4748,9207.28,'aabri')

statement ok
INSERT INTO tab0 VALUES(360,7084,2766.13,'mgcbz',4593,5494.94,'omoub')

statement ok
INSERT INTO tab0 VALUES(361,3908,5499.89,'wzpxg',678,5096.49,'gnxjq')

statement ok
INSERT INTO tab0 VALUES(362,7262,7718.58,'qrscn',1669,1081.30,'dlnzj')

statement ok
INSERT INTO tab0 VALUES(363,7769,6596.33,'moeev',6721,5043.16,'amzou')

statement ok
INSERT INTO tab0 VALUES(364,9677,8470.53,'mbycy',1726,8370.95,'lodlv')

statement ok
INSERT INTO tab0 VALUES(365,7089,86.43,'aphuq',5564,3481.22,'wrgrt')

statement ok
INSERT INTO tab0 VALUES(366,224,9850.79,'zcxyq',6356,5082.84,'wtzxo')

statement ok
INSERT INTO tab0 VALUES(367,1664,9077.49,'vocmv',7709,1411.61,'ddttu')

statement ok
INSERT INTO tab0 VALUES(368,2605,3245.18,'utvfj',4968,5495.92,'tgrpa')

statement ok
INSERT INTO tab0 VALUES(369,9106,3240.88,'azxnu',1467,6393.32,'efugq')

statement ok
INSERT INTO tab0 VALUES(370,8338,6609.59,'qjdwh',4780,452.19,'lmbid')

statement ok
INSERT INTO tab0 VALUES(371,4836,8808.96,'uvgqm',2966,2551.35,'fnsfi')

statement ok
INSERT INTO tab0 VALUES(372,6748,4455.89,'aljnz',824,7357.68,'agdkg')

statement ok
INSERT INTO tab0 VALUES(373,2088,8576.15,'dhtfd',6141,571.76,'hychk')

statement ok
INSERT INTO tab0 VALUES(374,4980,7859.1,'mxgel',5944,179.46,'opzra')

statement ok
INSERT INTO tab0 VALUES(375,1912,1902.89,'zxecm',2745,7701.62,'nrrnv')

statement ok
INSERT INTO tab0 VALUES(376,7985,4462.93,'itrnc',797,9446.61,'cuxno')

statement ok
INSERT INTO tab0 VALUES(377,3166,4706.66,'cjnqo',2340,7753.3,'tehpw')

statement ok
INSERT INTO tab0 VALUES(378,8531,8465.61,'wnqrs',3853,2250.26,'mmkum')

statement ok
INSERT INTO tab0 VALUES(379,2522,9472.95,'htxty',2302,1099.13,'wfzrh')

statement ok
INSERT INTO tab0 VALUES(380,7195,629.9,'hvkjm',1392,2247.55,'mnaaq')

statement ok
INSERT INTO tab0 VALUES(381,4468,5244.32,'ejoym',7817,4678.10,'qtxlf')

statement ok
INSERT INTO tab0 VALUES(382,2369,8285.42,'sqgsc',7510,9655.68,'ucocj')

statement ok
INSERT INTO tab0 VALUES(383,1296,7631.32,'hwydo',1890,1984.13,'qdlde')

statement ok
INSERT INTO tab0 VALUES(384,256,5321.68,'cfsom',4112,1253.86,'bpxom')

statement ok
INSERT INTO tab0 VALUES(385,3120,8803.55,'ccpwv',5328,3081.45,'mybod')

statement ok
INSERT INTO tab0 VALUES(386,4873,2187.98,'jclcc',919,380.33,'tvdkc')

statement ok
INSERT INTO tab0 VALUES(387,7861,7493.65,'zghdl',6760,4593.34,'oalzl')

statement ok
INSERT INTO tab0 VALUES(388,8046,5899.48,'ibgfu',5462,6947.38,'itngt')

statement ok
INSERT INTO tab0 VALUES(389,1254,8996.27,'duwoy',2624,9704.7,'inifx')

statement ok
INSERT INTO tab0 VALUES(390,9420,33.34,'rgceu',3085,7416.32,'brjel')

statement ok
INSERT INTO tab0 VALUES(391,6846,5171.11,'vllfh',7170,8165.98,'axtrp')

statement ok
INSERT INTO tab0 VALUES(392,6447,4729.77,'cadwt',6944,4819.48,'edhun')

statement ok
INSERT INTO tab0 VALUES(393,5998,4437.89,'beftv',2460,6285.42,'vwuee')

statement ok
INSERT INTO tab0 VALUES(394,4392,6688.3,'caqss',8010,8749.24,'vayht')

statement ok
INSERT INTO tab0 VALUES(395,7882,1778.37,'vjnig',8372,6938.79,'uksfp')

statement ok
INSERT INTO tab0 VALUES(396,6335,9171.37,'cfbqd',5064,5360.0,'blfhj')

statement ok
INSERT INTO tab0 VALUES(397,9482,8739.33,'bnwqu',2592,9560.50,'ztzsg')

statement ok
INSERT INTO tab0 VALUES(398,3167,2010.9,'ltybw',4700,7369.20,'vubby')

statement ok
INSERT INTO tab0 VALUES(399,5596,3428.59,'nnyxz',620,150.0,'fnedx')

statement ok
INSERT INTO tab0 VALUES(400,3591,1591.54,'kecbf',1324,4055.31,'yvsjl')

statement ok
INSERT INTO tab0 VALUES(401,8834,514.1,'ynoxv',5771,1269.77,'imaou')

statement ok
INSERT INTO tab0 VALUES(402,4124,868.7,'chkcu',7157,1263.81,'fmllg')

statement ok
INSERT INTO tab0 VALUES(403,9617,1232.45,'xytyr',9169,2041.32,'bullm')

statement ok
INSERT INTO tab0 VALUES(404,9092,9352.9,'qdyft',2230,1394.25,'gateg')

statement ok
INSERT INTO tab0 VALUES(405,6182,3377.83,'ympuh',9752,8150.21,'awmza')

statement ok
INSERT INTO tab0 VALUES(406,639,3121.1,'zbxbo',4386,3419.17,'ryhvq')

statement ok
INSERT INTO tab0 VALUES(407,4575,1307.10,'mcmux',4271,1847.0,'rijru')

statement ok
INSERT INTO tab0 VALUES(408,8718,4816.30,'zqrrb',9284,2866.17,'xdvhz')

statement ok
INSERT INTO tab0 VALUES(409,8656,5941.22,'vctxc',7289,1942.31,'ofmtr')

statement ok
INSERT INTO tab0 VALUES(410,1094,3636.45,'lwvuq',391,1827.88,'jrhaj')

statement ok
INSERT INTO tab0 VALUES(411,98,4763.21,'gzcmx',1181,3808.77,'xrhuw')

statement ok
INSERT INTO tab0 VALUES(412,3348,5346.97,'avaoy',8991,8957.94,'jxvyf')

statement ok
INSERT INTO tab0 VALUES(413,8386,1819.10,'vqhns',6136,9381.4,'zhncz')

statement ok
INSERT INTO tab0 VALUES(414,8870,2274.43,'iccqc',4176,9473.4,'bormi')

statement ok
INSERT INTO tab0 VALUES(415,816,7716.95,'rdflh',154,7161.83,'zwajc')

statement ok
INSERT INTO tab0 VALUES(416,8860,3308.39,'wjbjq',2854,5156.90,'onfrz')

statement ok
INSERT INTO tab0 VALUES(417,2498,9624.87,'svyiy',9047,2208.88,'ymtvq')

statement ok
INSERT INTO tab0 VALUES(418,2339,3899.48,'bqclm',9744,4348.26,'wbcbv')

statement ok
INSERT INTO tab0 VALUES(419,6356,1277.98,'umuqg',6442,6145.94,'umxim')

statement ok
INSERT INTO tab0 VALUES(420,6134,6062.35,'ehmfz',7378,1953.90,'yqasy')

statement ok
INSERT INTO tab0 VALUES(421,3124,2969.45,'sybmh',3776,7159.81,'lnsex')

statement ok
INSERT INTO tab0 VALUES(422,4886,8374.66,'mwsmv',6099,9438.64,'nbvet')

statement ok
INSERT INTO tab0 VALUES(423,1348,1560.48,'dnill',299,9998.31,'qosgn')

statement ok
INSERT INTO tab0 VALUES(424,1049,5974.14,'bgkqe',4066,4834.31,'klppd')

statement ok
INSERT INTO tab0 VALUES(425,7390,469.27,'pshsp',3076,5974.31,'bunxm')

statement ok
INSERT INTO tab0 VALUES(426,4888,562.51,'zpioq',7246,2732.79,'fbnrk')

statement ok
INSERT INTO tab0 VALUES(427,5422,9679.9,'qhwxz',6981,2959.68,'hrwce')

statement ok
INSERT INTO tab0 VALUES(428,4483,405.54,'ekltc',9422,4288.97,'mpzog')

statement ok
INSERT INTO tab0 VALUES(429,7427,5736.23,'pjafh',4702,5390.35,'wasgo')

statement ok
INSERT INTO tab0 VALUES(430,5724,328.5,'lhwby',2188,7219.80,'pylyr')

statement ok
INSERT INTO tab0 VALUES(431,3204,1747.1,'vhxjo',6740,4266.27,'cwdkx')

statement ok
INSERT INTO tab0 VALUES(432,2315,1719.86,'phucy',4951,5805.24,'ahodh')

statement ok
INSERT INTO tab0 VALUES(433,363,7383.76,'nrxrh',1407,7146.42,'kokax')

statement ok
INSERT INTO tab0 VALUES(434,5772,2532.84,'gtrui',4931,887.29,'xefxx')

statement ok
INSERT INTO tab0 VALUES(435,4326,8876.87,'tmnwj',4830,8156.81,'xrwer')

statement ok
INSERT INTO tab0 VALUES(436,5424,7281.88,'adbtp',7156,9870.98,'mgjnw')

statement ok
INSERT INTO tab0 VALUES(437,8328,9023.76,'nwiha',946,7980.49,'tedls')

statement ok
INSERT INTO tab0 VALUES(438,8412,7041.92,'hirbt',4474,6723.36,'uqcxx')

statement ok
INSERT INTO tab0 VALUES(439,3415,7181.87,'vidvz',1505,3869.53,'zfxyy')

statement ok
INSERT INTO tab0 VALUES(440,9689,9452.90,'mmkhf',7541,1974.52,'vqujc')

statement ok
INSERT INTO tab0 VALUES(441,8449,9271.66,'lryuj',6071,9631.4,'mgahr')

statement ok
INSERT INTO tab0 VALUES(442,2023,8716.82,'xygru',4825,2716.60,'muvva')

statement ok
INSERT INTO tab0 VALUES(443,101,7775.65,'dxwdq',2019,5642.16,'danrs')

statement ok
INSERT INTO tab0 VALUES(444,5990,5596.70,'sllvr',2609,2604.74,'ncutp')

statement ok
INSERT INTO tab0 VALUES(445,2857,1739.37,'njcis',4538,8837.5,'qnqvc')

statement ok
INSERT INTO tab0 VALUES(446,9984,7420.59,'zyizn',4987,1856.13,'qpncd')

statement ok
INSERT INTO tab0 VALUES(447,9013,6770.39,'dqdiy',6627,2714.93,'rcqxk')

statement ok
INSERT INTO tab0 VALUES(448,205,8077.57,'trjwg',3866,7250.98,'lvwjl')

statement ok
INSERT INTO tab0 VALUES(449,8679,5040.31,'lroqa',9100,7335.1,'vibuq')

statement ok
INSERT INTO tab0 VALUES(450,671,8018.76,'xfguj',8964,2129.54,'llyop')

statement ok
INSERT INTO tab0 VALUES(451,9778,5281.75,'vobhc',582,9323.56,'mbnrb')

statement ok
INSERT INTO tab0 VALUES(452,3212,4257.31,'ghuns',7814,6398.9,'bnyhe')

statement ok
INSERT INTO tab0 VALUES(453,8045,2788.38,'ovowr',7511,1865.88,'blvln')

statement ok
INSERT INTO tab0 VALUES(454,7467,372.39,'ocykl',5760,6524.94,'phhpf')

statement ok
INSERT INTO tab0 VALUES(455,7027,9420.90,'tebms',4137,4708.28,'vyald')

statement ok
INSERT INTO tab0 VALUES(456,890,6501.21,'kxvih',5958,5537.44,'hjmez')

statement ok
INSERT INTO tab0 VALUES(457,8084,3596.2,'npwto',2525,4164.14,'yfcom')

statement ok
INSERT INTO tab0 VALUES(458,8748,9346.1,'hltsh',9378,3029.94,'hogdi')

statement ok
INSERT INTO tab0 VALUES(459,388,300.50,'wkgwz',2548,3106.83,'nhlfo')

statement ok
INSERT INTO tab0 VALUES(460,4120,2171.6,'sqekw',4801,5155.86,'palek')

statement ok
INSERT INTO tab0 VALUES(461,924,1077.13,'rftmc',1895,971.78,'nlkjs')

statement ok
INSERT INTO tab0 VALUES(462,7895,4802.66,'asqyx',8823,4633.64,'febov')

statement ok
INSERT INTO tab0 VALUES(463,4490,4036.73,'svbwo',6067,3757.43,'wfqnu')

statement ok
INSERT INTO tab0 VALUES(464,8778,3630.15,'kcyiq',5848,742.72,'yrgyo')

statement ok
INSERT INTO tab0 VALUES(465,3695,3846.13,'skzug',8376,4024.27,'ikwrz')

statement ok
INSERT INTO tab0 VALUES(466,3286,1605.88,'cryav',8098,9325.63,'oojra')

statement ok
INSERT INTO tab0 VALUES(467,8962,1740.23,'gkomo',9950,897.81,'rfykw')

statement ok
INSERT INTO tab0 VALUES(468,5746,8242.27,'cltcl',5603,2785.88,'saxoa')

statement ok
INSERT INTO tab0 VALUES(469,8525,5514.94,'wmwui',9722,62.59,'nyztm')

statement ok
INSERT INTO tab0 VALUES(470,8299,9617.72,'ktpeb',6207,90.3,'aqlyu')

statement ok
INSERT INTO tab0 VALUES(471,8432,9525.20,'apgcs',4447,3766.19,'iieif')

statement ok
INSERT INTO tab0 VALUES(472,900,8370.71,'wfujo',5568,6771.67,'xgvsx')

statement ok
INSERT INTO tab0 VALUES(473,2544,4695.97,'zcdgi',8686,822.22,'oixhy')

statement ok
INSERT INTO tab0 VALUES(474,1315,3703.71,'huyhf',7155,4920.43,'txzfl')

statement ok
INSERT INTO tab0 VALUES(475,2531,8506.27,'zmdsd',4815,6602.32,'oeshv')

statement ok
INSERT INTO tab0 VALUES(476,5122,1378.26,'jpsdt',2249,3191.96,'xazmv')

statement ok
INSERT INTO tab0 VALUES(477,7879,5003.14,'foajp',3023,3043.14,'okfpo')

statement ok
INSERT INTO tab0 VALUES(478,5711,3724.90,'rexrg',9944,6801.95,'mgdxw')

statement ok
INSERT INTO tab0 VALUES(479,5886,4794.64,'rltsm',1308,8643.41,'suhwe')

statement ok
INSERT INTO tab0 VALUES(480,4288,9628.22,'tfpbh',7149,7531.11,'rnwnv')

statement ok
INSERT INTO tab0 VALUES(481,1661,5051.87,'iojub',5486,8464.83,'clykc')

statement ok
INSERT INTO tab0 VALUES(482,9867,4544.82,'ytxct',8580,3723.18,'kbrwi')

statement ok
INSERT INTO tab0 VALUES(483,8127,9504.95,'coccy',9180,4128.19,'qthze')

statement ok
INSERT INTO tab0 VALUES(484,9278,485.37,'pdddz',6771,281.95,'qkjan')

statement ok
INSERT INTO tab0 VALUES(485,6018,8451.6,'qkksd',3829,3822.26,'fhszs')

statement ok
INSERT INTO tab0 VALUES(486,7161,6027.52,'xyuzs',2067,8673.23,'ejtnu')

statement ok
INSERT INTO tab0 VALUES(487,3882,8749.38,'sithd',8683,2294.89,'pbogj')

statement ok
INSERT INTO tab0 VALUES(488,5971,6310.0,'balzc',2621,6928.86,'uxyna')

statement ok
INSERT INTO tab0 VALUES(489,8079,3818.14,'wvwxt',806,855.92,'mfodf')

statement ok
INSERT INTO tab0 VALUES(490,5973,3036.54,'aoybr',4405,9639.70,'amcsm')

statement ok
INSERT INTO tab0 VALUES(491,2781,3641.59,'orqgf',5073,7380.24,'lhlor')

statement ok
INSERT INTO tab0 VALUES(492,5246,4468.92,'tyerl',1201,7961.67,'mspgx')

statement ok
INSERT INTO tab0 VALUES(493,9973,4764.3,'gacng',4171,6303.78,'ybcbt')

statement ok
INSERT INTO tab0 VALUES(494,9681,5434.93,'czhss',4602,9611.70,'plyvi')

statement ok
INSERT INTO tab0 VALUES(495,9452,295.75,'ddjkx',8602,8624.48,'lnsnq')

statement ok
INSERT INTO tab0 VALUES(496,2981,7322.40,'vpmsg',4016,8868.75,'tyuth')

statement ok
INSERT INTO tab0 VALUES(497,8828,9903.13,'ntugq',6679,1903.32,'fblnb')

statement ok
INSERT INTO tab0 VALUES(498,6664,8085.21,'ocfsh',6661,6575.82,'dqyur')

statement ok
INSERT INTO tab0 VALUES(499,3524,9823.53,'ltwth',4526,264.32,'lhscd')

statement ok
INSERT INTO tab0 VALUES(500,57,2237.24,'xbagj',7570,137.94,'yikgt')

statement ok
INSERT INTO tab0 VALUES(501,6774,2849.37,'zhghf',6081,4394.52,'hdcvj')

statement ok
INSERT INTO tab0 VALUES(502,301,851.43,'hhpln',9586,3521.18,'gziyh')

statement ok
INSERT INTO tab0 VALUES(503,9626,8339.8,'zxsrb',3097,1540.0,'urhur')

statement ok
INSERT INTO tab0 VALUES(504,2351,1571.73,'cxxwb',3366,7257.61,'bayhk')

statement ok
INSERT INTO tab0 VALUES(505,940,2214.15,'jjgiz',5134,5514.86,'umerk')

statement ok
INSERT INTO tab0 VALUES(506,8971,8597.13,'pjwrw',7380,8437.37,'fyufp')

statement ok
INSERT INTO tab0 VALUES(507,7388,6956.8,'uryek',9570,6688.24,'fgurg')

statement ok
INSERT INTO tab0 VALUES(508,742,2692.66,'aitoc',7909,716.61,'nskcv')

statement ok
INSERT INTO tab0 VALUES(509,3454,1612.41,'xfaph',1590,2001.31,'uzgkj')

statement ok
INSERT INTO tab0 VALUES(510,6054,1184.36,'ltwtj',9048,4311.50,'rloyl')

statement ok
INSERT INTO tab0 VALUES(511,1437,1788.70,'hozqx',4198,4948.27,'czzeh')

statement ok
INSERT INTO tab0 VALUES(512,1110,7962.33,'vxtyt',2702,6001.28,'gpdgd')

statement ok
INSERT INTO tab0 VALUES(513,1445,9656.55,'jpkub',8205,5594.36,'uxlsa')

statement ok
INSERT INTO tab0 VALUES(514,9927,3477.66,'diblp',8591,9131.18,'vxhac')

statement ok
INSERT INTO tab0 VALUES(515,1403,3947.59,'nmybu',2236,558.83,'gnhiv')

statement ok
INSERT INTO tab0 VALUES(516,4333,4744.51,'duocu',4249,3603.48,'tifwx')

statement ok
INSERT INTO tab0 VALUES(517,7043,9959.48,'asobs',2730,3030.98,'xbhwn')

statement ok
INSERT INTO tab0 VALUES(518,3970,5339.6,'pkgeo',5017,5886.50,'jggwm')

statement ok
INSERT INTO tab0 VALUES(519,1914,9547.92,'zahxr',3937,2755.14,'zxlrh')

statement ok
INSERT INTO tab0 VALUES(520,6290,5402.51,'cogpo',4054,9008.57,'wdzqf')

statement ok
INSERT INTO tab0 VALUES(521,7376,1134.63,'ojjed',7921,6047.65,'rtwqq')

statement ok
INSERT INTO tab0 VALUES(522,6561,1634.11,'ewwsv',2328,2563.7,'lxcix')

statement ok
INSERT INTO tab0 VALUES(523,2601,9702.37,'sccbt',4556,5779.92,'bqveq')

statement ok
INSERT INTO tab0 VALUES(524,7862,7299.12,'hinmu',8621,2705.40,'crmrs')

statement ok
INSERT INTO tab0 VALUES(525,4573,6795.92,'iuagt',1531,1386.36,'evugp')

statement ok
INSERT INTO tab0 VALUES(526,1080,2223.84,'tdvjj',9528,2152.99,'vvcvg')

statement ok
INSERT INTO tab0 VALUES(527,8091,523.9,'eguzr',4346,8867.41,'wvhjj')

statement ok
INSERT INTO tab0 VALUES(528,4924,2797.62,'wuniu',9454,182.32,'kccpm')

statement ok
INSERT INTO tab0 VALUES(529,5763,5556.75,'zrhbm',6314,783.57,'yncnc')

statement ok
INSERT INTO tab0 VALUES(530,5906,1972.81,'samyu',621,2640.87,'qlwfu')

statement ok
INSERT INTO tab0 VALUES(531,8013,3126.76,'jmmro',8561,6258.34,'eadhk')

statement ok
INSERT INTO tab0 VALUES(532,8357,9261.80,'lzvgj',3058,1928.18,'edwob')

statement ok
INSERT INTO tab0 VALUES(533,3075,1029.4,'vimsr',24,977.27,'csksi')

statement ok
INSERT INTO tab0 VALUES(534,4417,742.4,'yytvb',4610,7015.81,'orrxl')

statement ok
INSERT INTO tab0 VALUES(535,6997,3623.53,'vgqoc',7455,7189.93,'hhekh')

statement ok
INSERT INTO tab0 VALUES(536,4040,9054.97,'eorut',935,3431.35,'jxeep')

statement ok
INSERT INTO tab0 VALUES(537,3638,1828.21,'qmiai',5415,3706.94,'erunc')

statement ok
INSERT INTO tab0 VALUES(538,4993,3398.9,'vsbus',5500,3267.41,'wyftn')

statement ok
INSERT INTO tab0 VALUES(539,4671,3127.8,'nognl',6282,5901.92,'cduty')

statement ok
INSERT INTO tab0 VALUES(540,2852,9538.91,'qqqas',9589,5179.66,'kezpr')

statement ok
INSERT INTO tab0 VALUES(541,9581,1565.17,'pidbu',241,5709.55,'gfugn')

statement ok
INSERT INTO tab0 VALUES(542,1011,9802.98,'ftfew',2295,4383.78,'efsex')

statement ok
INSERT INTO tab0 VALUES(543,5664,7752.56,'yxxcg',5295,1629.53,'vrtvw')

statement ok
INSERT INTO tab0 VALUES(544,672,3078.90,'avwnp',525,2660.49,'gbulk')

statement ok
INSERT INTO tab0 VALUES(545,8031,8855.35,'nrqqz',7256,8723.58,'wqork')

statement ok
INSERT INTO tab0 VALUES(546,851,4818.27,'mwrvy',8999,8352.53,'rxuaw')

statement ok
INSERT INTO tab0 VALUES(547,9605,8610.66,'diofr',9750,8614.48,'ilkqa')

statement ok
INSERT INTO tab0 VALUES(548,8378,7036.75,'xgaje',247,6363.67,'ihmxj')

statement ok
INSERT INTO tab0 VALUES(549,1121,7311.65,'qtcwi',4214,9488.2,'ctpay')

statement ok
INSERT INTO tab0 VALUES(550,7893,8309.0,'nmiab',4614,8161.74,'bytri')

statement ok
INSERT INTO tab0 VALUES(551,1123,1816.52,'pgugk',9899,6093.2,'xmprv')

statement ok
INSERT INTO tab0 VALUES(552,1849,3776.73,'lngcs',258,1776.57,'pzpnq')

statement ok
INSERT INTO tab0 VALUES(553,5975,5545.50,'rhins',3868,9740.33,'mwbbv')

statement ok
INSERT INTO tab0 VALUES(554,514,5139.70,'smopr',3146,1800.4,'otgbp')

statement ok
INSERT INTO tab0 VALUES(555,9847,8482.83,'hrogo',997,8304.86,'kjrai')

statement ok
INSERT INTO tab0 VALUES(556,1966,661.65,'huevz',7062,9388.4,'mwbnv')

statement ok
INSERT INTO tab0 VALUES(557,4725,7172.80,'xnnso',3973,6220.51,'dtlah')

statement ok
INSERT INTO tab0 VALUES(558,2095,5170.41,'yefwx',3599,2256.51,'vhmon')

statement ok
INSERT INTO tab0 VALUES(559,406,4722.74,'tdkpu',3297,4486.24,'ddlcz')

statement ok
INSERT INTO tab0 VALUES(560,4607,54.80,'mqydh',2163,4685.43,'ydhlb')

statement ok
INSERT INTO tab0 VALUES(561,4500,803.98,'cfnet',7282,664.2,'puxsc')

statement ok
INSERT INTO tab0 VALUES(562,1013,3266.68,'omtrg',9975,7519.77,'byfal')

statement ok
INSERT INTO tab0 VALUES(563,2429,1649.92,'zqjgi',2053,8877.88,'wjfzs')

statement ok
INSERT INTO tab0 VALUES(564,646,9319.96,'sffbj',2068,8159.59,'jlgnx')

statement ok
INSERT INTO tab0 VALUES(565,7428,55.95,'lpodt',1549,9811.85,'ytzkk')

statement ok
INSERT INTO tab0 VALUES(566,647,9290.55,'lsbis',1280,573.92,'yrovf')

statement ok
INSERT INTO tab0 VALUES(567,544,5376.53,'lpxrt',7520,715.1,'ycwdb')

statement ok
INSERT INTO tab0 VALUES(568,5969,819.43,'ccdep',8273,5293.87,'tzlim')

statement ok
INSERT INTO tab0 VALUES(569,637,3536.13,'ulkhk',9529,5400.47,'sxtuz')

statement ok
INSERT INTO tab0 VALUES(570,9378,8106.56,'zumrv',1516,9411.56,'rxhxf')

statement ok
INSERT INTO tab0 VALUES(571,4381,7017.25,'fccvg',4952,4239.36,'exrpy')

statement ok
INSERT INTO tab0 VALUES(572,1452,1917.99,'gmhor',7290,6581.14,'nkuai')

statement ok
INSERT INTO tab0 VALUES(573,6258,4864.43,'iyiud',8975,4168.86,'zdvea')

statement ok
INSERT INTO tab0 VALUES(574,582,9577.75,'bcyar',9689,7019.80,'ypqpf')

statement ok
INSERT INTO tab0 VALUES(575,8568,2685.24,'solcl',4601,8687.17,'wvnvv')

statement ok
INSERT INTO tab0 VALUES(576,2973,4976.94,'ixxux',619,7507.75,'pytfp')

statement ok
INSERT INTO tab0 VALUES(577,1419,2650.27,'uxcix',2641,7855.51,'irhih')

statement ok
INSERT INTO tab0 VALUES(578,2640,8566.66,'alveu',9801,1387.23,'ydfvq')

statement ok
INSERT INTO tab0 VALUES(579,2734,3493.16,'zbzmo',1722,8296.15,'wlsys')

statement ok
INSERT INTO tab0 VALUES(580,1412,7360.7,'ynoyc',4765,8563.54,'kfgbw')

statement ok
INSERT INTO tab0 VALUES(581,8248,508.57,'mnzeb',8281,3197.38,'vjrmx')

statement ok
INSERT INTO tab0 VALUES(582,8930,4936.34,'ufbld',857,5501.97,'imhyh')

statement ok
INSERT INTO tab0 VALUES(583,1509,5554.18,'nzelg',6569,2306.54,'mubib')

statement ok
INSERT INTO tab0 VALUES(584,2648,1432.67,'fykpn',1232,5233.87,'ymueg')

statement ok
INSERT INTO tab0 VALUES(585,3875,7051.73,'dntyn',8648,1935.97,'zmbks')

statement ok
INSERT INTO tab0 VALUES(586,7832,3551.31,'sizsh',6862,7351.61,'jbnwn')

statement ok
INSERT INTO tab0 VALUES(587,4520,9982.97,'ilbwp',4396,4955.6,'mlycv')

statement ok
INSERT INTO tab0 VALUES(588,7560,4556.53,'thduy',6523,8009.60,'pjish')

statement ok
INSERT INTO tab0 VALUES(589,429,5892.69,'zgqxs',1498,805.24,'xwcjw')

statement ok
INSERT INTO tab0 VALUES(590,2723,4273.63,'rcmgh',6815,9924.27,'queze')

statement ok
INSERT INTO tab0 VALUES(591,6398,2890.69,'xvegi',5456,5481.95,'fkdrn')

statement ok
INSERT INTO tab0 VALUES(592,2026,9094.95,'sfvdr',2143,2377.8,'ibhid')

statement ok
INSERT INTO tab0 VALUES(593,5038,5646.84,'vimrn',9583,1546.67,'ewucb')

statement ok
INSERT INTO tab0 VALUES(594,1983,1968.84,'akaxr',4725,1209.53,'ldxcp')

statement ok
INSERT INTO tab0 VALUES(595,6771,5391.1,'hennp',5004,8122.92,'mhzdc')

statement ok
INSERT INTO tab0 VALUES(596,6925,2409.70,'sterl',4595,8826.16,'phpdi')

statement ok
INSERT INTO tab0 VALUES(597,9832,9924.12,'vcmoj',658,896.25,'veuwp')

statement ok
INSERT INTO tab0 VALUES(598,8992,6496.97,'sqlzm',9595,4194.52,'jazsq')

statement ok
INSERT INTO tab0 VALUES(599,269,3599.10,'xaqda',3294,2117.22,'qejyz')

statement ok
INSERT INTO tab0 VALUES(600,8137,6866.10,'wqgzp',5383,2845.19,'heuta')

statement ok
INSERT INTO tab0 VALUES(601,1939,7340.93,'vflde',7476,595.45,'kfcmc')

statement ok
INSERT INTO tab0 VALUES(602,2127,2722.57,'aqdie',5428,6868.82,'rvaqq')

statement ok
INSERT INTO tab0 VALUES(603,6517,8627.61,'vhnau',5579,1703.57,'lnndf')

statement ok
INSERT INTO tab0 VALUES(604,6540,9827.91,'vwuex',1011,4921.4,'wkuhv')

statement ok
INSERT INTO tab0 VALUES(605,3488,1507.19,'bapri',7792,7379.25,'asiew')

statement ok
INSERT INTO tab0 VALUES(606,4769,509.68,'xuafx',1228,2219.54,'bnhgl')

statement ok
INSERT INTO tab0 VALUES(607,148,6588.73,'nnrqo',2114,6119.20,'gmjgf')

statement ok
INSERT INTO tab0 VALUES(608,8014,3658.45,'tewtq',2984,543.5,'heamt')

statement ok
INSERT INTO tab0 VALUES(609,1608,1041.32,'fetwt',4359,2987.13,'fzpqp')

statement ok
INSERT INTO tab0 VALUES(610,5293,4721.83,'szmnc',999,8235.30,'qommk')

statement ok
INSERT INTO tab0 VALUES(611,790,8595.10,'ardym',8152,9605.83,'woefy')

statement ok
INSERT INTO tab0 VALUES(612,3435,5716.49,'epdvo',7804,199.7,'afljl')

statement ok
INSERT INTO tab0 VALUES(613,8428,9550.23,'qzexv',4932,4158.7,'bwlgn')

statement ok
INSERT INTO tab0 VALUES(614,7082,5220.8,'iojov',3745,7904.18,'ewurj')

statement ok
INSERT INTO tab0 VALUES(615,3071,762.90,'thlkf',3947,1396.69,'pvvpy')

statement ok
INSERT INTO tab0 VALUES(616,8686,8371.32,'huuaa',6976,9378.47,'xqopa')

statement ok
INSERT INTO tab0 VALUES(617,1078,5652.34,'adqmn',2262,2630.55,'dryri')

statement ok
INSERT INTO tab0 VALUES(618,8513,6870.96,'ckdcq',3469,9215.80,'vvytr')

statement ok
INSERT INTO tab0 VALUES(619,6621,4202.80,'vipxy',4753,5733.52,'gwpkd')

statement ok
INSERT INTO tab0 VALUES(620,4643,8982.75,'swkhs',7005,8568.8,'mdhja')

statement ok
INSERT INTO tab0 VALUES(621,1685,2958.57,'pxiue',4106,5140.54,'lesfg')

statement ok
INSERT INTO tab0 VALUES(622,2966,9534.51,'ywqno',9177,5157.25,'hkdce')

statement ok
INSERT INTO tab0 VALUES(623,6067,5028.96,'rhcyh',823,1852.54,'drjdz')

statement ok
INSERT INTO tab0 VALUES(624,7185,9915.6,'nldhz',9393,3256.28,'pxcfp')

statement ok
INSERT INTO tab0 VALUES(625,7379,82.33,'ebdos',3218,3428.52,'qsuqa')

statement ok
INSERT INTO tab0 VALUES(626,8818,8076.27,'evhqd',4871,3714.15,'sbaof')

statement ok
INSERT INTO tab0 VALUES(627,1040,4354.5,'hdzlk',7414,3993.48,'cwlum')

statement ok
INSERT INTO tab0 VALUES(628,7193,8532.4,'lqrcp',644,3932.90,'uqxji')

statement ok
INSERT INTO tab0 VALUES(629,288,639.53,'zugxp',7349,4232.47,'sbuew')

statement ok
INSERT INTO tab0 VALUES(630,9450,2359.29,'trzti',9556,7879.59,'wrxnq')

statement ok
INSERT INTO tab0 VALUES(631,6169,8292.22,'fltvz',4776,1395.57,'llqxb')

statement ok
INSERT INTO tab0 VALUES(632,545,7196.98,'nbcgv',2263,6084.31,'ymfkh')

statement ok
INSERT INTO tab0 VALUES(633,7542,3237.60,'twlpq',9256,9290.88,'gnupl')

statement ok
INSERT INTO tab0 VALUES(634,3240,7245.10,'agbsu',7235,5639.18,'wrkyu')

statement ok
INSERT INTO tab0 VALUES(635,1082,7957.16,'kdkdd',8165,8293.54,'zahlt')

statement ok
INSERT INTO tab0 VALUES(636,8118,3270.68,'kwcvl',753,200.35,'mezmc')

statement ok
INSERT INTO tab0 VALUES(637,1269,1496.44,'uozvi',6692,5340.71,'iyieb')

statement ok
INSERT INTO tab0 VALUES(638,3567,2599.75,'vxmvl',6912,9982.2,'lbtcn')

statement ok
INSERT INTO tab0 VALUES(639,407,9461.2,'ltnso',7835,3929.61,'zqgzz')

statement ok
INSERT INTO tab0 VALUES(640,5793,9119.42,'gquku',7620,1767.39,'kvqgv')

statement ok
INSERT INTO tab0 VALUES(641,8599,4024.49,'uorho',2409,4120.46,'npnrk')

statement ok
INSERT INTO tab0 VALUES(642,8810,3283.78,'oopxx',2593,8897.72,'ufrye')

statement ok
INSERT INTO tab0 VALUES(643,3773,3917.47,'gfcol',9040,2539.52,'iuvmt')

statement ok
INSERT INTO tab0 VALUES(644,4448,2105.99,'qmvbp',9364,1275.27,'becuc')

statement ok
INSERT INTO tab0 VALUES(645,1662,4436.64,'aetds',4164,2369.8,'wiyhs')

statement ok
INSERT INTO tab0 VALUES(646,2348,9161.23,'doigd',3278,5370.37,'objwp')

statement ok
INSERT INTO tab0 VALUES(647,5810,7440.68,'icqdj',2033,3362.49,'iqwmm')

statement ok
INSERT INTO tab0 VALUES(648,9963,4591.69,'sbije',7492,4212.35,'mkxuz')

statement ok
INSERT INTO tab0 VALUES(649,7582,1347.32,'gbzds',4709,1826.8,'xkswr')

statement ok
INSERT INTO tab0 VALUES(650,2559,5097.63,'xtbts',2974,7392.59,'zbmtc')

statement ok
INSERT INTO tab0 VALUES(651,6148,1142.31,'tjdin',2069,635.25,'tqsyf')

statement ok
INSERT INTO tab0 VALUES(652,3215,3941.31,'dmzjh',8507,1066.48,'bdzjc')

statement ok
INSERT INTO tab0 VALUES(653,8492,8368.45,'soqlz',1798,9227.41,'uqiqm')

statement ok
INSERT INTO tab0 VALUES(654,2822,953.4,'snrrr',684,7950.47,'scixp')

statement ok
INSERT INTO tab0 VALUES(655,9413,1136.53,'zndqp',9399,3599.3,'yydli')

statement ok
INSERT INTO tab0 VALUES(656,85,2519.14,'zjxrf',3914,7325.2,'tptdo')

statement ok
INSERT INTO tab0 VALUES(657,1520,4114.43,'ejntk',8185,6312.57,'zjxjy')

statement ok
INSERT INTO tab0 VALUES(658,1518,1592.98,'auoxn',412,5032.62,'yrnij')

statement ok
INSERT INTO tab0 VALUES(659,5236,8536.1,'znrzs',9281,6490.55,'ranyx')

statement ok
INSERT INTO tab0 VALUES(660,1450,7686.97,'xvsjt',214,3478.81,'zmehw')

statement ok
INSERT INTO tab0 VALUES(661,8796,2196.19,'wsvgj',1624,9819.53,'qibgf')

statement ok
INSERT INTO tab0 VALUES(662,2024,6878.51,'pvqri',1162,1627.15,'kqiih')

statement ok
INSERT INTO tab0 VALUES(663,8246,9455.91,'ixgwo',1966,8548.6,'izsyi')

statement ok
INSERT INTO tab0 VALUES(664,3191,1808.8,'alpye',8913,5012.22,'sytjg')

statement ok
INSERT INTO tab0 VALUES(665,7447,1230.14,'pjkrv',4869,1487.76,'gxvda')

statement ok
INSERT INTO tab0 VALUES(666,2718,1675.17,'xogwp',6937,5723.65,'gnkvs')

statement ok
INSERT INTO tab0 VALUES(667,8071,5006.89,'yqwdg',2709,4010.27,'rjznp')

statement ok
INSERT INTO tab0 VALUES(668,6185,2246.28,'ykfoh',6516,2416.9,'irjpf')

statement ok
INSERT INTO tab0 VALUES(669,9603,4828.20,'nolfe',5674,8052.25,'xwugq')

statement ok
INSERT INTO tab0 VALUES(670,6981,5845.98,'ujiuj',6742,309.23,'eeyem')

statement ok
INSERT INTO tab0 VALUES(671,9340,1303.12,'kgbim',1765,2114.41,'bgece')

statement ok
INSERT INTO tab0 VALUES(672,6311,1364.85,'whqtj',4524,1180.78,'vwoow')

statement ok
INSERT INTO tab0 VALUES(673,2392,1603.78,'mbuni',6793,1255.48,'fnbis')

statement ok
INSERT INTO tab0 VALUES(674,3192,4253.51,'pcgge',4581,3372.49,'jbthy')

statement ok
INSERT INTO tab0 VALUES(675,1682,9459.28,'rrvij',4387,7099.22,'hdqtb')

statement ok
INSERT INTO tab0 VALUES(676,4450,6526.59,'ifnpe',9444,1789.14,'qefkv')

statement ok
INSERT INTO tab0 VALUES(677,7944,5168.80,'rbqwr',5980,7832.42,'esfrm')

statement ok
INSERT INTO tab0 VALUES(678,4728,6760.77,'gdypb',9457,7900.54,'ncxny')

statement ok
INSERT INTO tab0 VALUES(679,9726,5587.52,'glsqq',827,8204.92,'qdfzr')

statement ok
INSERT INTO tab0 VALUES(680,6479,94.78,'nubra',2778,9734.53,'svvte')

statement ok
INSERT INTO tab0 VALUES(681,3388,4137.67,'xjuwd',5988,9523.56,'jrhas')

statement ok
INSERT INTO tab0 VALUES(682,9969,8230.33,'disro',5374,1883.98,'vseuu')

statement ok
INSERT INTO tab0 VALUES(683,3101,801.51,'dosek',6342,9665.23,'yuftb')

statement ok
INSERT INTO tab0 VALUES(684,3277,9433.21,'aewaq',1029,5502.80,'klium')

statement ok
INSERT INTO tab0 VALUES(685,852,2123.32,'jenuc',4572,3065.86,'whxqp')

statement ok
INSERT INTO tab0 VALUES(686,2294,8528.22,'cbosr',1337,8308.92,'ywumj')

statement ok
INSERT INTO tab0 VALUES(687,6436,1213.42,'vnjki',1182,722.40,'nempp')

statement ok
INSERT INTO tab0 VALUES(688,723,6905.18,'azcgj',5704,4582.30,'uffcx')

statement ok
INSERT INTO tab0 VALUES(689,7464,7740.15,'vgphn',4485,197.82,'gudgv')

statement ok
INSERT INTO tab0 VALUES(690,81,2515.97,'mfvqy',1349,5164.95,'dzkla')

statement ok
INSERT INTO tab0 VALUES(691,9359,7937.91,'jhrfo',606,5614.14,'fuioi')

statement ok
INSERT INTO tab0 VALUES(692,9697,9873.48,'chjkz',3304,7461.89,'ewjva')

statement ok
INSERT INTO tab0 VALUES(693,5355,4600.9,'owhot',3482,7657.40,'oboer')

statement ok
INSERT INTO tab0 VALUES(694,6711,56.64,'gcksl',4414,5439.18,'azgbp')

statement ok
INSERT INTO tab0 VALUES(695,6500,7087.15,'gkedt',4864,3577.68,'itods')

statement ok
INSERT INTO tab0 VALUES(696,6076,702.40,'jtvct',5334,9628.92,'poipw')

statement ok
INSERT INTO tab0 VALUES(697,1654,1318.55,'hsiis',4315,7184.9,'kigcv')

statement ok
INSERT INTO tab0 VALUES(698,560,6013.76,'srrtr',4535,7121.22,'uhahk')

statement ok
INSERT INTO tab0 VALUES(699,7488,9775.80,'luwbd',9212,8851.76,'qoohs')

statement ok
INSERT INTO tab0 VALUES(700,741,6753.73,'mmdjl',8298,1572.24,'lzzoc')

statement ok
INSERT INTO tab0 VALUES(701,8488,1505.44,'hbauw',4740,5516.87,'tmtjn')

statement ok
INSERT INTO tab0 VALUES(702,5010,249.36,'tfndr',718,3199.16,'fvyli')

statement ok
INSERT INTO tab0 VALUES(703,6951,2115.4,'lagmg',3902,675.60,'midpu')

statement ok
INSERT INTO tab0 VALUES(704,9570,391.75,'ockuj',3701,8781.21,'rhvjv')

statement ok
INSERT INTO tab0 VALUES(705,2988,3777.19,'iqcqb',7512,4390.20,'htfsl')

statement ok
INSERT INTO tab0 VALUES(706,8437,7630.1,'urkqy',6965,1660.62,'jcrks')

statement ok
INSERT INTO tab0 VALUES(707,4518,5221.52,'hajhj',4190,3285.93,'rpqax')

statement ok
INSERT INTO tab0 VALUES(708,3441,8979.27,'gsilh',8044,1742.68,'sfudp')

statement ok
INSERT INTO tab0 VALUES(709,7280,1744.70,'pswas',379,545.9,'zafiw')

statement ok
INSERT INTO tab0 VALUES(710,4222,8016.73,'rjdpp',2282,5496.69,'aymqk')

statement ok
INSERT INTO tab0 VALUES(711,4819,5204.42,'lisyb',328,5921.98,'nkwbo')

statement ok
INSERT INTO tab0 VALUES(712,8234,4879.58,'rzzxn',2834,5946.0,'nwimn')

statement ok
INSERT INTO tab0 VALUES(713,8495,4022.39,'xzcst',6101,5721.99,'lntbm')

statement ok
INSERT INTO tab0 VALUES(714,743,6271.6,'tedvc',670,2688.98,'ivaxs')

statement ok
INSERT INTO tab0 VALUES(715,7481,3179.39,'yyjln',7572,2820.48,'wieur')

statement ok
INSERT INTO tab0 VALUES(716,4828,5456.45,'kybgu',4573,1878.4,'nzkpe')

statement ok
INSERT INTO tab0 VALUES(717,8762,4223.19,'mtawo',2429,6654.89,'tcmdn')

statement ok
INSERT INTO tab0 VALUES(718,1379,2261.5,'zmiqn',1135,6230.60,'asnsn')

statement ok
INSERT INTO tab0 VALUES(719,7135,5260.61,'hitel',6174,3221.66,'zkiqu')

statement ok
INSERT INTO tab0 VALUES(720,9101,3073.19,'rhsal',4766,5027.13,'rsnbs')

statement ok
INSERT INTO tab0 VALUES(721,5210,961.50,'ctkec',2118,5083.27,'kbmir')

statement ok
INSERT INTO tab0 VALUES(722,4509,845.24,'qdcwi',7681,4110.13,'fzrwe')

statement ok
INSERT INTO tab0 VALUES(723,3409,2559.74,'mmzdx',1388,1363.23,'setrv')

statement ok
INSERT INTO tab0 VALUES(724,1932,5092.59,'zcnhg',9915,5883.14,'juozr')

statement ok
INSERT INTO tab0 VALUES(725,9967,7849.24,'zfows',2922,7747.10,'nehwf')

statement ok
INSERT INTO tab0 VALUES(726,3512,1890.67,'rnhwq',9311,9314.70,'rpsum')

statement ok
INSERT INTO tab0 VALUES(727,931,1189.95,'zxmsq',3470,6483.44,'jtgkh')

statement ok
INSERT INTO tab0 VALUES(728,9202,3879.93,'oasrl',1526,9933.41,'pizcf')

statement ok
INSERT INTO tab0 VALUES(729,163,3500.83,'wwfpi',7594,6383.87,'rvznv')

statement ok
INSERT INTO tab0 VALUES(730,6767,257.87,'lnikz',29,9832.90,'oahiv')

statement ok
INSERT INTO tab0 VALUES(731,9331,2264.90,'wqjxt',7671,2821.2,'uqcvc')

statement ok
INSERT INTO tab0 VALUES(732,1787,6793.1,'zewgw',6102,7666.81,'ubulv')

statement ok
INSERT INTO tab0 VALUES(733,7925,1527.19,'bzrpj',3889,7731.55,'xtfot')

statement ok
INSERT INTO tab0 VALUES(734,7652,5811.90,'ulcwh',8839,8652.83,'ncrxb')

statement ok
INSERT INTO tab0 VALUES(735,3239,1897.76,'ocbdy',4248,9612.18,'fyelu')

statement ok
INSERT INTO tab0 VALUES(736,5567,6200.12,'kddnu',5147,4570.68,'wdkss')

statement ok
INSERT INTO tab0 VALUES(737,383,544.72,'hhqos',4560,3526.63,'thzsr')

statement ok
INSERT INTO tab0 VALUES(738,4547,3414.29,'egaaf',9745,3762.11,'uinkm')

statement ok
INSERT INTO tab0 VALUES(739,2281,1202.19,'chqji',8452,1097.66,'neugh')

statement ok
INSERT INTO tab0 VALUES(740,1421,9464.5,'kbdpy',5166,1133.89,'hgual')

statement ok
INSERT INTO tab0 VALUES(741,9667,4165.35,'ualrv',7361,510.7,'ifxiv')

statement ok
INSERT INTO tab0 VALUES(742,830,2929.8,'jfzet',1752,8151.83,'cqbez')

statement ok
INSERT INTO tab0 VALUES(743,1088,6291.37,'ztqbq',538,6514.69,'akuub')

statement ok
INSERT INTO tab0 VALUES(744,5229,8947.9,'eoccs',7661,2844.77,'yokab')

statement ok
INSERT INTO tab0 VALUES(745,2904,1001.49,'izwqo',5046,420.37,'fwbbz')

statement ok
INSERT INTO tab0 VALUES(746,4881,9220.82,'vhifk',8861,9805.59,'nkzdx')

statement ok
INSERT INTO tab0 VALUES(747,9288,8279.95,'gmzcy',5761,102.99,'menbc')

statement ok
INSERT INTO tab0 VALUES(748,7868,2012.3,'htpat',9136,190.19,'jufuf')

statement ok
INSERT INTO tab0 VALUES(749,2225,4667.6,'dicqj',5369,9993.49,'jbbfh')

statement ok
INSERT INTO tab0 VALUES(750,79,2150.54,'kuzlf',79,1716.50,'wqnrb')

statement ok
INSERT INTO tab0 VALUES(751,1511,3277.11,'ixhay',4373,1096.80,'hptow')

statement ok
INSERT INTO tab0 VALUES(752,5218,1023.17,'xhtym',1097,8661.50,'kcqaz')

statement ok
INSERT INTO tab0 VALUES(753,4757,3778.47,'zwykc',1085,970.91,'mcxon')

statement ok
INSERT INTO tab0 VALUES(754,4801,6584.78,'lpiqs',9409,1868.17,'gsjqv')

statement ok
INSERT INTO tab0 VALUES(755,7385,2400.51,'gcnru',1629,3738.84,'eewhj')

statement ok
INSERT INTO tab0 VALUES(756,8157,3344.28,'qfhao',46,8743.55,'ofrvy')

statement ok
INSERT INTO tab0 VALUES(757,2630,1942.64,'durcg',9947,86.63,'ntgjw')

statement ok
INSERT INTO tab0 VALUES(758,2428,2382.60,'xbhez',5854,3253.70,'jprla')

statement ok
INSERT INTO tab0 VALUES(759,1539,1761.31,'omfpd',7784,7264.34,'nzrvz')

statement ok
INSERT INTO tab0 VALUES(760,2566,1336.68,'pfhsj',7488,5928.70,'sfswf')

statement ok
INSERT INTO tab0 VALUES(761,2972,7777.99,'jxclq',4707,9082.10,'didib')

statement ok
INSERT INTO tab0 VALUES(762,1337,4409.52,'bcgvm',3418,3857.68,'lrfza')

statement ok
INSERT INTO tab0 VALUES(763,3602,2472.18,'fckre',8089,5914.3,'doncu')

statement ok
INSERT INTO tab0 VALUES(764,9321,2365.14,'mkkep',9926,699.80,'jwufk')

statement ok
INSERT INTO tab0 VALUES(765,7913,4894.43,'vkkqs',3121,4695.32,'irdui')

statement ok
INSERT INTO tab0 VALUES(766,4909,2529.59,'wbbzi',1106,6669.23,'eyepr')

statement ok
INSERT INTO tab0 VALUES(767,8203,649.29,'tlojn',2801,8924.14,'hzgqf')

statement ok
INSERT INTO tab0 VALUES(768,4303,8792.95,'ijtyq',8864,8549.20,'liipo')

statement ok
INSERT INTO tab0 VALUES(769,4581,1907.37,'jbkjj',7943,8663.75,'rcjlj')

statement ok
INSERT INTO tab0 VALUES(770,1521,1606.35,'ruekz',2921,9678.95,'rugfh')

statement ok
INSERT INTO tab0 VALUES(771,4971,7403.30,'cmsqj',7172,535.40,'vvdme')

statement ok
INSERT INTO tab0 VALUES(772,8391,7981.92,'ufbce',807,9963.76,'ihgbi')

statement ok
INSERT INTO tab0 VALUES(773,5106,2430.74,'fcjua',6926,7726.39,'fjjpk')

statement ok
INSERT INTO tab0 VALUES(774,4553,1499.84,'jskkm',7867,5871.94,'sdrfs')

statement ok
INSERT INTO tab0 VALUES(775,8534,8847.39,'wslyi',1456,9983.3,'cdacg')

statement ok
INSERT INTO tab0 VALUES(776,7463,2296.32,'apjfo',7258,432.83,'axbhu')

statement ok
INSERT INTO tab0 VALUES(777,6616,5075.95,'chbrp',2825,3620.70,'pugzf')

statement ok
INSERT INTO tab0 VALUES(778,2384,7075.60,'hqgwk',9948,3237.80,'klcli')

statement ok
INSERT INTO tab0 VALUES(779,8522,6097.20,'ydrep',908,1508.77,'vkhoh')

statement ok
INSERT INTO tab0 VALUES(780,2397,9100.95,'qqkjn',4721,442.26,'ywrkn')

statement ok
INSERT INTO tab0 VALUES(781,1239,3457.92,'hwtem',3393,3084.66,'ufpwy')

statement ok
INSERT INTO tab0 VALUES(782,8393,8197.68,'jymcu',8169,4764.65,'nquiz')

statement ok
INSERT INTO tab0 VALUES(783,9564,5263.92,'xgzwv',5780,4347.83,'kusiw')

statement ok
INSERT INTO tab0 VALUES(784,9074,756.11,'tzgck',4699,5484.41,'akpar')

statement ok
INSERT INTO tab0 VALUES(785,6106,6187.6,'onfap',1384,7857.76,'lylzi')

statement ok
INSERT INTO tab0 VALUES(786,957,9728.61,'ynpxc',6532,8441.7,'fdktz')

statement ok
INSERT INTO tab0 VALUES(787,1288,7460.21,'tcjxe',1329,2739.96,'purkx')

statement ok
INSERT INTO tab0 VALUES(788,8372,4627.57,'arpny',3698,6042.14,'sdsit')

statement ok
INSERT INTO tab0 VALUES(789,3345,6880.28,'cprji',4552,4679.98,'oslqq')

statement ok
INSERT INTO tab0 VALUES(790,3458,1531.43,'xslpk',4042,7255.18,'exebe')

statement ok
INSERT INTO tab0 VALUES(791,8323,8802.23,'zbjsh',7806,5922.38,'ulyiw')

statement ok
INSERT INTO tab0 VALUES(792,6170,3077.64,'njnel',3220,7608.78,'zgnxx')

statement ok
INSERT INTO tab0 VALUES(793,84,4148.1,'stxxq',5651,5282.78,'sxppn')

statement ok
INSERT INTO tab0 VALUES(794,4272,8468.69,'qlkzg',4703,9195.57,'oiuez')

statement ok
INSERT INTO tab0 VALUES(795,299,401.65,'ofyij',466,8134.81,'sjxct')

statement ok
INSERT INTO tab0 VALUES(796,3346,3469.10,'yeteq',9770,670.21,'fmzxt')

statement ok
INSERT INTO tab0 VALUES(797,8453,1607.16,'qhgxw',2436,6411.15,'zhnbg')

statement ok
INSERT INTO tab0 VALUES(798,6780,406.43,'zaklr',7518,7251.36,'cjlmd')

statement ok
INSERT INTO tab0 VALUES(799,1844,8372.63,'zbscz',9562,8613.50,'soifa')

statement ok
INSERT INTO tab0 VALUES(800,5779,8166.25,'itjbx',4852,717.76,'jyozj')

statement ok
INSERT INTO tab0 VALUES(801,5581,7865.15,'pxkzy',7658,9801.57,'yfivn')

statement ok
INSERT INTO tab0 VALUES(802,9361,2203.18,'jjuuf',6492,9407.15,'zlxwb')

statement ok
INSERT INTO tab0 VALUES(803,2587,3823.85,'iopbj',350,1599.15,'qpqsq')

statement ok
INSERT INTO tab0 VALUES(804,4350,4531.87,'keaxy',4969,8231.30,'wjefi')

statement ok
INSERT INTO tab0 VALUES(805,3597,6540.89,'xxqkh',9074,2192.2,'qloep')

statement ok
INSERT INTO tab0 VALUES(806,5390,3726.0,'qumhw',628,8655.3,'wmhro')

statement ok
INSERT INTO tab0 VALUES(807,7158,2874.46,'aedpe',3082,6329.1,'tpobb')

statement ok
INSERT INTO tab0 VALUES(808,4936,4928.26,'gcwhc',1625,8709.18,'sxhis')

statement ok
INSERT INTO tab0 VALUES(809,9628,6244.79,'hytuv',9436,5321.97,'qhhud')

statement ok
INSERT INTO tab0 VALUES(810,3251,9838.41,'ffxue',3931,2982.62,'zyjva')

statement ok
INSERT INTO tab0 VALUES(811,5183,6635.14,'dzjrn',3013,7041.80,'xamhh')

statement ok
INSERT INTO tab0 VALUES(812,8416,6720.21,'lpogc',1325,2632.26,'hgwje')

statement ok
INSERT INTO tab0 VALUES(813,5805,591.78,'ziaky',6499,5926.41,'aocak')

statement ok
INSERT INTO tab0 VALUES(814,6321,402.47,'wzxqf',5404,5376.2,'zsisg')

statement ok
INSERT INTO tab0 VALUES(815,1294,1220.4,'ckvtd',396,2974.75,'jyfyu')

statement ok
INSERT INTO tab0 VALUES(816,669,2271.1,'fielg',5572,5497.87,'nojau')

statement ok
INSERT INTO tab0 VALUES(817,9257,8192.27,'sveyo',3016,5746.81,'hocpc')

statement ok
INSERT INTO tab0 VALUES(818,2978,280.71,'cywrl',9050,3826.82,'cgveq')

statement ok
INSERT INTO tab0 VALUES(819,5118,7741.84,'ymmtc',486,8366.59,'azdzm')

statement ok
INSERT INTO tab0 VALUES(820,2576,8943.45,'cklwj',870,7780.89,'xotld')

statement ok
INSERT INTO tab0 VALUES(821,2112,2662.17,'upmob',6743,1519.15,'pdwtl')

statement ok
INSERT INTO tab0 VALUES(822,6602,9336.17,'snsdz',3876,1321.14,'zneqz')

statement ok
INSERT INTO tab0 VALUES(823,1285,2313.71,'erxec',3536,2455.55,'cdzyx')

statement ok
INSERT INTO tab0 VALUES(824,12,9101.82,'hlmcr',8723,4979.66,'illfi')

statement ok
INSERT INTO tab0 VALUES(825,1498,2614.91,'ymszi',429,4984.91,'mnlge')

statement ok
INSERT INTO tab0 VALUES(826,5571,2434.39,'covxh',3883,5847.57,'vxzzq')

statement ok
INSERT INTO tab0 VALUES(827,9423,6049.30,'vabxh',4800,9480.36,'mzeva')

statement ok
INSERT INTO tab0 VALUES(828,3859,105.91,'jnxgz',5635,7579.3,'hvkly')

statement ok
INSERT INTO tab0 VALUES(829,9546,1662.40,'lqejw',1833,2878.6,'ypljn')

statement ok
INSERT INTO tab0 VALUES(830,3265,9438.42,'ekcik',8076,394.23,'jjmpm')

statement ok
INSERT INTO tab0 VALUES(831,4162,5061.78,'izmkj',4532,8678.23,'mpndg')

statement ok
INSERT INTO tab0 VALUES(832,9994,4322.1,'miwwp',3110,1070.37,'rydwb')

statement ok
INSERT INTO tab0 VALUES(833,3383,8418.5,'zwpmm',3707,6941.10,'qqxxe')

statement ok
INSERT INTO tab0 VALUES(834,8063,5209.80,'gyyey',458,2949.70,'shkiv')

statement ok
INSERT INTO tab0 VALUES(835,5130,464.82,'tnfql',7986,4289.91,'wnhxp')

statement ok
INSERT INTO tab0 VALUES(836,9298,9113.92,'gybah',7651,5938.52,'xlnoi')

statement ok
INSERT INTO tab0 VALUES(837,5139,6495.77,'wkgmr',3805,3909.98,'okmaj')

statement ok
INSERT INTO tab0 VALUES(838,3342,9611.56,'fnfuv',35,6502.11,'pqxpo')

statement ok
INSERT INTO tab0 VALUES(839,6957,8373.59,'oftbe',5416,5550.63,'zujqo')

statement ok
INSERT INTO tab0 VALUES(840,7648,6842.93,'uanhh',764,7673.83,'ancdv')

statement ok
INSERT INTO tab0 VALUES(841,1943,4508.0,'vyxah',4881,5523.20,'iheou')

statement ok
INSERT INTO tab0 VALUES(842,7482,7508.25,'nbizr',809,9040.5,'gwthz')

statement ok
INSERT INTO tab0 VALUES(843,9270,9893.75,'tzdpt',2076,6700.10,'rychb')

statement ok
INSERT INTO tab0 VALUES(844,5924,4873.82,'tpztl',8951,6761.17,'iwttd')

statement ok
INSERT INTO tab0 VALUES(845,5925,6069.64,'dvwkt',4689,8518.74,'zbpts')

statement ok
INSERT INTO tab0 VALUES(846,3863,8763.91,'yxdcm',1784,3552.62,'hwcuy')

statement ok
INSERT INTO tab0 VALUES(847,6700,1145.47,'vivru',8219,6977.56,'echlp')

statement ok
INSERT INTO tab0 VALUES(848,4,5569.88,'hkozb',3716,8693.87,'xqzbj')

statement ok
INSERT INTO tab0 VALUES(849,4209,9792.10,'nrvfy',5982,4636.6,'bzwjw')

statement ok
INSERT INTO tab0 VALUES(850,863,8239.8,'xxcxu',6966,8026.43,'ycryu')

statement ok
INSERT INTO tab0 VALUES(851,8875,3241.41,'vhxhn',9101,619.74,'jugbd')

statement ok
INSERT INTO tab0 VALUES(852,768,6292.92,'ylxav',1563,3956.81,'bnsfb')

statement ok
INSERT INTO tab0 VALUES(853,8931,9291.31,'gwdti',6514,612.15,'rynqk')

statement ok
INSERT INTO tab0 VALUES(854,1860,5800.81,'ilcvq',5355,5450.58,'wmrek')

statement ok
INSERT INTO tab0 VALUES(855,423,2294.74,'fivwn',4974,6864.86,'ncvrc')

statement ok
INSERT INTO tab0 VALUES(856,4094,9405.21,'xldlc',3997,8038.33,'vtktk')

statement ok
INSERT INTO tab0 VALUES(857,7214,3175.68,'bamqt',9041,8186.49,'jeqjc')

statement ok
INSERT INTO tab0 VALUES(858,5235,6308.62,'mkmpn',958,3812.67,'hlvjc')

statement ok
INSERT INTO tab0 VALUES(859,974,9417.13,'alrja',1091,9952.66,'mnjio')

statement ok
INSERT INTO tab0 VALUES(860,861,2275.56,'ulxkq',7521,1427.28,'pyaxx')

statement ok
INSERT INTO tab0 VALUES(861,1189,5063.13,'gmfcr',1314,4096.5,'eoutd')

statement ok
INSERT INTO tab0 VALUES(862,4365,9629.57,'jceon',2779,6101.82,'xxhxs')

statement ok
INSERT INTO tab0 VALUES(863,5157,722.74,'teyfp',7710,9119.36,'pdslx')

statement ok
INSERT INTO tab0 VALUES(864,6949,7260.24,'scijs',3639,9613.2,'iiaht')

statement ok
INSERT INTO tab0 VALUES(865,7987,3565.6,'mxavz',6089,81.32,'sicqh')

statement ok
INSERT INTO tab0 VALUES(866,7630,621.19,'gmood',4820,305.62,'wiuqq')

statement ok
INSERT INTO tab0 VALUES(867,2600,7324.23,'jkqjm',80,8605.69,'pndcd')

statement ok
INSERT INTO tab0 VALUES(868,3235,1727.6,'iodld',8855,3083.40,'lnjal')

statement ok
INSERT INTO tab0 VALUES(869,9733,2295.41,'cpfac',1119,2879.27,'aicfs')

statement ok
INSERT INTO tab0 VALUES(870,8407,9153.53,'cwnxl',6087,9413.17,'xqozr')

statement ok
INSERT INTO tab0 VALUES(871,7933,5335.78,'jthhh',8458,7648.88,'vsnus')

statement ok
INSERT INTO tab0 VALUES(872,5600,8038.30,'sjjge',4673,7326.77,'yumsd')

statement ok
INSERT INTO tab0 VALUES(873,3559,9866.58,'uhojx',9159,4225.38,'rwxge')

statement ok
INSERT INTO tab0 VALUES(874,5078,87.12,'fdmkx',583,1412.8,'eghve')

statement ok
INSERT INTO tab0 VALUES(875,784,5870.5,'vdxxf',8097,485.66,'dgdnp')

statement ok
INSERT INTO tab0 VALUES(876,4921,3557.76,'ortuq',7596,2828.99,'etiok')

statement ok
INSERT INTO tab0 VALUES(877,1593,9892.49,'vghcw',7961,862.63,'sscax')

statement ok
INSERT INTO tab0 VALUES(878,1020,3282.1,'nrhkq',7310,9291.45,'jceoz')

statement ok
INSERT INTO tab0 VALUES(879,5783,5041.21,'ebzqw',9251,6356.26,'vgwui')

statement ok
INSERT INTO tab0 VALUES(880,7078,647.72,'ydjrt',9129,5080.42,'dpkdj')

statement ok
INSERT INTO tab0 VALUES(881,964,986.56,'zegwu',280,4526.50,'tftio')

statement ok
INSERT INTO tab0 VALUES(882,4613,3606.74,'kijsd',274,1039.89,'rqxbl')

statement ok
INSERT INTO tab0 VALUES(883,2558,3962.91,'knomm',7174,9498.40,'rgbqu')

statement ok
INSERT INTO tab0 VALUES(884,2950,6671.98,'mvwzi',5776,4571.90,'faqpp')

statement ok
INSERT INTO tab0 VALUES(885,3800,6762.69,'mhqxe',3870,1956.97,'uekil')

statement ok
INSERT INTO tab0 VALUES(886,9233,920.35,'smpqk',8480,9642.50,'vtsrl')

statement ok
INSERT INTO tab0 VALUES(887,5738,8745.34,'caexp',2119,9150.31,'agmom')

statement ok
INSERT INTO tab0 VALUES(888,8415,3992.82,'jklgr',9042,6785.79,'lflpe')

statement ok
INSERT INTO tab0 VALUES(889,7231,5831.32,'clfww',1485,4890.51,'kxewf')

statement ok
INSERT INTO tab0 VALUES(890,3262,89.38,'uzkaa',5929,3091.88,'yyhih')

statement ok
INSERT INTO tab0 VALUES(891,4952,504.15,'fovtm',9342,9550.73,'swnnj')

statement ok
INSERT INTO tab0 VALUES(892,4353,9974.4,'eyrpo',5427,2214.94,'evvhs')

statement ok
INSERT INTO tab0 VALUES(893,3385,1809.35,'btiai',878,66.11,'seqfc')

statement ok
INSERT INTO tab0 VALUES(894,80,975.87,'pfqez',1627,4372.14,'ubwkv')

statement ok
INSERT INTO tab0 VALUES(895,4829,5353.51,'fqbpt',9095,9936.87,'adycy')

statement ok
INSERT INTO tab0 VALUES(896,6642,6980.94,'bctog',6438,6345.71,'jcggi')

statement ok
INSERT INTO tab0 VALUES(897,9392,3309.20,'dbqzl',4804,9222.24,'fixbw')

statement ok
INSERT INTO tab0 VALUES(898,8061,8442.36,'lbryv',9578,1475.88,'mpcuk')

statement ok
INSERT INTO tab0 VALUES(899,4997,4404.8,'efvho',6632,4547.48,'tqdhe')

statement ok
INSERT INTO tab0 VALUES(900,6162,2373.29,'pyhfo',8579,9667.94,'mqqqp')

statement ok
INSERT INTO tab0 VALUES(901,7380,3939.79,'oonaq',1016,9097.88,'aemrr')

statement ok
INSERT INTO tab0 VALUES(902,587,2352.90,'gwmqq',8784,7387.20,'tmfvx')

statement ok
INSERT INTO tab0 VALUES(903,86,3771.3,'nvsip',2208,2621.77,'gjxon')

statement ok
INSERT INTO tab0 VALUES(904,8791,8517.39,'zekdt',3352,7667.0,'mrlbq')

statement ok
INSERT INTO tab0 VALUES(905,4478,2319.53,'daquu',2269,1429.51,'rznwh')

statement ok
INSERT INTO tab0 VALUES(906,6038,7570.2,'jmjcx',3251,97.71,'ucymp')

statement ok
INSERT INTO tab0 VALUES(907,9725,4568.4,'ikrqy',1076,8396.48,'iypzy')

statement ok
INSERT INTO tab0 VALUES(908,5408,2774.4,'vvvkp',7262,834.74,'swqsz')

statement ok
INSERT INTO tab0 VALUES(909,2815,2061.34,'aqimh',6317,2957.11,'goowx')

statement ok
INSERT INTO tab0 VALUES(910,7791,3958.76,'rjhpq',1981,2594.52,'glmbk')

statement ok
INSERT INTO tab0 VALUES(911,8543,2907.25,'bnhqj',3490,1110.84,'hkaay')

statement ok
INSERT INTO tab0 VALUES(912,1891,7510.52,'foppj',1814,4657.51,'flrfv')

statement ok
INSERT INTO tab0 VALUES(913,9520,1776.63,'bfjvc',5911,5418.15,'hgslg')

statement ok
INSERT INTO tab0 VALUES(914,4225,6379.67,'ohqzl',8980,2865.21,'jwnmq')

statement ok
INSERT INTO tab0 VALUES(915,6168,9901.35,'sdnjy',4705,5003.56,'yhnrm')

statement ok
INSERT INTO tab0 VALUES(916,9121,83.62,'unltx',3510,9623.16,'jamrr')

statement ok
INSERT INTO tab0 VALUES(917,6723,5078.32,'mzzap',8809,5551.62,'xwnvo')

statement ok
INSERT INTO tab0 VALUES(918,8720,4698.79,'wqgoh',3250,3898.90,'vkgdd')

statement ok
INSERT INTO tab0 VALUES(919,7229,1665.49,'utyei',4975,9311.89,'qrxnz')

statement ok
INSERT INTO tab0 VALUES(920,2977,2862.64,'shlml',3856,1222.4,'dvosq')

statement ok
INSERT INTO tab0 VALUES(921,1550,582.83,'zryss',7763,5094.99,'prcvo')

statement ok
INSERT INTO tab0 VALUES(922,9654,3780.66,'aiofs',8796,3683.20,'ujkwg')

statement ok
INSERT INTO tab0 VALUES(923,1356,9311.35,'iinbt',9550,8274.55,'dcomm')

statement ok
INSERT INTO tab0 VALUES(924,4334,3207.37,'ozhwu',9353,9939.2,'zmwnj')

statement ok
INSERT INTO tab0 VALUES(925,6132,1899.8,'xnnev',526,786.25,'mmzrk')

statement ok
INSERT INTO tab0 VALUES(926,408,3479.32,'wsleu',4621,5206.20,'yuedv')

statement ok
INSERT INTO tab0 VALUES(927,3177,8953.8,'tjvmu',8688,1129.77,'uqgdk')

statement ok
INSERT INTO tab0 VALUES(928,9075,3483.9,'mgblf',1508,1265.44,'ugusx')

statement ok
INSERT INTO tab0 VALUES(929,7599,7644.47,'xhmaw',9852,4364.86,'cqhdx')

statement ok
INSERT INTO tab0 VALUES(930,9356,8669.31,'vqija',4731,6458.50,'nbhtf')

statement ok
INSERT INTO tab0 VALUES(931,2321,1704.36,'exvir',7642,2115.28,'kvimt')

statement ok
INSERT INTO tab0 VALUES(932,8078,1052.61,'demhn',6822,5689.16,'auijv')

statement ok
INSERT INTO tab0 VALUES(933,8077,1429.92,'hfdnf',3497,2568.82,'nxplf')

statement ok
INSERT INTO tab0 VALUES(934,9881,9208.67,'slbal',8127,425.22,'ddxdg')

statement ok
INSERT INTO tab0 VALUES(935,3416,3676.34,'akada',6666,470.30,'htqzz')

statement ok
INSERT INTO tab0 VALUES(936,7221,8280.20,'qkqzz',6802,8697.53,'qfzfi')

statement ok
INSERT INTO tab0 VALUES(937,4697,1867.85,'vbfiv',4213,7134.59,'iqzkc')

statement ok
INSERT INTO tab0 VALUES(938,362,9961.94,'xcaap',8815,5097.96,'bkteh')

statement ok
INSERT INTO tab0 VALUES(939,6093,5985.71,'hwntz',1013,433.57,'gzmrb')

statement ok
INSERT INTO tab0 VALUES(940,7383,2523.66,'oaepi',7949,778.41,'ahqhx')

statement ok
INSERT INTO tab0 VALUES(941,9588,2573.19,'tazqb',9754,4802.16,'opagt')

statement ok
INSERT INTO tab0 VALUES(942,4176,791.17,'jxeqn',5798,3460.81,'yvbgf')

statement ok
INSERT INTO tab0 VALUES(943,4981,4742.61,'nutrf',938,1241.31,'qnscm')

statement ok
INSERT INTO tab0 VALUES(944,4444,6152.29,'brhsy',1769,7421.14,'eetis')

statement ok
INSERT INTO tab0 VALUES(945,9878,2160.33,'uweew',2876,7311.0,'tcutx')

statement ok
INSERT INTO tab0 VALUES(946,9703,6560.89,'jswtt',3081,3504.39,'zvngh')

statement ok
INSERT INTO tab0 VALUES(947,9705,1061.72,'reuwx',1694,7135.25,'fyrge')

statement ok
INSERT INTO tab0 VALUES(948,9282,6528.86,'unojn',4767,9115.58,'nnijh')

statement ok
INSERT INTO tab0 VALUES(949,7696,4240.20,'fwdcq',1330,1397.9,'bywdu')

statement ok
INSERT INTO tab0 VALUES(950,321,7133.14,'wyaqh',9666,8218.79,'wihnk')

statement ok
INSERT INTO tab0 VALUES(951,4898,1055.21,'hmcts',3535,7348.62,'ihdpr')

statement ok
INSERT INTO tab0 VALUES(952,7776,6014.68,'wcdek',8440,1345.9,'zotcu')

statement ok
INSERT INTO tab0 VALUES(953,171,7427.22,'utpeg',7872,8800.92,'ymjfc')

statement ok
INSERT INTO tab0 VALUES(954,4411,7372.47,'ekydr',9559,5588.62,'uuday')

statement ok
INSERT INTO tab0 VALUES(955,187,5838.74,'jttht',1716,6123.14,'uenew')

statement ok
INSERT INTO tab0 VALUES(956,805,1973.54,'xprfi',9300,4602.52,'axbru')

statement ok
INSERT INTO tab0 VALUES(957,9432,5334.30,'stvnd',255,6257.46,'omvfb')

statement ok
INSERT INTO tab0 VALUES(958,5349,7141.92,'foqoz',1761,8604.98,'kcgbr')

statement ok
INSERT INTO tab0 VALUES(959,202,1523.47,'kkfxt',6489,2698.6,'xqpnh')

statement ok
INSERT INTO tab0 VALUES(960,4247,6768.72,'boiaq',8849,4374.22,'ovrrj')

statement ok
INSERT INTO tab0 VALUES(961,2607,6575.85,'vqoxg',1233,5507.18,'kgixl')

statement ok
INSERT INTO tab0 VALUES(962,1516,1304.3,'ielts',279,5254.48,'ztioa')

statement ok
INSERT INTO tab0 VALUES(963,4571,9223.7,'jnoou',3111,1309.9,'merqw')

statement ok
INSERT INTO tab0 VALUES(964,8983,7763.2,'xetra',3587,1149.5,'bxnhd')

statement ok
INSERT INTO tab0 VALUES(965,1827,4762.80,'hpcmx',8560,3289.8,'oplnj')

statement ok
INSERT INTO tab0 VALUES(966,6291,9722.84,'lrgaw',4116,8526.22,'cppxt')

statement ok
INSERT INTO tab0 VALUES(967,7923,5052.71,'uohso',1832,1196.47,'wxqzc')

statement ok
INSERT INTO tab0 VALUES(968,891,7321.72,'pbbtw',5023,415.35,'kkcce')

statement ok
INSERT INTO tab0 VALUES(969,4219,8510.55,'ylgyc',4990,1340.62,'qviwq')

statement ok
INSERT INTO tab0 VALUES(970,8050,7261.89,'xylfb',7275,6408.81,'beszg')

statement ok
INSERT INTO tab0 VALUES(971,9888,121.6,'rnjhg',7908,6596.91,'wjuqz')

statement ok
INSERT INTO tab0 VALUES(972,848,2805.62,'uhwix',9656,2661.99,'czigd')

statement ok
INSERT INTO tab0 VALUES(973,1913,1431.3,'zjeme',8276,6399.78,'qtmem')

statement ok
INSERT INTO tab0 VALUES(974,176,3781.22,'ohypg',8760,1890.34,'mhxej')

statement ok
INSERT INTO tab0 VALUES(975,5093,546.28,'owjtm',1593,6748.66,'upyby')

statement ok
INSERT INTO tab0 VALUES(976,4736,6578.2,'onkzk',8994,3375.31,'aurla')

statement ok
INSERT INTO tab0 VALUES(977,2037,2138.65,'ddtvc',1173,204.4,'dszfo')

statement ok
INSERT INTO tab0 VALUES(978,1045,4149.38,'wqdaa',2577,2958.42,'omxim')

statement ok
INSERT INTO tab0 VALUES(979,3168,7441.3,'jphbk',8141,5891.44,'kemgy')

statement ok
INSERT INTO tab0 VALUES(980,3432,6005.69,'ovfue',9540,3044.18,'rocsm')

statement ok
INSERT INTO tab0 VALUES(981,6721,4246.33,'szeef',1435,9494.39,'flmgc')

statement ok
INSERT INTO tab0 VALUES(982,4037,8225.39,'pjxvk',7727,2601.15,'zpwxj')

statement ok
INSERT INTO tab0 VALUES(983,8743,6915.48,'qgrqs',7805,3262.48,'ceeum')

statement ok
INSERT INTO tab0 VALUES(984,2816,3448.94,'xwnhn',2796,7158.93,'zlbtq')

statement ok
INSERT INTO tab0 VALUES(985,3093,6367.47,'lrgfm',1815,6006.30,'lizaw')

statement ok
INSERT INTO tab0 VALUES(986,7532,875.81,'fhqvw',8216,8807.15,'zdosk')

statement ok
INSERT INTO tab0 VALUES(987,1446,88.31,'jumko',5840,3099.50,'qdkqn')

statement ok
INSERT INTO tab0 VALUES(988,3764,6270.55,'gtxpe',5268,6204.44,'icmsa')

statement ok
INSERT INTO tab0 VALUES(989,3282,2114.1,'olfdh',68,6479.74,'xccuc')

statement ok
INSERT INTO tab0 VALUES(990,4906,1960.68,'oufin',3046,4566.50,'fzdfl')

statement ok
INSERT INTO tab0 VALUES(991,5903,3063.66,'cthwo',2499,4642.73,'jcyho')

statement ok
INSERT INTO tab0 VALUES(992,2159,6835.81,'scohl',8459,5672.4,'stweu')

statement ok
INSERT INTO tab0 VALUES(993,4418,101.35,'wbwqo',8947,2651.20,'cdyik')

statement ok
INSERT INTO tab0 VALUES(994,4540,6345.54,'atbew',6223,7864.82,'wqhfu')

statement ok
INSERT INTO tab0 VALUES(995,1748,6962.22,'tbcyx',3036,2130.92,'bpfun')

statement ok
INSERT INTO tab0 VALUES(996,8877,2507.51,'xabjc',8872,6048.60,'ocdin')

statement ok
INSERT INTO tab0 VALUES(997,8628,3942.42,'oslxr',6206,1261.66,'deaxd')

statement ok
INSERT INTO tab0 VALUES(998,9813,3893.42,'mobrx',6750,1691.20,'qyhdj')

statement ok
INSERT INTO tab0 VALUES(999,3606,7896.14,'xfkch',7705,4797.27,'btyqx')

query I rowsort label-2380
SELECT DISTINCT - + COUNT ( col4 ) FROM tab0 AS cor0 WHERE NOT 8 IS NULL
----
-1000
^D

thread 'worker thread 2' panicked at 'Datum::unwrap_float64 called on Int64(1)', src/materialize/repr.rs:109:18
stack backtrace:
   0:     0x5595e22a7103 - std::sys::unix::backtrace::tracing::imp::unwind_backtrace::hc7b76c57e25c7d91
                               at src/libstd/sys/unix/backtrace/tracing/gcc_s.rs:39
   1:     0x5595e22a1d5b - std::sys_common::backtrace::_print::h203699a8f1b8b948
                               at src/libstd/sys_common/backtrace.rs:70
   2:     0x5595e22a4e66 - std::panicking::default_hook::{{closure}}::h74fcb67c61dc8b92
                               at src/libstd/sys_common/backtrace.rs:58
                               at src/libstd/panicking.rs:200
   3:     0x5595e22a4be4 - std::panicking::default_hook::hb2e06978a2102930
                               at src/libstd/panicking.rs:215
   4:     0x5595e22a556f - std::panicking::rust_panic_with_hook::h4ac0b487eb7e54da
                               at src/libstd/panicking.rs:478
   5:     0x5595e22a50f1 - std::panicking::continue_panic_fmt::h23ca3ed77b25f368
                               at src/libstd/panicking.rs:385
   6:     0x5595e22a503e - std::panicking::begin_panic_fmt::he14137966dec33cc
                               at src/libstd/panicking.rs:340
   7:     0x5595e15632e8 - materialize::repr::Datum::unwrap_float64::ha202e6ac120b42c0
                               at src/materialize/repr.rs:109
   8:     0x5595e1422010 - materialize::dataflow::func::neg_float64::h5d16d9b92fb7a3b3
                               at src/materialize/dataflow/func.rs:211
   9:     0x5595e1771c02 - materialize::dataflow::render::eval_expr::h7328560a17a0733e
                               at src/materialize/dataflow/render.rs:260
  10:     0x5595e1760db6 - materialize::dataflow::render::build_plan::{{closure}}::{{closure}}::h5133bddfcaf42494
                               at src/materialize/dataflow/render.rs:99
  11:     0x5595e12a0318 - <core::iter::adapters::Map<I, F> as core::iter::traits::iterator::Iterator>::fold::{{closure}}::h3eefbf837ec9d100
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/iter/adapters/mod.rs:587
  12:     0x5595e11b4889 - <core::slice::Iter<'a, T> as core::iter::traits::iterator::Iterator>::fold::hf1c51c2ce9b586ff
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/slice/mod.rs:3003
  13:     0x5595e129d86d - <core::iter::adapters::Map<I, F> as core::iter::traits::iterator::Iterator>::fold::h46deeb93a0a96833
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/iter/adapters/mod.rs:587
  14:     0x5595e12b0b44 - core::iter::traits::iterator::Iterator::for_each::h809e32d290c93bb0
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/iter/traits/iterator.rs:604
  15:     0x5595e13b1b03 - <alloc::vec::Vec<T> as alloc::vec::SpecExtend<T, I>>::spec_extend::h9ce085069d27b5c0
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/vec.rs:1852
  16:     0x5595e13ba30a - <alloc::vec::Vec<T> as alloc::vec::SpecExtend<T, I>>::from_iter::hb90bf236cfe4b912
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/vec.rs:1835
  17:     0x5595e13c1cfd - <alloc::vec::Vec<T> as core::iter::traits::collect::FromIterator<T>>::from_iter::h04e5bc305d509b35
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/vec.rs:1721
  18:     0x5595e12af8b0 - core::iter::traits::iterator::Iterator::collect::h4ad30aa98da2afc6
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/iter/traits/iterator.rs:1465
  19:     0x5595e1760e85 - materialize::dataflow::render::build_plan::{{closure}}::ha72fa73fb9dd32c9
                               at src/materialize/dataflow/render.rs:99
  20:     0x5595e14ea94c - <differential_dataflow::collection::Collection<G, D, R>>::map::{{closure}}::h2c9873f2fe23e119
                               at /home/jamie/.cargo/git/checkouts/differential-dataflow-d065d23d797aa027/3773f17/src/collection.rs:83
  21:     0x5595e13e198f - <timely::dataflow::stream::Stream<S, D> as timely::dataflow::operators::map::Map<S, D>>::map::{{closure}}::{{closure}}::{{closure}}::{{closure}}::ha1e24b5aebc33c11
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/map.rs:57
  22:     0x5595e13e1cef - core::ops::function::impls::<impl core::ops::function::FnOnce<A> for &mut F>::call_once::h55195cc66962a499
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/ops/function.rs:279
  23:     0x5595e12595e2 - <core::option::Option<T>>::map::hba3c9b44b54f41d5
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/option.rs:414
  24:     0x5595e12a4cc3 - <core::iter::adapters::Map<I, F> as core::iter::traits::iterator::Iterator>::next::h57be63e519468cfc
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/iter/adapters/mod.rs:568
  25:     0x5595e12d03a7 - <timely::dataflow::channels::pushers::buffer::Session<'a, T, D, P>>::give_iterator::hd386fe268a5740ee
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/channels/pushers/buffer.rs:105
  26:     0x5595e13e158c - <timely::dataflow::stream::Stream<S, D> as timely::dataflow::operators::map::Map<S, D>>::map::{{closure}}::{{closure}}::{{closure}}::he9900267fd3b9748
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/map.rs:57
  27:     0x5595e15516b6 - <timely::dataflow::operators::generic::handles::InputHandle<T, D, P>>::for_each::ha5b5a4a0e2c14c18
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/generic/handles.rs:82
  28:     0x5595e13e0df4 - <timely::dataflow::stream::Stream<S, D> as timely::dataflow::operators::map::Map<S, D>>::map::{{closure}}::{{closure}}::h82be47e86c5585df
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/map.rs:55
  29:     0x5595e143de9c - <timely::dataflow::stream::Stream<G, D1> as timely::dataflow::operators::generic::operator::Operator<G, D1>>::unary::{{closure}}::{{closure}}::h3e0dc04604957c83
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/generic/operator.rs:402
  30:     0x5595e15fc679 - <timely::dataflow::operators::generic::builder_rc::OperatorBuilder<G>>::build::{{closure}}::h75efa4dcb21547ce
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/generic/builder_rc.rs:136
  31:     0x5595e1491e82 - <timely::dataflow::operators::generic::builder_raw::OperatorCore<T, PEP, PIP> as timely::scheduling::Schedule>::schedule::hb81e829325d08a75
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/dataflow/operators/generic/builder_raw.rs:219
  32:     0x5595e173a81e - <timely::progress::subgraph::PerOperatorState<T>>::schedule::hebfc4f0fde4b8d41
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:646
  33:     0x5595e173b157 - <timely::progress::subgraph::Subgraph<TOuter, TInner>>::activate_child::h9ae5d7849b54e217
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:329
  34:     0x5595e173630b - <timely::progress::subgraph::Subgraph<TOuter, TInner> as timely::scheduling::Schedule>::schedule::hcacdec1bef1082b3
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/progress/subgraph.rs:295
  35:     0x5595e1f2ced4 - timely::worker::Wrapper::step::{{closure}}::h971ad292bc084c03
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:489
  36:     0x5595e1f2bdcd - <core::option::Option<T>>::map::hcd877f91cd6d63f8
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/option.rs:414
  37:     0x5595e1f2ccf2 - timely::worker::Wrapper::step::h079de6881be60958
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:489
  38:     0x5595e147fbc0 - <timely::worker::Worker<A>>::step_or_park::ha3cf7ccb3a665639
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:227
  39:     0x5595e14828c2 - <timely::worker::Worker<A>>::step::hb691f6f1f3a3f589
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/worker.rs:144
  40:     0x5595e1424a30 - <materialize::dataflow::server::Worker<'w, A>>::run::hda8e00cce02a56ae
                               at src/materialize/dataflow/server.rs:129
  41:     0x5595e14243aa - materialize::dataflow::server::serve::{{closure}}::h6bc7e17ac31d0bce
                               at src/materialize/dataflow/server.rs:47
  42:     0x5595e1774840 - timely::execute::execute::{{closure}}::h1f93346fe3e1bc0d
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/timely/src/execute.rs:201
  43:     0x5595e1442709 - timely_communication::initialize::initialize_from::{{closure}}::h1252acab1e6178d6
                               at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/ef2f213/communication/src/initialize.rs:269
  44:     0x5595e156ffe3 - std::sys_common::backtrace::__rust_begin_short_backtrace::h12d7320851696d2d
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/sys_common/backtrace.rs:135
  45:     0x5595e156c9e5 - std::thread::Builder::spawn_unchecked::{{closure}}::{{closure}}::hdb80b6cb28504035
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/thread/mod.rs:469
  46:     0x5595e17a1835 - <std::panic::AssertUnwindSafe<F> as core::ops::function::FnOnce<()>>::call_once::h6a208203b39e18a3
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panic.rs:309
  47:     0x5595e177aa39 - std::panicking::try::do_call::hc0a910baaa414789
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:297
  48:     0x5595e22aa729 - __rust_maybe_catch_panic
                               at src/libpanic_unwind/lib.rs:87
  49:     0x5595e177a91f - std::panicking::try::h2f37f341d87162d4
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:276
  50:     0x5595e17a1875 - std::panic::catch_unwind::hc732feefc72cf84c
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panic.rs:388
  51:     0x5595e156c7af - std::thread::Builder::spawn_unchecked::{{closure}}::h6e0ecf038c3a03fe
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/thread/mod.rs:468
  52:     0x5595e156cab7 - <F as alloc::boxed::FnBox<A>>::call_box::ha04220c2aedddf29
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/boxed.rs:749
  53:     0x5595e22a9acd - std::sys::unix::thread::Thread::new::thread_start::h7fc2d82a3b91bc88
                               at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/boxed.rs:759
                               at src/libstd/sys_common/thread.rs:14
                               at src/libstd/sys/unix/thread.rs:81
  54:     0x7f96948c3ef6 - start_thread
  55:     0x7f96945e322e - __GI___clone
  56:                0x0 - <unknown>

Control plane consistency

Metastore currently stores its data in ZooKeeper, which is... not great. The psuedo-filesystem that ZooKeeper chooses to expose makes life difficult when you want a strictly ordered stream of events across the whole filesystem, not just across one file or one directory. We're suffering at the moment because PEEK is not properly sequenced with CREATE DATAFLOW*, and therefore differential often sees a PEEK foo command come through before it sees the corresponding CREATE DATAFLOW foo ... command. Not to mention that we globally sequence things twice! ZooKeeper does one round of consensus, and then we do another round of exchanging with differential's sequencer.

The question, then, is what would be a suitable replacement for ZooKeeper? @frankmcsherry and @jamii have both proposed using a single-partition Kafka topic. For reasons I'll describe below, I'm not sold on that particular solution. Let me start by listing out what I think we need out of our metadata storage layer.

  • Consistency. We need to make sure that a dataflow D cannot be deleted while there exists another dataflow that depends on D. This effectively means that the metadata layer needs to support transactions, or that there is some means of electing a leader who is the sole arbiter of metadata updates.

  • Sequencing. The metadata storage should be able to produce an ordered log of all operations, including read commands, like PEEK, and write commands, like CREATE DATAFLOW.

  • Durability. If the server goes down, rebooting materialized should, at the very least, reinstall all dataflows that were previously installed. This is a prerequisite for durably storing computation state. Ideally, the metadata layer would be selectively durable. It doesn't need to store old versions of dataflows, for example, nor does it need to remember read-only commands.

A single-partition Kafka topic gives us sequencing and durability. @jamii mentioned that clever use of keys might even allow Kafka to garbage-collect old versions of dataflows and all but the last PEEK command, allowing Kafka to meet the stricter goal of selective durability.

The problem with a single-partition Kafka topic, AFAICT, is maintaining consistency. You can either validate consistency on the way in, or the way out. I think performing validation on the way in is totally doomed, because Kafka has no write-only-if-nothing-changed primitive. Put another way, let's say that two users simultaneously execute CREATE DATAFLOW bar SELECT * FROM foo; and DROP DATAFLOW bar;. Only one of these commands can succeed. Either the CREATE wins and the DROP fails, because bar is now depended upon by foo, or the DROP wins and the CREATE fails because bar no longer exists. But if you have two clients performing validation simultaneously, they won't know about the other command yet, and so they'll both decide that the command is valid and push the computed action into Kafka, where the computed action is either "definitely build this dataflow" or "definitely drop this dataflow", corresponding to today's dataflow::Command struct. And now whatever's reading from Kafka on the other end will explode.

So what about performing validation on the way out? In this world I think you push raw SQL into the Kafka topic. Then whatever reads from Kafka parses and validates the SQL, potentially rejecting the command with an error. This seems fundamentally viable, except that the durability is in the wrong place. When materialized crashes and you turn it back on, you'll have all these busted queries sitting in the Kafka topic that you have to run through. It's also not clear how you would use Kafka's automatic garbage collection in this model; you wouldn't want CREATE DATAFLOW foo AS <valid>; CREATE DATAFLOW foo AS <invalid> to only remember the invalid command (because it's later)!

There's also the problem that Kafka's strength is not minimizing latency, it's maximizing throughput, so pushing commands through Kafka in the first place might add unacceptable latency to the interactive experience.

FWIW, I fully accept that I may be suffering from insufficient imagination here, and I would love for someone to point out flaws in my reasoning! :)

So what does satisfy all three properties? As far as I can tell, both etcd3 and CockroachDB do, and not much else. CockroachDB isn't a great fit, given how resource hungry it is. etcd3 is actually intended for exactly our desired use case, but there aren't Rust bindings for its v3 API (and the v2 API doesn't actually meet our needs) and it's yet another service that would need to be deployed alongside materialized.

I'm pretty unsure of what the upshot is. One nuclear option is that we could punt on this problem until later, when we tackle active–active replication. That would allow us to rip out the ZooKeeper goop, and get a super-fast in-memory metadata store for tests. If we needed durability in the short term, we could just have one of the workers be responsible for saving metadata updates to disk.

Other, less nuclear options involve:

  • Continuing to hack our way around ZooKeeper. I think this is workable, but
    painful.

  • Switching to etcd3, with the intention of one day bundling it with
    materialized itself.

*I am apparently using CREATE DATAFLOW to mean either CREATE MATERIALIZED VIEW or CREATE DATA SOURCE.

dataflow: PEEK consistency

PEEK doesn't do anything to make sure that it's seeing a consistent snapshot of the data. It needs to figure out what timestamp the sequencer assigned it, wait for the referenced dataflow to catch up to that timestamp, and then slice the arrangement at that timestamp.�

Duplicate rows on first insert

Do stuff in this order:

cargo run

kafka-avro-console-producer --topic bug --broker-list localhost:9092 --property value.schema='{"type": "record", "name": "na", "fields": [{"name": "c", "type": "float"}]}'
{"c": 0.9}

CREATE DATA SOURCE bug FROM 'kafka://localhost/bug' USING SCHEMA '{"type": "record", "name": "na", "fields": [{"name": "c", "type": "float"}]}';
CREATE DATA SOURCE

peek bug;
  c  
-----
 0.9
 0.9
(2 rows)

On restart the duplicate row disappears.

Conditions on outer joins should only cause rows to be filtered on their outer side

postgres=# create table foo(a integer);
CREATE TABLE
postgres=# create table bar(a integer);
CREATE TABLE
postgres=# insert into foo values (1);
INSERT 0 1
postgres=# insert into foo values (2);
INSERT 0 1
postgres=# insert into bar values (3);
INSERT 0 1
postgres=# insert into bar values (4);
INSERT 0 1

postgres=# select * from foo left join bar on 1 = 0;
 a | a 
---+---
 1 |  
 2 |  
(2 rows)

postgres=# select * from foo right join bar on 1 = 0;
 a | a 
---+---
   | 3
   | 4
(2 rows)

postgres=# select * from foo full outer join bar on 1 = 0;
 a | a 
---+---
   | 3
   | 4
 1 |  
 2 |  
(4 rows)

testing: jepsen

Materialize will make specific claims about correctness. We should think carefully about what correctness claims we will be making, and have them be independently verified by Jepsen.io.

The lead time on Jepsen tests is 6-9 months, and the testing process takes ~2-3 months, depending on how much prep work we do ahead of time. Kyle mentioned the following prep items that make it easier on him:

As for prep work, yes, concrete claims are great--if you don't have them, I'll
ask you what it's supposed to do, and say ".. but X's engineers say the system
does Y and Z...". The best investments you can make before Jepsen testing are:

  1. Have a JVM client in place, or some sort of HTTP/JSON-alike interface--the
    more standardized it is, the easier my job will be getting Jepsen to talk to
    your system.

  2. Have a simple, robust, and repeatable cluster install and join process.
    Looooots of databases have race conditions or crashes that pop up when you start
    doing repeated/concurrent cluster setup, and adding workarounds to make that
    process reliable can eat up a lot of time.

  3. Documentation! This can be public or private, but the more I can read up
    front, the better. If you don't have this, no worries--I can ask lots of
    questions in chat, if you're open to it. :)

Oh, keep in mind that you'll also have an embargo period--once we finish the
report, you'll have up to 3 months to defer publication if you'd like to get new
releases in order, fix bugs, etc, and I'll come back to make small updates to
the report just prior to publishing. Totally optional; we can also publish
immediately if you like. :)

timely: allocating IDs serially vs namespacing IDs

Consider this scenario (that @rjnn and I just discussed):

  1. User creates a new dataflow frazzled. The dataflow is stored in ZK at /materialize/dataflows/frazzled and is assigned a czxid ("creation zookeeper transaction id").
  2. ZK sends change notifications to every node watching the directory. If multiple dataflows were created, no big deal, since each node can discover the order of creation by looking at the czxid.
  3. User drops frazzled.
  4. User creates a new frazzled.

ZK does not expose a changefeed, so it's possible for a node to only see the later definition of frazzled and not the first. That node will not allocate channel IDs for the old version, and will fall out of sync on channel IDs.

For now I'm planning to solve this by never letting users drop a dataflow, which is... not ideal. One potential workaround is to, upon deleting a dataflow, store how many channel IDs it had allocated, so that future nodes could simply waste that many channel IDs. But that seems like a lot of work for a clunky workaround. Perhaps it's time to consider namespacing channel IDs in Timely?

sqllogictest output failure on left inequality join

From test/random/groupby/slt_good_1.test:

SELECT DISTINCT cor0.col2 AS col0 FROM tab0 AS cor0 LEFT JOIN tab1 ON NOT NULL < - cor0.col2 + cor0.col2 GROUP BY cor0.col2
OutputFailure { expected_output: Values(["24", "38", "79"]), actual_output: [] }

server: support TAIL

The MVP will support two commands: PEEK and TAIL. PEEK shows the rolled up state of a dataflow at this moment in time. TAIL should show a continuous stream of updates as the stream evolved. At the moment only PEEK is supported.

Mux doesn't get poisoned correctly

In this sequence:

pgwire installs receiver for connection_uuid in PeekResultsMux
pgwire sends PEEK
pgwire blocks on receiver

dataflow panics before receiving peek

The PeekResultsMux is poisoned, but pgwire is already waiting on the receiver. Noone was holding the sender yet, so the receiver itself isn't poisoned.

This means that all the workers go down but materialized stays up in zombie mode, rather than exiting and being restarted by some supervisor.

Panic on division by 0

Sqlite returns null. Postgres returns an error. Either way, materialize should stay up.

Active/active replication

To start, Materialized will require a hardcoded list of nodes in the cluster. The future API should perform cluster bootstrapping at the Materialized level, but not start any Timely clusters until the user acknowledges that the node membership is reasonably fixed (so we don't waste time tearing down/spinning up clusters for every node as nodes join).

Planner produces wrong column number

jamie=> create table foo (a integer);
CREATE TABLE
jamie=> insert into foo values (null), (1);
INSERT 0 2
jamie=> select * from foo join foo using (a);
 thread: worker thread 1
message: index out of bounds: the len is 2 but the index is 2
stack backtrace:
   0: materialized::handle_panic::h101bb2cee025e5e3 (0x5642c61adb26)
             at src/materialize/bin/materialized.rs:65
   1: core::ops::function::Fn::call::hf93d75db9219b4ff (0x5642c61aa968)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/ops/function.rs:69
   2: rust_panic_with_hook (0x5642c78286c9)
             at src/libstd/panicking.rs:482
   3: continue_panic_fmt (0x5642c7828162)
             at src/libstd/panicking.rs:385
   4: rust_begin_unwind (0x5642c7828046)
   5: panic_fmt (0x5642c784129d)
             at src/libcore/panicking.rs:85
   6: panic_bounds_check (0x5642c7841255)
             at src/libcore/panicking.rs:61
   7: <usize as core::slice::SliceIndex<[T]>>::index::h08bb0a0bcf6cf780 (0x5642c74600fe)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/slice/mod.rs:2539
   8: core::slice::<impl core::ops::index::Index<I> for [T]>::index::h95eb29ec98901dd5 (0x5642c745fe28)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libcore/slice/mod.rs:2396
   9: <alloc::vec::Vec<T> as core::ops::index::Index<I>>::index::hc0b84d80b059c765 (0x5642c74604fb)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/vec.rs:1677
  10: materialize::dataflow::transform::predicate_pushdown::PredicatePushdown::action::{{closure}}::h00b7a85a278de55a (0x5642c69bbbe3)
             at src/materialize/dataflow/transform.rs:260
  11: materialize::dataflow::optimize::<impl materialize::dataflow::types::ScalarExpr>::visit_inner::h3f2dc51f85d933d3 (0x5642c68ac964)
             at src/materialize/dataflow/optimize.rs:45
  12: materialize::dataflow::optimize::<impl materialize::dataflow::types::ScalarExpr>::visit_inner::{{closure}}::h35b4ddb92d0ec6b4 (0x5642c6bbcbf4)
             at src/materialize/dataflow/optimize.rs:44
  13: materialize::dataflow::optimize::<impl materialize::dataflow::types::ScalarExpr>::visit1::hf70305fbd0f3d3bb (0x5642c68ac74d)
             at src/materialize/dataflow/optimize.rs:25
  14: materialize::dataflow::optimize::<impl materialize::dataflow::types::ScalarExpr>::visit_inner::h3f2dc51f85d933d3 (0x5642c68ac948)
             at src/materialize/dataflow/optimize.rs:44
  15: materialize::dataflow::optimize::<impl materialize::dataflow::types::ScalarExpr>::visit::haaffec1f37bc5a6b (0x5642c68ac994)
             at src/materialize/dataflow/optimize.rs:52
  16: materialize::dataflow::transform::predicate_pushdown::PredicatePushdown::action::h34268baf3e59a8cd (0x5642c6c119a5)
             at src/materialize/dataflow/transform.rs:258
  17: materialize::dataflow::transform::predicate_pushdown::PredicatePushdown::transform::{{closure}}::h197f1186d22e46f8 (0x5642c69bbac2)
             at src/materialize/dataflow/transform.rs:227
  18: materialize::dataflow::optimize::<impl materialize::dataflow::types::RelationExpr>::visit_mut_inner_pre::hb267e3f81e7bc0e0 (0x5642c68aef27)
             at src/materialize/dataflow/optimize.rs:289
  19: materialize::dataflow::optimize::<impl materialize::dataflow::types::RelationExpr>::visit_mut_inner_pre::{{closure}}::h360423791a7703bd (0x5642c6bbd124)
             at src/materialize/dataflow/optimize.rs:290
  20: materialize::dataflow::optimize::<impl materialize::dataflow::types::RelationExpr>::visit1_mut::h0ce8504f6e5dcd0a (0x5642c68ada9b)
             at src/materialize/dataflow/optimize.rs:244
  21: materialize::dataflow::optimize::<impl materialize::dataflow::types::RelationExpr>::visit_mut_inner_pre::hb267e3f81e7bc0e0 (0x5642c68aef41)
             at src/materialize/dataflow/optimize.rs:290
  22: materialize::dataflow::optimize::<impl materialize::dataflow::types::RelationExpr>::visit_mut_inner_pre::{{closure}}::h360423791a7703bd (0x5642c6bbd124)
             at src/materialize/dataflow/optimize.rs:290
  23: materialize::dataflow::optimize::<impl materialize::dataflow::types::RelationExpr>::visit1_mut::h0ce8504f6e5dcd0a (0x5642c68adad8)
             at src/materialize/dataflow/optimize.rs:247
  24: materialize::dataflow::optimize::<impl materialize::dataflow::types::RelationExpr>::visit_mut_inner_pre::hb267e3f81e7bc0e0 (0x5642c68aef41)
             at src/materialize/dataflow/optimize.rs:290
  25: materialize::dataflow::optimize::<impl materialize::dataflow::types::RelationExpr>::visit_mut_inner_pre::{{closure}}::h360423791a7703bd (0x5642c6bbd124)
             at src/materialize/dataflow/optimize.rs:290
  26: materialize::dataflow::optimize::<impl materialize::dataflow::types::RelationExpr>::visit1_mut::h0ce8504f6e5dcd0a (0x5642c68ada9b)
             at src/materialize/dataflow/optimize.rs:244
  27: materialize::dataflow::optimize::<impl materialize::dataflow::types::RelationExpr>::visit_mut_inner_pre::hb267e3f81e7bc0e0 (0x5642c68aef41)
             at src/materialize/dataflow/optimize.rs:290
  28: materialize::dataflow::transform::predicate_pushdown::PredicatePushdown::transform::h613d2a9cff6f47d9 (0x5642c6c112fd)
             at src/materialize/dataflow/transform.rs:226
  29: <materialize::dataflow::transform::predicate_pushdown::PredicatePushdown as materialize::dataflow::transform::Transform>::transform::h33a1df6061c73a40 (0x5642c6c112c6)
             at src/materialize/dataflow/transform.rs:220
  30: materialize::dataflow::render::build_dataflow::{{closure}}::h39c8dfedb4b63354 (0x5642c69f0fc5)
             at src/materialize/dataflow/render.rs:143
  31: <timely::worker::Worker<A>>::dataflow::{{closure}}::h243782381d500eca (0x5642c689c4b1)
             at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/9430c46/timely/src/worker.rs:363
  32: <timely::worker::Worker<A>>::dataflow_core::hd9a06e68101571b1 (0x5642c689a88d)
             at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/9430c46/timely/src/worker.rs:410
  33: <timely::worker::Worker<A>>::dataflow::h577779b87e5c2ebf (0x5642c689c0aa)
             at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/9430c46/timely/src/worker.rs:363
  34: materialize::dataflow::render::build_dataflow::h04d3031c76720484 (0x5642c69ef872)
             at src/materialize/dataflow/render.rs:72
  35: <materialize::dataflow::server::Worker<'w, A>>::handle_command::hb7a92510a9d8385d (0x5642c6941943)
             at src/materialize/dataflow/server.rs:185
  36: <materialize::dataflow::server::Worker<'w, A>>::run::h682119883b7a2673 (0x5642c694091a)
             at src/materialize/dataflow/server.rs:126
  37: materialize::dataflow::server::serve::{{closure}}::h59a86281a39d063c (0x5642c6940368)
             at src/materialize/dataflow/server.rs:48
  38: timely::execute::execute::{{closure}}::h8cd3063732fe37c9 (0x5642c66e5501)
             at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/9430c46/timely/src/execute.rs:201
  39: timely_communication::initialize::initialize_from::{{closure}}::h46df01e666c6edcd (0x5642c647228a)
             at /home/jamie/.cargo/git/checkouts/timely-dataflow-4c0cc365061cd263/9430c46/communication/src/initialize.rs:269
  40: std::sys_common::backtrace::__rust_begin_short_backtrace::h3d85b6cdceecb821 (0x5642c6897e74)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/sys_common/backtrace.rs:135
  41: std::thread::Builder::spawn_unchecked::{{closure}}::{{closure}}::hc0b17e214599bf4a (0x5642c6238356)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/thread/mod.rs:469
  42: <std::panic::AssertUnwindSafe<F> as core::ops::function::FnOnce<()>>::call_once::h9836b4965318724e (0x5642c69e7986)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panic.rs:309
  43: std::panicking::try::do_call::h1d3b81acc1438aeb (0x5642c6b69fea)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:297
  44: __rust_maybe_catch_panic (0x5642c782d17a)
             at src/libpanic_unwind/lib.rs:87
  45: std::panicking::try::h4411ffb913c96c3f (0x5642c6b69d90)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panicking.rs:276
  46: std::panic::catch_unwind::hdc83f1ca62d041cc (0x5642c69e7a16)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/panic.rs:388
  47: std::thread::Builder::spawn_unchecked::{{closure}}::h1b40cefb8685954c (0x5642c6237d70)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/libstd/thread/mod.rs:468
  48: <F as alloc::boxed::FnBox<A>>::call_box::hfdffd056c0d5f0db (0x5642c6238728)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/boxed.rs:749
  49: call_once<(),()> (0x5642c782c50e)
             at /rustc/fc50f328b0353b285421b8ff5d4100966387a997/src/liballoc/boxed.rs:759
      start_thread
             at src/libstd/sys_common/thread.rs:14
      thread_start
             at src/libstd/sys/unix/thread.rs:81
  50: start_thread (0x7f103ba42ef7)
  51: __GI___clone (0x7f103b76222f)
  52: <unknown> (0x0)

sql: aggregates over empty collections

Currently aggregates over empty collections return no rows.

jamie=> peek aggdata4;
 a | b 
---+---
(0 rows)

jamie=> CREATE MATERIALIZED VIEW test1 AS SELECT count(a) FROM aggdata4;
CREATE VIEW
jamie=> peek test1;
 ?column? 
----------
(0 rows)

jamie=> CREATE MATERIALIZED VIEW test2 AS SELECT min(a) FROM aggdata4;
CREATE VIEW
jamie=> peek test2;
 ?column? 
----------
(0 rows)

This is reasonable behavior, but sql would have otherwise:

postgres=# select count(a) from aggdata;
 count 
-------
     0
(1 row)

postgres=# select min(a) from aggdata;
 min 
-----
    
(1 row)

metastore: topological sort

Metastore can cause a crash on reboot by presenting dataflows in the "wrong" order, i.e., an order that is not topologically sorted by dependencies.

Sort out actions with observable side-effects

Actions like DROP can cause observable behavior that reflects when the action was processed. In the case of DROP this is because it detaches inputs and causes the downstream dataflow to reflect now-empty inputs. In a sense, DROP can be viewed as a data-plane action but one that currently races with the data plane.

One resolution is to tweak all of the action definitions to make sure that we can never view things that derive from other things that have been dropped, which seems totally sane (i.e. all queries are function views, and DROP only affects which subset of the results we observe).

There are some potential stress points if DROP is not synchronized across workers, where we might get subsets of updates at different workers, hosing consistency (and likely creating surprising outputs / screwing up our ability to say "functional view, subset of results"). One resolution might be to have DROP at TIME which ensures all updates are included iff not greater or equal to TIME, but which introduces a bit more synchronization (to make sure that we are able to get the instruction to all workers before their inputs pass that time).

infra: consistency guarantees

Some thoughts on what sort of consistency we want to demand from Materialize. This isn't necessary for the immediate prototype, but I'm putting it here so I can get it out of my brain.

Control plane operations should support strict serializability. Some assorted examples:

  • If I create a dataflow named F on one machine, as soon as that create operation succeeds, I should be able to select ... from F on another machine.

  • If I drop a dataflow named F on one machine, as soon as that operation succeeds it should be impossible to select ... from F on another machine. Note that this makes caching of dataflow names impossible unless we make assumptions about clock synchronization. Perhaps a little bit of staleness here is acceptable.

  • If I create a view of source S, it should not be possible to drop S until all of its dependent views have been dropped. If a view dependent on S is executed concurrently with the deletion of S, only one of the operations can succeed.

Proposal: data plane operations must never see time go backwards. If I SELECT FROM F on one machine and see results at time t, I should never be able to SELECT FROM F on another machine and see results at time t' < t. It's hard to see how to make this fault tolerant without requiring active-active-active replication (i.e., majority consensus), so perhaps we want to avoid this requirement.

metastore: retry on ZK transaction failure

Materialize does not currently retry a failed CREATE MATERIALIZED VIEW or DROP MATERIALIZED VIEW command. It should, as these can fail spuriously if another user is modifying the same dataflows concurrently.

I'm somewhat loathe to invest too much time into this before we decide if we're going to stick with ZooKeeper for metadata storage or not.

testing: e2e chbenchmark

Migrating from #1.

My acceptance criteria are:

Ingest TPC-H data using the Kafka file source. This will require a Kafka wire implementation that is correct enough that the upstream Kafka connectors think we're a legit Kafka instance.
Create a materialized view TPC-H query 1 using the Rust Postgres adapter. This will require a Postgres wire implementation that supports connection handshaking and query execution.
Read from the materialized view and see the correct answer. This will require a SQL translation layer that can handle basic aggregations and filters.

The plan is to generalize the benchmark to a mashup of TPC-C and TPC-H, called "chbenchmark". The schemas are roughly the same, so we're using the updates from TPC-C to create a changefeed that we'll plug into Materialize, which will maintain essentially all the not-hard TPC-H queries with very low latency updates.

Updated acceptance criteria for the demo:

  • Scale factor 10
  • All queries except the "hard" ones (q11 and q22, which involve inequality joins)
  • "High" throughput
  • "Low" latency

Recommend Projects

  • React photo React

    A declarative, efficient, and flexible JavaScript library for building user interfaces.

  • Vue.js photo Vue.js

    🖖 Vue.js is a progressive, incrementally-adoptable JavaScript framework for building UI on the web.

  • Typescript photo Typescript

    TypeScript is a superset of JavaScript that compiles to clean JavaScript output.

  • TensorFlow photo TensorFlow

    An Open Source Machine Learning Framework for Everyone

  • Django photo Django

    The Web framework for perfectionists with deadlines.

  • D3 photo D3

    Bring data to life with SVG, Canvas and HTML. 📊📈🎉

Recommend Topics

  • javascript

    JavaScript (JS) is a lightweight interpreted programming language with first-class functions.

  • web

    Some thing interesting about web. New door for the world.

  • server

    A server is a program made to process requests and deliver data to clients.

  • Machine learning

    Machine learning is a way of modeling and interpreting data that allows a piece of software to respond intelligently.

  • Game

    Some thing interesting about game, make everyone happy.

Recommend Org

  • Facebook photo Facebook

    We are working to build community through open source technology. NB: members must have two-factor auth.

  • Microsoft photo Microsoft

    Open source projects and samples from Microsoft.

  • Google photo Google

    Google ❤️ Open Source for everyone.

  • D3 photo D3

    Data-Driven Documents codes.