├── .flake8 ├── .github ├── pull_request_template.md └── workflows │ ├── ci.yml │ └── publish-to-pypi.yml ├── .gitignore ├── .isort.cfg ├── .pre-commit-config.yaml ├── LICENSE ├── MANIFEST.in ├── Makefile ├── README-development.md ├── README.md ├── deltacat ├── __init__.py ├── annotations.py ├── api.py ├── aws │ ├── __init__.py │ ├── clients.py │ ├── constants.py │ └── s3u.py ├── benchmarking │ ├── README.md │ ├── __init__.py │ ├── benchmark_engine.py │ ├── benchmark_parquet_reads.py │ ├── benchmark_report.py │ ├── benchmark_suite.py │ ├── conftest.py │ ├── data │ │ ├── __init__.py │ │ ├── random_row_generator.py │ │ └── row_generator.py │ └── test_benchmark_pipeline.py ├── catalog │ ├── __init__.py │ ├── delegate.py │ ├── interface.py │ ├── main │ │ ├── __init__.py │ │ └── impl.py │ └── model │ │ ├── __init__.py │ │ ├── catalog.py │ │ ├── properties.py │ │ └── table_definition.py ├── compute │ ├── __init__.py │ ├── compactor │ │ ├── README.md │ │ ├── TheFlashCompactorDesign.pdf │ │ ├── __init__.py │ │ ├── compaction_session.py │ │ ├── model │ │ │ ├── __init__.py │ │ │ ├── compact_partition_params.py │ │ │ ├── compaction_session_audit_info.py │ │ │ ├── compactor_version.py │ │ │ ├── dedupe_result.py │ │ │ ├── delta_annotated.py │ │ │ ├── delta_file_envelope.py │ │ │ ├── delta_file_locator.py │ │ │ ├── hash_bucket_result.py │ │ │ ├── materialize_result.py │ │ │ ├── primary_key_index.py │ │ │ ├── pyarrow_write_result.py │ │ │ ├── repartition_result.py │ │ │ ├── round_completion_info.py │ │ │ └── table_object_store.py │ │ ├── repartition_session.py │ │ ├── steps │ │ │ ├── __init__.py │ │ │ ├── dedupe.py │ │ │ ├── hash_bucket.py │ │ │ ├── materialize.py │ │ │ └── repartition.py │ │ └── utils │ │ │ ├── __init__.py │ │ │ ├── io.py │ │ │ ├── primary_key_index.py │ │ │ ├── round_completion_reader.py │ │ │ ├── sort_key.py │ │ │ └── system_columns.py │ ├── compactor_v2 │ │ ├── __init__.py │ │ ├── compaction_session.py │ │ ├── constants.py │ │ ├── deletes │ │ │ ├── __init__.py │ │ │ ├── delete_file_envelope.py │ │ │ ├── delete_strategy.py │ │ │ ├── delete_strategy_equality_delete.py │ │ │ ├── model.py │ │ │ └── utils.py │ │ ├── model │ │ │ ├── __init__.py │ │ │ ├── evaluate_compaction_result.py │ │ │ ├── hash_bucket_input.py │ │ │ ├── hash_bucket_result.py │ │ │ ├── merge_file_group.py │ │ │ ├── merge_input.py │ │ │ └── merge_result.py │ │ ├── private │ │ │ ├── __init__.py │ │ │ └── compaction_utils.py │ │ ├── steps │ │ │ ├── __init__.py │ │ │ ├── hash_bucket.py │ │ │ └── merge.py │ │ └── utils │ │ │ ├── __init__.py │ │ │ ├── content_type_params.py │ │ │ ├── dedupe.py │ │ │ ├── delta.py │ │ │ ├── io.py │ │ │ ├── merge.py │ │ │ ├── primary_key_index.py │ │ │ └── task_options.py │ ├── converter │ │ ├── __init__.py │ │ ├── constants.py │ │ ├── converter_session.py │ │ ├── model │ │ │ ├── __init__.py │ │ │ ├── convert_input.py │ │ │ ├── convert_input_files.py │ │ │ ├── convert_result.py │ │ │ └── converter_session_params.py │ │ ├── pyiceberg │ │ │ ├── __init__.py │ │ │ ├── catalog.py │ │ │ ├── overrides.py │ │ │ └── update_snapshot_overrides.py │ │ ├── steps │ │ │ ├── __init__.py │ │ │ ├── convert.py │ │ │ └── dedupe.py │ │ └── utils │ │ │ ├── __init__.py │ │ │ ├── convert_task_options.py │ │ │ ├── converter_session_utils.py │ │ │ ├── iceberg_columns.py │ │ │ ├── io.py │ │ │ └── s3u.py │ ├── janitor.py │ ├── jobs │ │ ├── __init__.py │ │ └── client.py │ ├── resource_estimation │ │ ├── __init__.py │ │ ├── delta.py │ │ ├── manifest.py │ │ ├── model.py │ │ └── parquet.py │ └── stats │ │ ├── __init__.py │ │ ├── models │ │ ├── __init__.py │ │ ├── delta_column_stats.py │ │ ├── delta_stats.py │ │ ├── delta_stats_cache_result.py │ │ ├── manifest_entry_stats.py │ │ └── stats_result.py │ │ └── types.py ├── constants.py ├── docs │ ├── __init__.py │ ├── autogen │ │ ├── __init__.py │ │ └── schema │ │ │ ├── __init__.py │ │ │ └── inference │ │ │ ├── __init__.py │ │ │ ├── generate_type_mappings.py │ │ │ └── parse_json_type_mappings.py │ ├── schema │ │ └── README.md │ └── table │ │ └── README.md ├── env.py ├── examples │ ├── __init__.py │ ├── basic_logging.py │ ├── compactor │ │ ├── README.md │ │ ├── __init__.py │ │ ├── aws │ │ │ ├── __init__.py │ │ │ └── deltacat.yaml │ │ ├── bootstrap.py │ │ ├── compactor.py │ │ ├── explorer.py │ │ ├── gcp │ │ │ ├── __init__.py │ │ │ └── deltacat.yaml │ │ ├── job_runner.py │ │ └── utils │ │ │ ├── __init__.py │ │ │ └── common.py │ ├── experimental │ │ ├── __init__.py │ │ ├── iceberg │ │ │ ├── __init__.py │ │ │ ├── converter │ │ │ │ ├── __init__.py │ │ │ │ └── beam │ │ │ │ │ ├── README.md │ │ │ │ │ ├── __init__.py │ │ │ │ │ ├── app.py │ │ │ │ │ ├── main.py │ │ │ │ │ ├── test_workflow.py │ │ │ │ │ └── utils │ │ │ │ │ ├── __init__.py │ │ │ │ │ ├── common.py │ │ │ │ │ └── spark.py │ │ │ ├── iceberg_bucket_writer.py │ │ │ └── iceberg_reader.py │ │ └── rivulet │ │ │ ├── data.csv │ │ │ ├── parquet_to_feather.ipynb │ │ │ └── pytorch_demo.ipynb │ ├── hello_world.py │ └── indexer │ │ ├── __init__.py │ │ ├── aws │ │ ├── __init__.py │ │ └── deltacat.yaml │ │ ├── gcp │ │ ├── __init__.py │ │ └── deltacat.yaml │ │ ├── indexer.py │ │ └── job_runner.py ├── exceptions.py ├── experimental │ ├── __init__.py │ ├── catalog │ │ ├── __init__.py │ │ └── iceberg │ │ │ ├── __init__.py │ │ │ ├── iceberg_catalog_config.py │ │ │ ├── impl.py │ │ │ └── overrides.py │ ├── compatibility │ │ ├── __init__.py │ │ └── backfill_transaction_partitions.py │ ├── converter_agent │ │ ├── __init__.py │ │ ├── beam │ │ │ ├── README.md │ │ │ ├── __init__.py │ │ │ └── managed.py │ │ ├── callbacks.py │ │ └── table_monitor.py │ ├── daft │ │ ├── __init__.py │ │ └── daft_catalog.py │ └── storage │ │ ├── __init__.py │ │ ├── iceberg │ │ ├── __init__.py │ │ ├── iceberg_scan_planner.py │ │ ├── impl.py │ │ ├── model.py │ │ └── visitor.py │ │ └── rivulet │ │ ├── __init__.py │ │ ├── arrow │ │ ├── __init__.py │ │ └── serializer.py │ │ ├── dataset.py │ │ ├── dataset_executor.py │ │ ├── feather │ │ ├── __init__.py │ │ ├── file_reader.py │ │ └── serializer.py │ │ ├── fs │ │ ├── __init__.py │ │ ├── file_provider.py │ │ ├── file_store.py │ │ ├── input_file.py │ │ └── output_file.py │ │ ├── logical_plan.py │ │ ├── metastore │ │ ├── __init__.py │ │ ├── delta.py │ │ ├── json_sst.py │ │ ├── sst.py │ │ └── sst_interval_tree.py │ │ ├── mvp │ │ ├── Table.py │ │ └── __init__.py │ │ ├── parquet │ │ ├── __init__.py │ │ ├── data_reader.py │ │ ├── file_reader.py │ │ └── serializer.py │ │ ├── reader │ │ ├── __init__.py │ │ ├── block_scanner.py │ │ ├── data_reader.py │ │ ├── data_scan.py │ │ ├── dataset_metastore.py │ │ ├── dataset_reader.py │ │ ├── pyarrow_data_reader.py │ │ ├── query_expression.py │ │ └── reader_type_registrar.py │ │ ├── schema │ │ ├── __init__.py │ │ ├── datatype.py │ │ └── schema.py │ │ ├── serializer.py │ │ ├── serializer_factory.py │ │ ├── shard │ │ ├── __init__.py │ │ └── range_shard.py │ │ └── writer │ │ ├── __init__.py │ │ ├── dataset_writer.py │ │ └── memtable_dataset_writer.py ├── io │ ├── __init__.py │ ├── dataset │ │ ├── __init__.py │ │ └── deltacat_dataset.py │ ├── datasink │ │ ├── __init__.py │ │ └── deltacat_datasink.py │ ├── datasource │ │ ├── __init__.py │ │ └── deltacat_datasource.py │ ├── file_object_store.py │ ├── memcached_object_store.py │ ├── object_store.py │ ├── ray_plasma_object_store.py │ ├── reader │ │ ├── __init__.py │ │ └── deltacat_read_api.py │ ├── redis_object_store.py │ └── s3_object_store.py ├── logs.py ├── storage │ ├── README.md │ ├── __init__.py │ ├── interface.py │ ├── main │ │ ├── __init__.py │ │ └── impl.py │ ├── model │ │ ├── __init__.py │ │ ├── delta.py │ │ ├── expression │ │ │ ├── __init__.py │ │ │ ├── expression.py │ │ │ └── visitor.py │ │ ├── interop.py │ │ ├── list_result.py │ │ ├── locator.py │ │ ├── manifest.py │ │ ├── metafile.py │ │ ├── namespace.py │ │ ├── partition.py │ │ ├── scan │ │ │ ├── __init__.py │ │ │ ├── push_down.py │ │ │ ├── scan_plan.py │ │ │ └── scan_task.py │ │ ├── schema.py │ │ ├── shard.py │ │ ├── sort_key.py │ │ ├── stream.py │ │ ├── table.py │ │ ├── table_version.py │ │ ├── transaction.py │ │ ├── transform.py │ │ └── types.py │ └── util │ │ ├── __init__.py │ │ └── scan_planner.py ├── tests │ ├── __init__.py │ ├── _io │ │ ├── __init__.py │ │ ├── reader │ │ │ ├── __init__.py │ │ │ └── test_deltacat_read_api.py │ │ ├── test_cloudpickle_bug_fix.py │ │ ├── test_file_object_store.py │ │ ├── test_memcached_object_store.py │ │ ├── test_ray_plasma_object_store.py │ │ ├── test_redis_object_store.py │ │ └── test_s3_object_store.py │ ├── aws │ │ ├── __init__.py │ │ ├── test_clients.py │ │ └── test_s3u.py │ ├── catalog │ │ ├── __init__.py │ │ ├── data │ │ │ ├── __init__.py │ │ │ └── sample_table.csv │ │ ├── main │ │ │ ├── __init__.py │ │ │ ├── test_catalog_impl_namespace_operations.py │ │ │ └── test_catalog_impl_table_operations.py │ │ ├── model │ │ │ ├── __init__.py │ │ │ ├── test_properties_transaction_migration.py │ │ │ └── test_table_definition.py │ │ ├── test_catalogs.py │ │ └── test_default_catalog_impl.py │ ├── compute │ │ ├── __init__.py │ │ ├── compact_partition_multiple_rounds_test_cases.py │ │ ├── compact_partition_rebase_test_cases.py │ │ ├── compact_partition_rebase_then_incremental_test_cases.py │ │ ├── compact_partition_test_cases.py │ │ ├── compactor │ │ │ ├── __init__.py │ │ │ ├── steps │ │ │ │ ├── __init__.py │ │ │ │ └── test_repartition.py │ │ │ └── utils │ │ │ │ ├── __init__.py │ │ │ │ ├── test_io.py │ │ │ │ └── test_round_completion_reader.py │ │ ├── compactor_v2 │ │ │ ├── __init__.py │ │ │ ├── data │ │ │ │ ├── backfill_source_date_pk.csv │ │ │ │ └── incremental_source_date_pk.csv │ │ │ ├── deletes │ │ │ │ ├── test_delete_file_envelope.py │ │ │ │ ├── test_delete_strategy_equality_delete.py │ │ │ │ └── test_delete_utils.py │ │ │ ├── steps │ │ │ │ ├── data │ │ │ │ │ ├── date_pk_table.csv │ │ │ │ │ ├── dedupe_base_compacted_table_date_pk.csv │ │ │ │ │ ├── dedupe_base_compacted_table_multiple_pk.csv │ │ │ │ │ ├── dedupe_base_compacted_table_multiple_pk_delete.csv │ │ │ │ │ ├── dedupe_base_compacted_table_string_pk.csv │ │ │ │ │ ├── dedupe_table_no_duplication_date_pk.csv │ │ │ │ │ ├── dedupe_table_no_duplication_multiple_pk.csv │ │ │ │ │ ├── dedupe_table_no_duplication_string_pk.csv │ │ │ │ │ ├── dedupe_table_with_duplication_date_pk.csv │ │ │ │ │ ├── dedupe_table_with_duplication_multiple_pk.csv │ │ │ │ │ ├── dedupe_table_with_duplication_string_pk.csv │ │ │ │ │ ├── multiple_pk_table.csv │ │ │ │ │ ├── no_pk_table.csv │ │ │ │ │ └── string_pk_table.csv │ │ │ │ ├── test_hash_bucket.py │ │ │ │ └── test_merge.py │ │ │ ├── test_compaction_session.py │ │ │ ├── test_hashlib.py │ │ │ └── utils │ │ │ │ ├── __init__.py │ │ │ │ ├── test_content_type_params.py │ │ │ │ ├── test_primary_key_index.py │ │ │ │ └── test_task_options.py │ │ ├── conftest.py │ │ ├── converter │ │ │ ├── __init__.py │ │ │ ├── integration │ │ │ │ ├── conftest.py │ │ │ │ ├── test_convert_session.py │ │ │ │ └── test_converter_commit_conflict_resolution.py │ │ │ ├── unit │ │ │ │ ├── __init__.py │ │ │ │ ├── test_convert.py │ │ │ │ ├── test_converter_session_utils.py │ │ │ │ └── test_dedupe.py │ │ │ └── utils.py │ │ ├── resource_estimation │ │ │ ├── __init__.py │ │ │ ├── data │ │ │ │ ├── DATA.md │ │ │ │ ├── __init__.py │ │ │ │ ├── date_pk_table.csv │ │ │ │ ├── sample_no_stats.parquet │ │ │ │ └── sample_with_stats.parquet │ │ │ ├── test_delta.py │ │ │ └── test_manifest.py │ │ ├── test_compact_partition_incremental.py │ │ ├── test_compact_partition_multiple_rounds.py │ │ ├── test_compact_partition_params.py │ │ ├── test_compact_partition_rebase.py │ │ ├── test_compact_partition_rebase_then_incremental.py │ │ ├── test_janitor.py │ │ ├── test_util_common.py │ │ └── test_util_constant.py │ ├── conftest.py │ ├── daft │ │ ├── __init__.py │ │ └── test_model.py │ ├── experimental │ │ ├── __init__.py │ │ ├── catalog │ │ │ ├── __init__.py │ │ │ └── iceberg │ │ │ │ ├── __init__.py │ │ │ │ └── test_iceberg_catalog.py │ │ ├── compatibility │ │ │ ├── __init__.py │ │ │ └── test_backfill_transaction_partitions.py │ │ ├── converter_agent │ │ │ ├── __init__.py │ │ │ ├── conftest.py │ │ │ ├── test_callbacks.py │ │ │ └── test_table_monitor.py │ │ ├── daft │ │ │ ├── __init__.py │ │ │ └── test_deltacat_daft_integration.py │ │ └── storage │ │ │ ├── __init__.py │ │ │ ├── iceberg │ │ │ └── test_visitor.py │ │ │ └── rivulet │ │ │ ├── __init__.py │ │ │ ├── conftest.py │ │ │ ├── fs │ │ │ ├── __init__.py │ │ │ └── test_file_location_provider.py │ │ │ ├── reader │ │ │ ├── __init__.py │ │ │ ├── query_expression.py │ │ │ ├── test_data_scan.py │ │ │ └── test_dataset_metastore.py │ │ │ ├── schema │ │ │ ├── __init__.py │ │ │ └── test_schema.py │ │ │ ├── shard │ │ │ ├── __init__.py │ │ │ └── test_range_shard.py │ │ │ ├── test_dataset.py │ │ │ ├── test_manifest.py │ │ │ ├── test_sst_interval_tree.py │ │ │ ├── test_utils.py │ │ │ └── writer │ │ │ ├── __init__.py │ │ │ ├── test_dataset_write_then_read.py │ │ │ ├── test_dataset_writer.py │ │ │ └── test_memtable_dataset_writer.py │ ├── integ │ │ └── catalog │ │ │ └── iceberg │ │ │ └── test_local_rest_catalog.py │ ├── storage │ │ ├── __init__.py │ │ ├── main │ │ │ ├── __init__.py │ │ │ ├── test_delta_ordering.py │ │ │ └── test_main_storage.py │ │ └── model │ │ │ ├── __init__.py │ │ │ ├── test_delete_parameters.py │ │ │ ├── test_expression.py │ │ │ ├── test_manifest.py │ │ │ ├── test_metafile_io.py │ │ │ ├── test_partition_scheme.py │ │ │ ├── test_schema.py │ │ │ ├── test_schema_update.py │ │ │ ├── test_shard.py │ │ │ ├── test_sort_scheme.py │ │ │ ├── test_table_version.py │ │ │ ├── test_transaction.py │ │ │ └── test_transaction_history.py │ ├── test_deltacat_api.py │ ├── test_exceptions.py │ ├── test_logs.py │ ├── test_utils │ │ ├── __init__.py │ │ ├── constants.py │ │ ├── filesystem.py │ │ ├── message_pack_utils.py │ │ ├── pyarrow.py │ │ ├── resources │ │ │ └── test_delta.json │ │ ├── storage.py │ │ └── utils.py │ ├── types │ │ ├── __init__.py │ │ └── test_tables.py │ └── utils │ │ ├── __init__.py │ │ ├── data │ │ ├── __init__.py │ │ ├── empty.csv │ │ ├── mvp.parquet │ │ ├── non_empty_compressed.bz2 │ │ ├── non_empty_compressed.gz │ │ ├── non_empty_valid.csv │ │ ├── overflowing_decimal_precision.csv │ │ ├── overflowing_decimal_scale.csv │ │ ├── test_file.parquet │ │ └── test_file.parquet.gz │ │ ├── exceptions.py │ │ ├── main_deltacat_storage_mock.py │ │ ├── ray_utils │ │ ├── __init__.py │ │ ├── test_concurrency.py │ │ └── test_dataset.py │ │ ├── test_cloudpickle.py │ │ ├── test_daft.py │ │ ├── test_filesystem.py │ │ ├── test_metrics.py │ │ ├── test_numpy.py │ │ ├── test_pandas.py │ │ ├── test_placement.py │ │ ├── test_polars.py │ │ ├── test_pyarrow.py │ │ ├── test_record_batch_tables.py │ │ └── test_resources.py ├── types │ ├── __init__.py │ ├── media.py │ ├── partial_download.py │ └── tables.py └── utils │ ├── __init__.py │ ├── arguments.py │ ├── cloudpickle.py │ ├── common.py │ ├── daft.py │ ├── export.py │ ├── filesystem.py │ ├── metafile_locator.py │ ├── metrics.py │ ├── numpy.py │ ├── pandas.py │ ├── performance.py │ ├── placement.py │ ├── polars.py │ ├── pyarrow.py │ ├── ray_utils │ ├── __init__.py │ ├── collections.py │ ├── concurrency.py │ ├── dataset.py │ ├── performance.py │ └── runtime.py │ ├── reader_compatibility_mapping.py │ ├── resources.py │ ├── schema.py │ └── url.py ├── dev-requirements.txt ├── dev ├── _sandbox │ └── compute │ │ └── converter │ │ ├── drop_duplicates_benchmark.py │ │ └── example_single_merge_key_converter.py ├── deploy │ └── aws │ │ └── scripts │ │ ├── common.py │ │ ├── runner.py │ │ └── s3-build-and-deploy.sh └── iceberg-integration │ ├── Dockerfile │ ├── docker-compose-integration.yml │ ├── entrypoint.sh │ ├── provision.py │ └── spark-defaults.conf ├── media ├── calico.jpg ├── customer_feedback_001.txt ├── customer_feedback_002.txt ├── customer_feedback_003.txt ├── deltacat-logo-alpha-750.png ├── deltacat-logo-alpha.png ├── deltacat-tech-overview.png ├── siamese.jpg └── tuxedo.jpg ├── pytest.ini ├── requirements.txt └── setup.py /.flake8: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/.flake8 -------------------------------------------------------------------------------- /.github/pull_request_template.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/.github/pull_request_template.md -------------------------------------------------------------------------------- /.github/workflows/ci.yml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/.github/workflows/ci.yml -------------------------------------------------------------------------------- /.github/workflows/publish-to-pypi.yml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/.github/workflows/publish-to-pypi.yml -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/.gitignore -------------------------------------------------------------------------------- /.isort.cfg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/.isort.cfg -------------------------------------------------------------------------------- /.pre-commit-config.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/.pre-commit-config.yaml -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/LICENSE -------------------------------------------------------------------------------- /MANIFEST.in: -------------------------------------------------------------------------------- 1 | recursive-include *.yaml 2 | -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/Makefile -------------------------------------------------------------------------------- /README-development.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/README-development.md -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/README.md -------------------------------------------------------------------------------- /deltacat/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/__init__.py -------------------------------------------------------------------------------- /deltacat/annotations.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/annotations.py -------------------------------------------------------------------------------- /deltacat/api.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/api.py -------------------------------------------------------------------------------- /deltacat/aws/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/aws/clients.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/aws/clients.py -------------------------------------------------------------------------------- /deltacat/aws/constants.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/aws/constants.py -------------------------------------------------------------------------------- /deltacat/aws/s3u.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/aws/s3u.py -------------------------------------------------------------------------------- /deltacat/benchmarking/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/benchmarking/README.md -------------------------------------------------------------------------------- /deltacat/benchmarking/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/benchmarking/benchmark_engine.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/benchmarking/benchmark_engine.py -------------------------------------------------------------------------------- /deltacat/benchmarking/benchmark_parquet_reads.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/benchmarking/benchmark_parquet_reads.py -------------------------------------------------------------------------------- /deltacat/benchmarking/benchmark_report.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/benchmarking/benchmark_report.py -------------------------------------------------------------------------------- /deltacat/benchmarking/benchmark_suite.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/benchmarking/benchmark_suite.py -------------------------------------------------------------------------------- /deltacat/benchmarking/conftest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/benchmarking/conftest.py -------------------------------------------------------------------------------- /deltacat/benchmarking/data/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/benchmarking/data/random_row_generator.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/benchmarking/data/random_row_generator.py -------------------------------------------------------------------------------- /deltacat/benchmarking/data/row_generator.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/benchmarking/data/row_generator.py -------------------------------------------------------------------------------- /deltacat/benchmarking/test_benchmark_pipeline.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/benchmarking/test_benchmark_pipeline.py -------------------------------------------------------------------------------- /deltacat/catalog/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/catalog/__init__.py -------------------------------------------------------------------------------- /deltacat/catalog/delegate.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/catalog/delegate.py -------------------------------------------------------------------------------- /deltacat/catalog/interface.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/catalog/interface.py -------------------------------------------------------------------------------- /deltacat/catalog/main/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/catalog/main/impl.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/catalog/main/impl.py -------------------------------------------------------------------------------- /deltacat/catalog/model/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/catalog/model/catalog.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/catalog/model/catalog.py -------------------------------------------------------------------------------- /deltacat/catalog/model/properties.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/catalog/model/properties.py -------------------------------------------------------------------------------- /deltacat/catalog/model/table_definition.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/catalog/model/table_definition.py -------------------------------------------------------------------------------- /deltacat/compute/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/__init__.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/README.md -------------------------------------------------------------------------------- /deltacat/compute/compactor/TheFlashCompactorDesign.pdf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/TheFlashCompactorDesign.pdf -------------------------------------------------------------------------------- /deltacat/compute/compactor/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/__init__.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/compaction_session.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/compaction_session.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/compact_partition_params.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/compact_partition_params.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/compaction_session_audit_info.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/compaction_session_audit_info.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/compactor_version.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/compactor_version.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/dedupe_result.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/dedupe_result.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/delta_annotated.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/delta_annotated.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/delta_file_envelope.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/delta_file_envelope.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/delta_file_locator.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/delta_file_locator.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/hash_bucket_result.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/hash_bucket_result.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/materialize_result.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/materialize_result.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/primary_key_index.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/primary_key_index.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/pyarrow_write_result.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/pyarrow_write_result.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/repartition_result.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/repartition_result.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/round_completion_info.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/round_completion_info.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/model/table_object_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/model/table_object_store.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/repartition_session.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/repartition_session.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/steps/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/compactor/steps/dedupe.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/steps/dedupe.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/steps/hash_bucket.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/steps/hash_bucket.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/steps/materialize.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/steps/materialize.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/steps/repartition.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/steps/repartition.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/utils/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/compactor/utils/io.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/utils/io.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/utils/primary_key_index.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/utils/primary_key_index.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/utils/round_completion_reader.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/utils/round_completion_reader.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/utils/sort_key.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/utils/sort_key.py -------------------------------------------------------------------------------- /deltacat/compute/compactor/utils/system_columns.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor/utils/system_columns.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/compaction_session.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/compaction_session.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/constants.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/constants.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/deletes/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/deletes/delete_file_envelope.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/deletes/delete_file_envelope.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/deletes/delete_strategy.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/deletes/delete_strategy.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/deletes/delete_strategy_equality_delete.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/deletes/delete_strategy_equality_delete.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/deletes/model.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/deletes/model.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/deletes/utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/deletes/utils.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/model/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/model/evaluate_compaction_result.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/model/evaluate_compaction_result.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/model/hash_bucket_input.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/model/hash_bucket_input.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/model/hash_bucket_result.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/model/hash_bucket_result.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/model/merge_file_group.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/model/merge_file_group.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/model/merge_input.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/model/merge_input.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/model/merge_result.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/model/merge_result.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/private/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/private/compaction_utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/private/compaction_utils.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/steps/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/steps/hash_bucket.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/steps/hash_bucket.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/steps/merge.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/steps/merge.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/utils/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/utils/content_type_params.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/utils/content_type_params.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/utils/dedupe.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/utils/dedupe.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/utils/delta.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/utils/delta.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/utils/io.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/utils/io.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/utils/merge.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/utils/merge.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/utils/primary_key_index.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/utils/primary_key_index.py -------------------------------------------------------------------------------- /deltacat/compute/compactor_v2/utils/task_options.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/compactor_v2/utils/task_options.py -------------------------------------------------------------------------------- /deltacat/compute/converter/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/converter/constants.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/constants.py -------------------------------------------------------------------------------- /deltacat/compute/converter/converter_session.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/converter_session.py -------------------------------------------------------------------------------- /deltacat/compute/converter/model/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/converter/model/convert_input.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/model/convert_input.py -------------------------------------------------------------------------------- /deltacat/compute/converter/model/convert_input_files.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/model/convert_input_files.py -------------------------------------------------------------------------------- /deltacat/compute/converter/model/convert_result.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/model/convert_result.py -------------------------------------------------------------------------------- /deltacat/compute/converter/model/converter_session_params.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/model/converter_session_params.py -------------------------------------------------------------------------------- /deltacat/compute/converter/pyiceberg/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/converter/pyiceberg/catalog.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/pyiceberg/catalog.py -------------------------------------------------------------------------------- /deltacat/compute/converter/pyiceberg/overrides.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/pyiceberg/overrides.py -------------------------------------------------------------------------------- /deltacat/compute/converter/pyiceberg/update_snapshot_overrides.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/pyiceberg/update_snapshot_overrides.py -------------------------------------------------------------------------------- /deltacat/compute/converter/steps/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/converter/steps/convert.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/steps/convert.py -------------------------------------------------------------------------------- /deltacat/compute/converter/steps/dedupe.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/steps/dedupe.py -------------------------------------------------------------------------------- /deltacat/compute/converter/utils/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/converter/utils/convert_task_options.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/utils/convert_task_options.py -------------------------------------------------------------------------------- /deltacat/compute/converter/utils/converter_session_utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/utils/converter_session_utils.py -------------------------------------------------------------------------------- /deltacat/compute/converter/utils/iceberg_columns.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/utils/iceberg_columns.py -------------------------------------------------------------------------------- /deltacat/compute/converter/utils/io.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/utils/io.py -------------------------------------------------------------------------------- /deltacat/compute/converter/utils/s3u.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/converter/utils/s3u.py -------------------------------------------------------------------------------- /deltacat/compute/janitor.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/janitor.py -------------------------------------------------------------------------------- /deltacat/compute/jobs/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/jobs/client.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/jobs/client.py -------------------------------------------------------------------------------- /deltacat/compute/resource_estimation/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/resource_estimation/__init__.py -------------------------------------------------------------------------------- /deltacat/compute/resource_estimation/delta.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/resource_estimation/delta.py -------------------------------------------------------------------------------- /deltacat/compute/resource_estimation/manifest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/resource_estimation/manifest.py -------------------------------------------------------------------------------- /deltacat/compute/resource_estimation/model.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/resource_estimation/model.py -------------------------------------------------------------------------------- /deltacat/compute/resource_estimation/parquet.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/resource_estimation/parquet.py -------------------------------------------------------------------------------- /deltacat/compute/stats/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/stats/models/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/compute/stats/models/delta_column_stats.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/stats/models/delta_column_stats.py -------------------------------------------------------------------------------- /deltacat/compute/stats/models/delta_stats.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/stats/models/delta_stats.py -------------------------------------------------------------------------------- /deltacat/compute/stats/models/delta_stats_cache_result.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/stats/models/delta_stats_cache_result.py -------------------------------------------------------------------------------- /deltacat/compute/stats/models/manifest_entry_stats.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/stats/models/manifest_entry_stats.py -------------------------------------------------------------------------------- /deltacat/compute/stats/models/stats_result.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/stats/models/stats_result.py -------------------------------------------------------------------------------- /deltacat/compute/stats/types.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/compute/stats/types.py -------------------------------------------------------------------------------- /deltacat/constants.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/constants.py -------------------------------------------------------------------------------- /deltacat/docs/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/docs/autogen/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/docs/autogen/schema/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/docs/autogen/schema/inference/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/docs/autogen/schema/inference/generate_type_mappings.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/docs/autogen/schema/inference/generate_type_mappings.py -------------------------------------------------------------------------------- /deltacat/docs/autogen/schema/inference/parse_json_type_mappings.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/docs/autogen/schema/inference/parse_json_type_mappings.py -------------------------------------------------------------------------------- /deltacat/docs/schema/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/docs/schema/README.md -------------------------------------------------------------------------------- /deltacat/docs/table/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/docs/table/README.md -------------------------------------------------------------------------------- /deltacat/env.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/env.py -------------------------------------------------------------------------------- /deltacat/examples/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/examples/basic_logging.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/basic_logging.py -------------------------------------------------------------------------------- /deltacat/examples/compactor/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/compactor/README.md -------------------------------------------------------------------------------- /deltacat/examples/compactor/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/examples/compactor/aws/__init__.py: -------------------------------------------------------------------------------- 1 | # DeltaCAT Compactor AWS Examples 2 | -------------------------------------------------------------------------------- /deltacat/examples/compactor/aws/deltacat.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/compactor/aws/deltacat.yaml -------------------------------------------------------------------------------- /deltacat/examples/compactor/bootstrap.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/compactor/bootstrap.py -------------------------------------------------------------------------------- /deltacat/examples/compactor/compactor.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/compactor/compactor.py -------------------------------------------------------------------------------- /deltacat/examples/compactor/explorer.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/compactor/explorer.py -------------------------------------------------------------------------------- /deltacat/examples/compactor/gcp/__init__.py: -------------------------------------------------------------------------------- 1 | # DeltaCAT Compactor GCP Examples 2 | -------------------------------------------------------------------------------- /deltacat/examples/compactor/gcp/deltacat.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/compactor/gcp/deltacat.yaml -------------------------------------------------------------------------------- /deltacat/examples/compactor/job_runner.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/compactor/job_runner.py -------------------------------------------------------------------------------- /deltacat/examples/compactor/utils/__init__.py: -------------------------------------------------------------------------------- 1 | # Common utilities for DeltaCAT compactor examples 2 | -------------------------------------------------------------------------------- /deltacat/examples/compactor/utils/common.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/compactor/utils/common.py -------------------------------------------------------------------------------- /deltacat/examples/experimental/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/examples/experimental/iceberg/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/examples/experimental/iceberg/converter/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/examples/experimental/iceberg/converter/beam/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/experimental/iceberg/converter/beam/README.md -------------------------------------------------------------------------------- /deltacat/examples/experimental/iceberg/converter/beam/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/examples/experimental/iceberg/converter/beam/app.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/experimental/iceberg/converter/beam/app.py -------------------------------------------------------------------------------- /deltacat/examples/experimental/iceberg/converter/beam/main.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/experimental/iceberg/converter/beam/main.py -------------------------------------------------------------------------------- /deltacat/examples/experimental/iceberg/converter/beam/test_workflow.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/experimental/iceberg/converter/beam/test_workflow.py -------------------------------------------------------------------------------- /deltacat/examples/experimental/iceberg/converter/beam/utils/__init__.py: -------------------------------------------------------------------------------- 1 | """ 2 | Utility modules for the Iceberg converter example. 3 | """ 4 | -------------------------------------------------------------------------------- /deltacat/examples/experimental/iceberg/converter/beam/utils/common.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/experimental/iceberg/converter/beam/utils/common.py -------------------------------------------------------------------------------- /deltacat/examples/experimental/iceberg/converter/beam/utils/spark.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/experimental/iceberg/converter/beam/utils/spark.py -------------------------------------------------------------------------------- /deltacat/examples/experimental/iceberg/iceberg_bucket_writer.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/experimental/iceberg/iceberg_bucket_writer.py -------------------------------------------------------------------------------- /deltacat/examples/experimental/iceberg/iceberg_reader.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/experimental/iceberg/iceberg_reader.py -------------------------------------------------------------------------------- /deltacat/examples/experimental/rivulet/data.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/experimental/rivulet/data.csv -------------------------------------------------------------------------------- /deltacat/examples/experimental/rivulet/parquet_to_feather.ipynb: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/experimental/rivulet/parquet_to_feather.ipynb -------------------------------------------------------------------------------- /deltacat/examples/experimental/rivulet/pytorch_demo.ipynb: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/experimental/rivulet/pytorch_demo.ipynb -------------------------------------------------------------------------------- /deltacat/examples/hello_world.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/hello_world.py -------------------------------------------------------------------------------- /deltacat/examples/indexer/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/examples/indexer/aws/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/examples/indexer/aws/deltacat.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/indexer/aws/deltacat.yaml -------------------------------------------------------------------------------- /deltacat/examples/indexer/gcp/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/examples/indexer/gcp/deltacat.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/indexer/gcp/deltacat.yaml -------------------------------------------------------------------------------- /deltacat/examples/indexer/indexer.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/indexer/indexer.py -------------------------------------------------------------------------------- /deltacat/examples/indexer/job_runner.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/examples/indexer/job_runner.py -------------------------------------------------------------------------------- /deltacat/exceptions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/exceptions.py -------------------------------------------------------------------------------- /deltacat/experimental/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/catalog/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/catalog/iceberg/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/catalog/iceberg/__init__.py -------------------------------------------------------------------------------- /deltacat/experimental/catalog/iceberg/iceberg_catalog_config.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/catalog/iceberg/iceberg_catalog_config.py -------------------------------------------------------------------------------- /deltacat/experimental/catalog/iceberg/impl.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/catalog/iceberg/impl.py -------------------------------------------------------------------------------- /deltacat/experimental/catalog/iceberg/overrides.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/catalog/iceberg/overrides.py -------------------------------------------------------------------------------- /deltacat/experimental/compatibility/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/compatibility/backfill_transaction_partitions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/compatibility/backfill_transaction_partitions.py -------------------------------------------------------------------------------- /deltacat/experimental/converter_agent/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/converter_agent/beam/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/converter_agent/beam/README.md -------------------------------------------------------------------------------- /deltacat/experimental/converter_agent/beam/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/converter_agent/beam/managed.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/converter_agent/beam/managed.py -------------------------------------------------------------------------------- /deltacat/experimental/converter_agent/callbacks.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/converter_agent/callbacks.py -------------------------------------------------------------------------------- /deltacat/experimental/converter_agent/table_monitor.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/converter_agent/table_monitor.py -------------------------------------------------------------------------------- /deltacat/experimental/daft/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/daft/__init__.py -------------------------------------------------------------------------------- /deltacat/experimental/daft/daft_catalog.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/daft/daft_catalog.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/storage/iceberg/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/storage/iceberg/iceberg_scan_planner.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/iceberg/iceberg_scan_planner.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/iceberg/impl.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/iceberg/impl.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/iceberg/model.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/iceberg/model.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/iceberg/visitor.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/iceberg/visitor.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/__init__.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/arrow/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/arrow/serializer.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/arrow/serializer.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/dataset.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/dataset.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/dataset_executor.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/dataset_executor.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/feather/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/feather/__init__.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/feather/file_reader.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/feather/file_reader.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/feather/serializer.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/feather/serializer.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/fs/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/fs/file_provider.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/fs/file_provider.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/fs/file_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/fs/file_store.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/fs/input_file.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/fs/input_file.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/fs/output_file.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/fs/output_file.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/logical_plan.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/logical_plan.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/metastore/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/metastore/delta.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/metastore/delta.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/metastore/json_sst.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/metastore/json_sst.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/metastore/sst.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/metastore/sst.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/metastore/sst_interval_tree.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/metastore/sst_interval_tree.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/mvp/Table.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/mvp/Table.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/mvp/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/mvp/__init__.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/parquet/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/parquet/__init__.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/parquet/data_reader.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/parquet/file_reader.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/parquet/file_reader.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/parquet/serializer.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/parquet/serializer.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/reader/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/reader/block_scanner.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/reader/block_scanner.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/reader/data_reader.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/reader/data_reader.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/reader/data_scan.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/reader/data_scan.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/reader/dataset_metastore.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/reader/dataset_metastore.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/reader/dataset_reader.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/reader/dataset_reader.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/reader/pyarrow_data_reader.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/reader/pyarrow_data_reader.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/reader/query_expression.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/reader/query_expression.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/reader/reader_type_registrar.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/reader/reader_type_registrar.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/schema/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/schema/datatype.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/schema/datatype.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/schema/schema.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/schema/schema.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/serializer.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/serializer.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/serializer_factory.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/serializer_factory.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/shard/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/shard/range_shard.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/shard/range_shard.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/writer/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/writer/dataset_writer.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/writer/dataset_writer.py -------------------------------------------------------------------------------- /deltacat/experimental/storage/rivulet/writer/memtable_dataset_writer.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/experimental/storage/rivulet/writer/memtable_dataset_writer.py -------------------------------------------------------------------------------- /deltacat/io/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/io/__init__.py -------------------------------------------------------------------------------- /deltacat/io/dataset/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/io/dataset/deltacat_dataset.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/io/dataset/deltacat_dataset.py -------------------------------------------------------------------------------- /deltacat/io/datasink/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/io/datasink/deltacat_datasink.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/io/datasink/deltacat_datasink.py -------------------------------------------------------------------------------- /deltacat/io/datasource/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/io/datasource/deltacat_datasource.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/io/datasource/deltacat_datasource.py -------------------------------------------------------------------------------- /deltacat/io/file_object_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/io/file_object_store.py -------------------------------------------------------------------------------- /deltacat/io/memcached_object_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/io/memcached_object_store.py -------------------------------------------------------------------------------- /deltacat/io/object_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/io/object_store.py -------------------------------------------------------------------------------- /deltacat/io/ray_plasma_object_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/io/ray_plasma_object_store.py -------------------------------------------------------------------------------- /deltacat/io/reader/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/io/reader/deltacat_read_api.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/io/reader/deltacat_read_api.py -------------------------------------------------------------------------------- /deltacat/io/redis_object_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/io/redis_object_store.py -------------------------------------------------------------------------------- /deltacat/io/s3_object_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/io/s3_object_store.py -------------------------------------------------------------------------------- /deltacat/logs.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/logs.py -------------------------------------------------------------------------------- /deltacat/storage/README.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/README.md -------------------------------------------------------------------------------- /deltacat/storage/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/__init__.py -------------------------------------------------------------------------------- /deltacat/storage/interface.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/interface.py -------------------------------------------------------------------------------- /deltacat/storage/main/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/storage/main/impl.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/main/impl.py -------------------------------------------------------------------------------- /deltacat/storage/model/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/storage/model/delta.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/delta.py -------------------------------------------------------------------------------- /deltacat/storage/model/expression/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/expression/__init__.py -------------------------------------------------------------------------------- /deltacat/storage/model/expression/expression.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/expression/expression.py -------------------------------------------------------------------------------- /deltacat/storage/model/expression/visitor.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/expression/visitor.py -------------------------------------------------------------------------------- /deltacat/storage/model/interop.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/interop.py -------------------------------------------------------------------------------- /deltacat/storage/model/list_result.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/list_result.py -------------------------------------------------------------------------------- /deltacat/storage/model/locator.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/locator.py -------------------------------------------------------------------------------- /deltacat/storage/model/manifest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/manifest.py -------------------------------------------------------------------------------- /deltacat/storage/model/metafile.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/metafile.py -------------------------------------------------------------------------------- /deltacat/storage/model/namespace.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/namespace.py -------------------------------------------------------------------------------- /deltacat/storage/model/partition.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/partition.py -------------------------------------------------------------------------------- /deltacat/storage/model/scan/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/storage/model/scan/push_down.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/scan/push_down.py -------------------------------------------------------------------------------- /deltacat/storage/model/scan/scan_plan.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/scan/scan_plan.py -------------------------------------------------------------------------------- /deltacat/storage/model/scan/scan_task.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/scan/scan_task.py -------------------------------------------------------------------------------- /deltacat/storage/model/schema.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/schema.py -------------------------------------------------------------------------------- /deltacat/storage/model/shard.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/shard.py -------------------------------------------------------------------------------- /deltacat/storage/model/sort_key.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/sort_key.py -------------------------------------------------------------------------------- /deltacat/storage/model/stream.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/stream.py -------------------------------------------------------------------------------- /deltacat/storage/model/table.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/table.py -------------------------------------------------------------------------------- /deltacat/storage/model/table_version.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/table_version.py -------------------------------------------------------------------------------- /deltacat/storage/model/transaction.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/transaction.py -------------------------------------------------------------------------------- /deltacat/storage/model/transform.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/transform.py -------------------------------------------------------------------------------- /deltacat/storage/model/types.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/model/types.py -------------------------------------------------------------------------------- /deltacat/storage/util/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/storage/util/scan_planner.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/storage/util/scan_planner.py -------------------------------------------------------------------------------- /deltacat/tests/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/_io/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/_io/__init__.py -------------------------------------------------------------------------------- /deltacat/tests/_io/reader/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/_io/reader/test_deltacat_read_api.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/_io/test_cloudpickle_bug_fix.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/_io/test_cloudpickle_bug_fix.py -------------------------------------------------------------------------------- /deltacat/tests/_io/test_file_object_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/_io/test_file_object_store.py -------------------------------------------------------------------------------- /deltacat/tests/_io/test_memcached_object_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/_io/test_memcached_object_store.py -------------------------------------------------------------------------------- /deltacat/tests/_io/test_ray_plasma_object_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/_io/test_ray_plasma_object_store.py -------------------------------------------------------------------------------- /deltacat/tests/_io/test_redis_object_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/_io/test_redis_object_store.py -------------------------------------------------------------------------------- /deltacat/tests/_io/test_s3_object_store.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/_io/test_s3_object_store.py -------------------------------------------------------------------------------- /deltacat/tests/aws/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/aws/test_clients.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/aws/test_clients.py -------------------------------------------------------------------------------- /deltacat/tests/aws/test_s3u.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/aws/test_s3u.py -------------------------------------------------------------------------------- /deltacat/tests/catalog/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/catalog/data/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/catalog/data/sample_table.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/catalog/data/sample_table.csv -------------------------------------------------------------------------------- /deltacat/tests/catalog/main/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/catalog/main/test_catalog_impl_namespace_operations.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/catalog/main/test_catalog_impl_namespace_operations.py -------------------------------------------------------------------------------- /deltacat/tests/catalog/main/test_catalog_impl_table_operations.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/catalog/main/test_catalog_impl_table_operations.py -------------------------------------------------------------------------------- /deltacat/tests/catalog/model/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/catalog/model/test_properties_transaction_migration.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/catalog/model/test_properties_transaction_migration.py -------------------------------------------------------------------------------- /deltacat/tests/catalog/model/test_table_definition.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/catalog/model/test_table_definition.py -------------------------------------------------------------------------------- /deltacat/tests/catalog/test_catalogs.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/catalog/test_catalogs.py -------------------------------------------------------------------------------- /deltacat/tests/catalog/test_default_catalog_impl.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/catalog/test_default_catalog_impl.py -------------------------------------------------------------------------------- /deltacat/tests/compute/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/compute/compact_partition_multiple_rounds_test_cases.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compact_partition_multiple_rounds_test_cases.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compact_partition_rebase_test_cases.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compact_partition_rebase_test_cases.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compact_partition_rebase_then_incremental_test_cases.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compact_partition_rebase_then_incremental_test_cases.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compact_partition_test_cases.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compact_partition_test_cases.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor/steps/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor/steps/test_repartition.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor/steps/test_repartition.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor/utils/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor/utils/test_io.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor/utils/test_io.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor/utils/test_round_completion_reader.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor/utils/test_round_completion_reader.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/data/backfill_source_date_pk.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/data/backfill_source_date_pk.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/data/incremental_source_date_pk.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/data/incremental_source_date_pk.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/deletes/test_delete_file_envelope.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/deletes/test_delete_file_envelope.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/deletes/test_delete_strategy_equality_delete.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/deletes/test_delete_strategy_equality_delete.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/deletes/test_delete_utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/deletes/test_delete_utils.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/date_pk_table.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/date_pk_table.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/dedupe_base_compacted_table_date_pk.csv: -------------------------------------------------------------------------------- 1 | pk,value 2 | 2022-10-26,1 3 | -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/dedupe_base_compacted_table_multiple_pk.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/dedupe_base_compacted_table_multiple_pk.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/dedupe_base_compacted_table_multiple_pk_delete.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/dedupe_base_compacted_table_multiple_pk_delete.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/dedupe_base_compacted_table_string_pk.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/dedupe_base_compacted_table_string_pk.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/dedupe_table_no_duplication_date_pk.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/dedupe_table_no_duplication_date_pk.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/dedupe_table_no_duplication_multiple_pk.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/dedupe_table_no_duplication_multiple_pk.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/dedupe_table_no_duplication_string_pk.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/dedupe_table_no_duplication_string_pk.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/dedupe_table_with_duplication_date_pk.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/dedupe_table_with_duplication_date_pk.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/dedupe_table_with_duplication_multiple_pk.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/dedupe_table_with_duplication_multiple_pk.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/dedupe_table_with_duplication_string_pk.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/dedupe_table_with_duplication_string_pk.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/multiple_pk_table.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/multiple_pk_table.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/no_pk_table.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/no_pk_table.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/data/string_pk_table.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/data/string_pk_table.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/test_hash_bucket.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/test_hash_bucket.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/steps/test_merge.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/steps/test_merge.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/test_compaction_session.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/test_compaction_session.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/test_hashlib.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/test_hashlib.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/utils/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/utils/test_content_type_params.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/utils/test_content_type_params.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/utils/test_primary_key_index.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/utils/test_primary_key_index.py -------------------------------------------------------------------------------- /deltacat/tests/compute/compactor_v2/utils/test_task_options.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/compactor_v2/utils/test_task_options.py -------------------------------------------------------------------------------- /deltacat/tests/compute/conftest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/conftest.py -------------------------------------------------------------------------------- /deltacat/tests/compute/converter/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/compute/converter/integration/conftest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/converter/integration/conftest.py -------------------------------------------------------------------------------- /deltacat/tests/compute/converter/integration/test_convert_session.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/converter/integration/test_convert_session.py -------------------------------------------------------------------------------- /deltacat/tests/compute/converter/integration/test_converter_commit_conflict_resolution.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/converter/integration/test_converter_commit_conflict_resolution.py -------------------------------------------------------------------------------- /deltacat/tests/compute/converter/unit/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/converter/unit/__init__.py -------------------------------------------------------------------------------- /deltacat/tests/compute/converter/unit/test_convert.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/converter/unit/test_convert.py -------------------------------------------------------------------------------- /deltacat/tests/compute/converter/unit/test_converter_session_utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/converter/unit/test_converter_session_utils.py -------------------------------------------------------------------------------- /deltacat/tests/compute/converter/unit/test_dedupe.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/converter/unit/test_dedupe.py -------------------------------------------------------------------------------- /deltacat/tests/compute/converter/utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/converter/utils.py -------------------------------------------------------------------------------- /deltacat/tests/compute/resource_estimation/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/compute/resource_estimation/data/DATA.md: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/resource_estimation/data/DATA.md -------------------------------------------------------------------------------- /deltacat/tests/compute/resource_estimation/data/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/compute/resource_estimation/data/date_pk_table.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/resource_estimation/data/date_pk_table.csv -------------------------------------------------------------------------------- /deltacat/tests/compute/resource_estimation/data/sample_no_stats.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/resource_estimation/data/sample_no_stats.parquet -------------------------------------------------------------------------------- /deltacat/tests/compute/resource_estimation/data/sample_with_stats.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/resource_estimation/data/sample_with_stats.parquet -------------------------------------------------------------------------------- /deltacat/tests/compute/resource_estimation/test_delta.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/resource_estimation/test_delta.py -------------------------------------------------------------------------------- /deltacat/tests/compute/resource_estimation/test_manifest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/resource_estimation/test_manifest.py -------------------------------------------------------------------------------- /deltacat/tests/compute/test_compact_partition_incremental.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/test_compact_partition_incremental.py -------------------------------------------------------------------------------- /deltacat/tests/compute/test_compact_partition_multiple_rounds.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/test_compact_partition_multiple_rounds.py -------------------------------------------------------------------------------- /deltacat/tests/compute/test_compact_partition_params.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/test_compact_partition_params.py -------------------------------------------------------------------------------- /deltacat/tests/compute/test_compact_partition_rebase.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/test_compact_partition_rebase.py -------------------------------------------------------------------------------- /deltacat/tests/compute/test_compact_partition_rebase_then_incremental.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/test_compact_partition_rebase_then_incremental.py -------------------------------------------------------------------------------- /deltacat/tests/compute/test_janitor.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/test_janitor.py -------------------------------------------------------------------------------- /deltacat/tests/compute/test_util_common.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/test_util_common.py -------------------------------------------------------------------------------- /deltacat/tests/compute/test_util_constant.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/compute/test_util_constant.py -------------------------------------------------------------------------------- /deltacat/tests/conftest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/conftest.py -------------------------------------------------------------------------------- /deltacat/tests/daft/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/daft/test_model.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/daft/test_model.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/__init__.py: -------------------------------------------------------------------------------- 1 | # Test package for experimental DeltaCAT features 2 | -------------------------------------------------------------------------------- /deltacat/tests/experimental/catalog/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/experimental/catalog/iceberg/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/experimental/catalog/iceberg/test_iceberg_catalog.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/catalog/iceberg/test_iceberg_catalog.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/compatibility/__init__.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/compatibility/__init__.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/compatibility/test_backfill_transaction_partitions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/compatibility/test_backfill_transaction_partitions.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/converter_agent/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/experimental/converter_agent/conftest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/converter_agent/conftest.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/converter_agent/test_callbacks.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/converter_agent/test_callbacks.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/converter_agent/test_table_monitor.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/converter_agent/test_table_monitor.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/daft/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/experimental/daft/test_deltacat_daft_integration.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/daft/test_deltacat_daft_integration.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/iceberg/test_visitor.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/iceberg/test_visitor.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/conftest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/conftest.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/fs/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/fs/test_file_location_provider.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/fs/test_file_location_provider.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/reader/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/reader/query_expression.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/reader/query_expression.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/reader/test_data_scan.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/reader/test_data_scan.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/reader/test_dataset_metastore.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/reader/test_dataset_metastore.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/schema/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/schema/test_schema.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/schema/test_schema.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/shard/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/shard/test_range_shard.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/shard/test_range_shard.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/test_dataset.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/test_dataset.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/test_manifest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/test_manifest.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/test_sst_interval_tree.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/test_sst_interval_tree.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/test_utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/test_utils.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/writer/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/writer/test_dataset_write_then_read.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/writer/test_dataset_write_then_read.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/writer/test_dataset_writer.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/writer/test_dataset_writer.py -------------------------------------------------------------------------------- /deltacat/tests/experimental/storage/rivulet/writer/test_memtable_dataset_writer.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/experimental/storage/rivulet/writer/test_memtable_dataset_writer.py -------------------------------------------------------------------------------- /deltacat/tests/integ/catalog/iceberg/test_local_rest_catalog.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/integ/catalog/iceberg/test_local_rest_catalog.py -------------------------------------------------------------------------------- /deltacat/tests/storage/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/storage/main/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/storage/main/test_delta_ordering.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/main/test_delta_ordering.py -------------------------------------------------------------------------------- /deltacat/tests/storage/main/test_main_storage.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/main/test_main_storage.py -------------------------------------------------------------------------------- /deltacat/tests/storage/model/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/storage/model/test_delete_parameters.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/model/test_delete_parameters.py -------------------------------------------------------------------------------- /deltacat/tests/storage/model/test_expression.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/model/test_expression.py -------------------------------------------------------------------------------- /deltacat/tests/storage/model/test_manifest.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/model/test_manifest.py -------------------------------------------------------------------------------- /deltacat/tests/storage/model/test_metafile_io.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/model/test_metafile_io.py -------------------------------------------------------------------------------- /deltacat/tests/storage/model/test_partition_scheme.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/model/test_partition_scheme.py -------------------------------------------------------------------------------- /deltacat/tests/storage/model/test_schema.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/model/test_schema.py -------------------------------------------------------------------------------- /deltacat/tests/storage/model/test_schema_update.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/model/test_schema_update.py -------------------------------------------------------------------------------- /deltacat/tests/storage/model/test_shard.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/model/test_shard.py -------------------------------------------------------------------------------- /deltacat/tests/storage/model/test_sort_scheme.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/model/test_sort_scheme.py -------------------------------------------------------------------------------- /deltacat/tests/storage/model/test_table_version.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/model/test_table_version.py -------------------------------------------------------------------------------- /deltacat/tests/storage/model/test_transaction.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/model/test_transaction.py -------------------------------------------------------------------------------- /deltacat/tests/storage/model/test_transaction_history.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/storage/model/test_transaction_history.py -------------------------------------------------------------------------------- /deltacat/tests/test_deltacat_api.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/test_deltacat_api.py -------------------------------------------------------------------------------- /deltacat/tests/test_exceptions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/test_exceptions.py -------------------------------------------------------------------------------- /deltacat/tests/test_logs.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/test_logs.py -------------------------------------------------------------------------------- /deltacat/tests/test_utils/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/test_utils/constants.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/test_utils/constants.py -------------------------------------------------------------------------------- /deltacat/tests/test_utils/filesystem.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/test_utils/filesystem.py -------------------------------------------------------------------------------- /deltacat/tests/test_utils/message_pack_utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/test_utils/message_pack_utils.py -------------------------------------------------------------------------------- /deltacat/tests/test_utils/pyarrow.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/test_utils/pyarrow.py -------------------------------------------------------------------------------- /deltacat/tests/test_utils/resources/test_delta.json: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/test_utils/resources/test_delta.json -------------------------------------------------------------------------------- /deltacat/tests/test_utils/storage.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/test_utils/storage.py -------------------------------------------------------------------------------- /deltacat/tests/test_utils/utils.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/test_utils/utils.py -------------------------------------------------------------------------------- /deltacat/tests/types/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/types/test_tables.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/types/test_tables.py -------------------------------------------------------------------------------- /deltacat/tests/utils/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/utils/data/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/utils/data/empty.csv: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/utils/data/mvp.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/data/mvp.parquet -------------------------------------------------------------------------------- /deltacat/tests/utils/data/non_empty_compressed.bz2: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/data/non_empty_compressed.bz2 -------------------------------------------------------------------------------- /deltacat/tests/utils/data/non_empty_compressed.gz: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/data/non_empty_compressed.gz -------------------------------------------------------------------------------- /deltacat/tests/utils/data/non_empty_valid.csv: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/data/non_empty_valid.csv -------------------------------------------------------------------------------- /deltacat/tests/utils/data/overflowing_decimal_precision.csv: -------------------------------------------------------------------------------- 1 | Y 322236.65 2 | "" 32.33 3 | N 0.40 4 | -------------------------------------------------------------------------------- /deltacat/tests/utils/data/overflowing_decimal_scale.csv: -------------------------------------------------------------------------------- 1 | Y 322236.65883738 2 | "" 32.33 3 | N 0.40 4 | -------------------------------------------------------------------------------- /deltacat/tests/utils/data/test_file.parquet: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/data/test_file.parquet -------------------------------------------------------------------------------- /deltacat/tests/utils/data/test_file.parquet.gz: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/data/test_file.parquet.gz -------------------------------------------------------------------------------- /deltacat/tests/utils/exceptions.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/exceptions.py -------------------------------------------------------------------------------- /deltacat/tests/utils/main_deltacat_storage_mock.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/main_deltacat_storage_mock.py -------------------------------------------------------------------------------- /deltacat/tests/utils/ray_utils/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/tests/utils/ray_utils/test_concurrency.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/ray_utils/test_concurrency.py -------------------------------------------------------------------------------- /deltacat/tests/utils/ray_utils/test_dataset.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/ray_utils/test_dataset.py -------------------------------------------------------------------------------- /deltacat/tests/utils/test_cloudpickle.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/test_cloudpickle.py -------------------------------------------------------------------------------- /deltacat/tests/utils/test_daft.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/test_daft.py -------------------------------------------------------------------------------- /deltacat/tests/utils/test_filesystem.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/test_filesystem.py -------------------------------------------------------------------------------- /deltacat/tests/utils/test_metrics.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/test_metrics.py -------------------------------------------------------------------------------- /deltacat/tests/utils/test_numpy.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/test_numpy.py -------------------------------------------------------------------------------- /deltacat/tests/utils/test_pandas.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/test_pandas.py -------------------------------------------------------------------------------- /deltacat/tests/utils/test_placement.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/test_placement.py -------------------------------------------------------------------------------- /deltacat/tests/utils/test_polars.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/test_polars.py -------------------------------------------------------------------------------- /deltacat/tests/utils/test_pyarrow.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/test_pyarrow.py -------------------------------------------------------------------------------- /deltacat/tests/utils/test_record_batch_tables.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/test_record_batch_tables.py -------------------------------------------------------------------------------- /deltacat/tests/utils/test_resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/tests/utils/test_resources.py -------------------------------------------------------------------------------- /deltacat/types/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/types/media.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/types/media.py -------------------------------------------------------------------------------- /deltacat/types/partial_download.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/types/partial_download.py -------------------------------------------------------------------------------- /deltacat/types/tables.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/types/tables.py -------------------------------------------------------------------------------- /deltacat/utils/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/utils/arguments.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/arguments.py -------------------------------------------------------------------------------- /deltacat/utils/cloudpickle.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/cloudpickle.py -------------------------------------------------------------------------------- /deltacat/utils/common.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/common.py -------------------------------------------------------------------------------- /deltacat/utils/daft.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/daft.py -------------------------------------------------------------------------------- /deltacat/utils/export.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/export.py -------------------------------------------------------------------------------- /deltacat/utils/filesystem.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/filesystem.py -------------------------------------------------------------------------------- /deltacat/utils/metafile_locator.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/metafile_locator.py -------------------------------------------------------------------------------- /deltacat/utils/metrics.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/metrics.py -------------------------------------------------------------------------------- /deltacat/utils/numpy.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/numpy.py -------------------------------------------------------------------------------- /deltacat/utils/pandas.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/pandas.py -------------------------------------------------------------------------------- /deltacat/utils/performance.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/performance.py -------------------------------------------------------------------------------- /deltacat/utils/placement.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/placement.py -------------------------------------------------------------------------------- /deltacat/utils/polars.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/polars.py -------------------------------------------------------------------------------- /deltacat/utils/pyarrow.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/pyarrow.py -------------------------------------------------------------------------------- /deltacat/utils/ray_utils/__init__.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /deltacat/utils/ray_utils/collections.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/ray_utils/collections.py -------------------------------------------------------------------------------- /deltacat/utils/ray_utils/concurrency.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/ray_utils/concurrency.py -------------------------------------------------------------------------------- /deltacat/utils/ray_utils/dataset.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/ray_utils/dataset.py -------------------------------------------------------------------------------- /deltacat/utils/ray_utils/performance.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/ray_utils/performance.py -------------------------------------------------------------------------------- /deltacat/utils/ray_utils/runtime.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/ray_utils/runtime.py -------------------------------------------------------------------------------- /deltacat/utils/reader_compatibility_mapping.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/reader_compatibility_mapping.py -------------------------------------------------------------------------------- /deltacat/utils/resources.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/resources.py -------------------------------------------------------------------------------- /deltacat/utils/schema.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/schema.py -------------------------------------------------------------------------------- /deltacat/utils/url.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/deltacat/utils/url.py -------------------------------------------------------------------------------- /dev-requirements.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/dev-requirements.txt -------------------------------------------------------------------------------- /dev/_sandbox/compute/converter/drop_duplicates_benchmark.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/dev/_sandbox/compute/converter/drop_duplicates_benchmark.py -------------------------------------------------------------------------------- /dev/_sandbox/compute/converter/example_single_merge_key_converter.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/dev/_sandbox/compute/converter/example_single_merge_key_converter.py -------------------------------------------------------------------------------- /dev/deploy/aws/scripts/common.py: -------------------------------------------------------------------------------- 1 | -------------------------------------------------------------------------------- /dev/deploy/aws/scripts/runner.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/dev/deploy/aws/scripts/runner.py -------------------------------------------------------------------------------- /dev/deploy/aws/scripts/s3-build-and-deploy.sh: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/dev/deploy/aws/scripts/s3-build-and-deploy.sh -------------------------------------------------------------------------------- /dev/iceberg-integration/Dockerfile: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/dev/iceberg-integration/Dockerfile -------------------------------------------------------------------------------- /dev/iceberg-integration/docker-compose-integration.yml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/dev/iceberg-integration/docker-compose-integration.yml -------------------------------------------------------------------------------- /dev/iceberg-integration/entrypoint.sh: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/dev/iceberg-integration/entrypoint.sh -------------------------------------------------------------------------------- /dev/iceberg-integration/provision.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/dev/iceberg-integration/provision.py -------------------------------------------------------------------------------- /dev/iceberg-integration/spark-defaults.conf: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/dev/iceberg-integration/spark-defaults.conf -------------------------------------------------------------------------------- /media/calico.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/media/calico.jpg -------------------------------------------------------------------------------- /media/customer_feedback_001.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/media/customer_feedback_001.txt -------------------------------------------------------------------------------- /media/customer_feedback_002.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/media/customer_feedback_002.txt -------------------------------------------------------------------------------- /media/customer_feedback_003.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/media/customer_feedback_003.txt -------------------------------------------------------------------------------- /media/deltacat-logo-alpha-750.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/media/deltacat-logo-alpha-750.png -------------------------------------------------------------------------------- /media/deltacat-logo-alpha.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/media/deltacat-logo-alpha.png -------------------------------------------------------------------------------- /media/deltacat-tech-overview.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/media/deltacat-tech-overview.png -------------------------------------------------------------------------------- /media/siamese.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/media/siamese.jpg -------------------------------------------------------------------------------- /media/tuxedo.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/media/tuxedo.jpg -------------------------------------------------------------------------------- /pytest.ini: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/pytest.ini -------------------------------------------------------------------------------- /requirements.txt: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/requirements.txt -------------------------------------------------------------------------------- /setup.py: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/ray-project/deltacat/HEAD/setup.py --------------------------------------------------------------------------------