From 1340c095147753138b8a6a91462b576f2bbb662c Mon Sep 17 00:00:00 2001 From: James Henderson Date: Fri, 4 Aug 2023 10:09:55 +0100 Subject: [PATCH 1/8] remove old scan cursor, #2663 --- core/src/main/clojure/xtdb/operator/scan.clj | 297 ++----------------- 1 file changed, 18 insertions(+), 279 deletions(-) diff --git a/core/src/main/clojure/xtdb/operator/scan.clj b/core/src/main/clojure/xtdb/operator/scan.clj index 7021688788..55e175bb91 100644 --- a/core/src/main/clojure/xtdb/operator/scan.clj +++ b/core/src/main/clojure/xtdb/operator/scan.clj @@ -4,7 +4,6 @@ [juxt.clojars-mirrors.integrant.core :as ig] [xtdb.bloom :as bloom] [xtdb.buffer-pool :as bp] - [xtdb.coalesce :as coalesce] [xtdb.expression :as expr] [xtdb.expression.metadata :as expr.meta] [xtdb.expression.walk :as expr.walk] @@ -19,10 +18,9 @@ [xtdb.vector.reader :as vr] [xtdb.vector.writer :as vw] xtdb.watermark) - (:import (clojure.lang IFn IPersistentMap IPersistentSet MapEntry) - (java.util ArrayList Arrays HashMap Iterator LinkedList List Map Queue Set) - (java.util.function BiFunction Consumer IntConsumer) - java.util.stream.IntStream + (:import (clojure.lang IFn IPersistentMap MapEntry) + (java.util ArrayList Arrays Iterator LinkedList List Map) + (java.util.function IntConsumer) org.apache.arrow.memory.ArrowBuf org.apache.arrow.memory.BufferAllocator [org.apache.arrow.memory.util ArrowBufPointer] @@ -30,7 +28,6 @@ (org.apache.arrow.vector.complex ListVector StructVector) (org.roaringbitmap RoaringBitmap) org.roaringbitmap.buffer.MutableRoaringBitmap - (org.roaringbitmap.longlong Roaring64Bitmap) xtdb.api.protocols.TransactionInstant xtdb.buffer_pool.IBufferPool xtdb.ICursor @@ -69,6 +66,8 @@ (def ^:dynamic *column->pushdown-bloom* {}) +;; TODO reinstate pushdown blooms +#_{:clj-kondo/ignore [:unused-private-var]} (defn- filter-pushdown-bloom-block-idxs [^IMetadataManager metadata-manager chunk-idx ^String table-name ^String col-name ^RoaringBitmap block-idxs] (if-let [^MutableRoaringBitmap pushdown-bloom (get *column->pushdown-bloom* (symbol col-name))] ;; would prefer this `^long` to be on the param but can only have 4 params in a primitive hinted function in Clojure @@ -95,227 +94,6 @@ filtered-block-idxs))))))) block-idxs)) -(deftype ContentChunkCursor [^BufferAllocator allocator - ^IMetadataManager metadata-mgr - ^IBufferPool buffer-pool - table-name content-col-names - ^Queue matching-chunks - ^ICursor current-cursor] - ICursor #_ - (tryAdvance [this c] - (loop [] - (or (when current-cursor - (or (.tryAdvance current-cursor - (reify Consumer - (accept [_ in-root] - (.accept c (-> (vr/<-root in-root) - (vr/with-absent-cols allocator content-col-names)))))) - (do - (util/try-close current-cursor) - (set! (.-current-cursor this) nil) - false))) - - (if-let [{:keys [chunk-idx block-idxs col-names]} (.poll matching-chunks)] - (if-let [block-idxs (reduce (fn [block-idxs col-name] - (or (->> block-idxs - (filter-pushdown-bloom-block-idxs metadata-mgr chunk-idx table-name col-name)) - (reduced nil))) - block-idxs - content-col-names)] - - (do - (set! (.current-cursor this) - (->> (for [col-name (set/intersection col-names content-col-names)] - (-> (.getBuffer buffer-pool (meta/->chunk-obj-key chunk-idx table-name col-name)) - (util/then-apply - (fn [buf] - (MapEntry/create col-name - (util/->chunks buf {:block-idxs block-idxs, :close-buffer? true})))))) - (remove nil?) - vec - (into {} (map deref)) - (util/rethrowing-cause) - (util/combine-col-cursors))) - - (recur)) - - (recur)) - - false)))) - - (close [_] - (some-> current-cursor util/try-close))) - -(defn- ->content-chunks ^xtdb.ICursor [^BufferAllocator allocator - ^IMetadataManager metadata-mgr - ^IBufferPool buffer-pool - table-name content-col-names - metadata-pred] - (ContentChunkCursor. allocator metadata-mgr buffer-pool table-name content-col-names - (LinkedList. (or (meta/matching-chunks metadata-mgr table-name metadata-pred) [])) - nil)) - -(defn- roaring-and - (^org.roaringbitmap.RoaringBitmap [] (RoaringBitmap.)) - (^org.roaringbitmap.RoaringBitmap [^RoaringBitmap x] x) - (^org.roaringbitmap.RoaringBitmap [^RoaringBitmap x ^RoaringBitmap y] - (doto x - (.and y)))) - -(defn- ->atemporal-row-id-bitmap [^BufferAllocator allocator, ^Map col-preds, ^RelationReader in-rel, params] - (let [row-id-rdr (.readerForName in-rel "_row_id") - res (Roaring64Bitmap.)] - - (if-let [content-col-preds (seq (remove (comp temporal/temporal-column? util/str->normal-form-str str key) col-preds))] - (let [^RoaringBitmap - idx-bitmap (->> (for [^IRelationSelector col-pred (vals content-col-preds)] - (RoaringBitmap/bitmapOf (.select col-pred allocator in-rel params))) - (reduce roaring-and))] - (.forEach idx-bitmap - (reify org.roaringbitmap.IntConsumer - (accept [_ idx] - (.addLong res (.getLong row-id-rdr idx)))))) - - (dotimes [idx (.valueCount row-id-rdr)] - (.addLong res (.getLong row-id-rdr idx)))) - - res)) - -(defn- adjust-temporal-min-range-to-row-id-range ^longs [^longs temporal-min-range ^Roaring64Bitmap row-id-bitmap] - (let [temporal-min-range (or (temporal/->copy-range temporal-min-range) (temporal/->min-range))] - (if (.isEmpty row-id-bitmap) - temporal-min-range - (let [min-row-id (.select row-id-bitmap 0)] - (doto temporal-min-range - (aset temporal/row-id-idx - (max min-row-id (aget temporal-min-range temporal/row-id-idx)))))))) - -(defn- adjust-temporal-max-range-to-row-id-range ^longs [^longs temporal-max-range ^Roaring64Bitmap row-id-bitmap] - (let [temporal-max-range (or (temporal/->copy-range temporal-max-range) (temporal/->max-range))] - (if (.isEmpty row-id-bitmap) - temporal-max-range - (let [max-row-id (.select row-id-bitmap (dec (.getLongCardinality row-id-bitmap)))] - (doto temporal-max-range - (aset temporal/row-id-idx - (min max-row-id (aget temporal-max-range temporal/row-id-idx)))))))) - -(defn- select-current-row-ids ^xtdb.vector.RelationReader [^RelationReader content-rel, ^Roaring64Bitmap atemporal-row-id-bitmap, ^IPersistentSet current-row-ids] - (let [sel (IntStream/builder) - row-id-rdr (.readerForName content-rel "_row_id")] - (dotimes [idx (.rowCount content-rel)] - (let [row-id (.getLong row-id-rdr idx)] - (when (and (.contains atemporal-row-id-bitmap row-id) - (.contains current-row-ids row-id)) - (.add sel idx)))) - - (.select content-rel (.toArray (.build sel))))) - -(defn- ->temporal-rel ^xtdb.vector.RelationReader [^IWatermark watermark, ^BufferAllocator allocator, ^List col-names ^longs temporal-min-range ^longs temporal-max-range atemporal-row-id-bitmap] - (let [temporal-min-range (adjust-temporal-min-range-to-row-id-range temporal-min-range atemporal-row-id-bitmap) - temporal-max-range (adjust-temporal-max-range-to-row-id-range temporal-max-range atemporal-row-id-bitmap)] - (.createTemporalRelation (.temporalRootsSource watermark) - allocator - (->> (conj col-names "_row_id") - (into [] (comp (distinct) (filter temporal/temporal-column?)))) - temporal-min-range - temporal-max-range - atemporal-row-id-bitmap))) - -(defn- apply-temporal-preds ^xtdb.vector.RelationReader [^RelationReader temporal-rel, ^BufferAllocator allocator, ^Map col-preds, params] - (->> (for [^IVectorReader col temporal-rel - :let [col-pred (get col-preds (.getName col))] - :when col-pred] - col-pred) - (reduce (fn [^RelationReader temporal-rel, ^IRelationSelector col-pred] - (.select temporal-rel (.select col-pred allocator temporal-rel params))) - temporal-rel))) - -(defn- ->row-id->repeat-count ^java.util.Map [^IVectorReader row-id-col] - (let [res (HashMap.)] - (dotimes [idx (.valueCount row-id-col)] - (let [row-id (.getLong row-id-col idx)] - (.compute res row-id (reify BiFunction - (apply [_ _k v] - (if v - (inc (long v)) - 1)))))) - res)) - -(defn align-vectors ^xtdb.vector.RelationReader [^RelationReader content-rel, ^RelationReader temporal-rel] - ;; assumption: temporal-rel is sorted by row-id - (let [temporal-row-id-col (.readerForName temporal-rel "_row_id") - content-row-id-rdr (.readerForName content-rel "_row_id") - row-id->repeat-count (->row-id->repeat-count temporal-row-id-col) - sel (IntStream/builder)] - (assert temporal-row-id-col) - - (dotimes [idx (.valueCount content-row-id-rdr)] - (let [row-id (.getLong content-row-id-rdr idx)] - (when-let [ns (.get row-id->repeat-count row-id)] - (dotimes [_ ns] - (.add sel idx))))) - - (vr/rel-reader (concat temporal-rel - (.select content-rel (.toArray (.build sel))))))) - -(defn- remove-col ^xtdb.vector.RelationReader [^RelationReader rel, ^String col-name] - (vr/rel-reader (remove #(= col-name (.getName ^IVectorReader %)) rel) - (.rowCount rel))) - -(defn- unnormalize-column-names ^xtdb.vector.RelationReader [^RelationReader rel col-names] - (vr/rel-reader - (map (fn [col-name] - (-> (.readerForName ^RelationReader rel (util/str->normal-form-str col-name)) - (.withName col-name))) - col-names))) - -(deftype ScanCursor [^BufferAllocator allocator - ^IMetadataManager metadata-manager - ^IWatermark watermark - ^Set content-col-names - ^Set temporal-col-names - ^Map col-preds - ^longs temporal-min-range - ^longs temporal-max-range - ^IPersistentSet current-row-ids - ^ICursor #_ blocks - params] - ICursor - (tryAdvance [_ c] - (let [keep-row-id-col? (contains? temporal-col-names "_row_id") - !advanced? (volatile! false) - normalized-temporal-col-names (into #{} (map util/str->normal-form-str) temporal-col-names)] - - (while (and (not @!advanced?) - (.tryAdvance blocks - (reify Consumer - (accept [_ content-rel] - (let [content-rel (unnormalize-column-names content-rel content-col-names) - atemporal-row-id-bitmap (->atemporal-row-id-bitmap allocator col-preds content-rel params)] - (letfn [(accept-rel [^RelationReader read-rel] - (when (and read-rel (pos? (.rowCount read-rel))) - (let [read-rel (cond-> read-rel - (not keep-row-id-col?) (remove-col "_row_id"))] - (.accept c read-rel) - (vreset! !advanced? true))))] - (if current-row-ids - (accept-rel (-> content-rel - (select-current-row-ids atemporal-row-id-bitmap current-row-ids))) - - (let [temporal-rel (->temporal-rel watermark allocator normalized-temporal-col-names - temporal-min-range temporal-max-range atemporal-row-id-bitmap)] - (try - (let [temporal-rel (-> temporal-rel - (unnormalize-column-names (conj temporal-col-names "_row_id")) - (apply-temporal-preds allocator col-preds params))] - (accept-rel (align-vectors content-rel temporal-rel))) - (finally - (util/try-close temporal-rel)))))))))))) - (boolean @!advanced?))) - - (close [_] - (util/try-close blocks))) - (defn ->temporal-min-max-range [^RelationReader params, {^TransactionInstant basis-tx :tx}, {:keys [for-valid-time for-system-time]}, selects] (let [min-range (temporal/->min-range) max-range (temporal/->max-range)] @@ -421,7 +199,7 @@ (def ^:const ^int system-to-lower-idx 6) (def ^:const ^int system-to-upper-idx 7) -(defn- ->temporal-range [^RelationReader params, {^TransactionInstant basis-tx :tx}, {:keys [for-valid-time for-system-time]}, selects] +(defn- ->temporal-range [^RelationReader params, {^TransactionInstant basis-tx :tx}, {:keys [for-valid-time for-system-time]}] (let [range (->range)] (letfn [(apply-bound [f col-name ^long time-μs] (let [range-idx-lower (* (->temporal-column-idx (util/str->normal-form-str (str col-name))) 2) @@ -481,12 +259,6 @@ (defn- scan-op-at-now [scan-op] (= :now (first (second scan-op)))) -(defn- at-now? [{:keys [for-valid-time for-system-time]}] - (and (or (nil? for-valid-time) - (scan-op-at-now for-valid-time)) - (or (nil? for-system-time) - (scan-op-at-now for-system-time)))) - (defn- scan-op-point? [scan-op] (= :at (first scan-op))) @@ -505,23 +277,6 @@ (scan-op-point? for-system-time)) (>= (util/instant->micros (:current-time basis)) (util/instant->micros (:system-time (:tx basis)))))) - -(defn use-current-row-id-cache? [^IWatermark watermark scan-opts basis temporal-col-names] - (and - (.txBasis watermark) - (= (:tx basis) - (.txBasis watermark)) - (at-now? scan-opts) - (>= (util/instant->micros (:current-time basis)) - (util/instant->micros (:system-time (:tx basis)))) - (empty? (remove #(= % "xt$id") temporal-col-names)))) - -(defn get-current-row-ids [^IWatermark watermark basis] - (.getCurrentRowIds - ^xtdb.temporal.ITemporalRelationSource - (.temporalRootsSource watermark) - (util/instant->micros (:current-time basis)))) - (defn tables-with-cols [basis ^IWatermarkSource wm-src ^IScanEmitter scan-emitter] (let [{:keys [tx, after-tx]} basis wm-tx (or tx after-tx)] @@ -1137,7 +892,6 @@ (->> col-names (group-by (comp temporal/temporal-column? util/str->normal-form-str str))) content-col-names (-> (set (map str content-col-names)) (conj "_row_id")) - normalized-content-col-names (set (map (comp util/str->normal-form-str) content-col-names)) temporal-col-names (into #{} (map (comp str)) temporal-col-names) normalized-table-name (util/str->normal-form-str (str table)) @@ -1175,35 +929,20 @@ {:col-types col-types :stats {:row-count row-count} :->cursor (fn [{:keys [allocator, ^IWatermark watermark, basis, params default-all-valid-time?]}] - (let [metadata-pred (expr.meta/->metadata-selector (cons 'and metadata-args) col-types params) + ;; TODO reinstate metadata checks on pages + (let [_metadata-pred (expr.meta/->metadata-selector (cons 'and metadata-args) col-types params) scan-opts (cond-> scan-opts (nil? for-valid-time) - (assoc :for-valid-time (if default-all-valid-time? [:all-time] [:at [:now :now]]))) - [temporal-min-range temporal-max-range] (->temporal-min-max-range params basis scan-opts selects)] - (if true - (->4r-cursor allocator object-store buffer-pool - watermark - normalized-table-name - (set/union content-col-names temporal-col-names) - (->temporal-range params basis scan-opts selects) - col-preds - params - basis - scan-opts) - - (let [current-row-ids (when (use-current-row-id-cache? watermark scan-opts basis temporal-col-names) - (get-current-row-ids watermark basis))] - (-> (ScanCursor. allocator metadata-mgr watermark - content-col-names temporal-col-names col-preds - temporal-min-range temporal-max-range current-row-ids - (util/->concat-cursor (->content-chunks allocator metadata-mgr buffer-pool - normalized-table-name normalized-content-col-names - metadata-pred) - (some-> (.liveChunk watermark) - (.liveTable normalized-table-name) - (.liveBlocks normalized-content-col-names metadata-pred))) - params) - (coalesce/->coalescing-cursor allocator))))))})))) + (assoc :for-valid-time (if default-all-valid-time? [:all-time] [:at [:now :now]])))] + (->4r-cursor allocator object-store buffer-pool + watermark + normalized-table-name + (set/union content-col-names temporal-col-names) + (->temporal-range params basis scan-opts) + col-preds + params + basis + scan-opts)))})))) (defmethod lp/emit-expr :scan [scan-expr {:keys [^IScanEmitter scan-emitter scan-col-types, param-types]}] (.emitScan scan-emitter scan-expr scan-col-types param-types)) From 86407d691e883f6693d44df6a26df142df48e1e0 Mon Sep 17 00:00:00 2001 From: James Henderson Date: Fri, 4 Aug 2023 10:46:39 +0100 Subject: [PATCH 2/8] remove old temporal index, #2663 --- core/src/main/clojure/xtdb/indexer.clj | 117 +--- core/src/main/clojure/xtdb/node.clj | 1 - core/src/main/clojure/xtdb/operator/scan.clj | 97 +-- core/src/main/clojure/xtdb/temporal.clj | 648 ----------------- core/src/main/clojure/xtdb/temporal/grid.clj | 522 -------------- .../main/clojure/xtdb/temporal/histogram.clj | 353 ---------- .../main/clojure/xtdb/temporal/kd_tree.clj | 640 ----------------- core/src/main/clojure/xtdb/types.clj | 8 + core/src/main/clojure/xtdb/watermark.clj | 25 +- dev/doc/kd_tree.adoc | 119 ---- .../clojure/xtdb/bench/multinode_tpch.clj | 9 +- src/main/clojure/xtdb/test_util.clj | 7 +- .../clojure/xtdb/current_row_ids_test.clj | 660 ------------------ src/test/clojure/xtdb/indexer_test.clj | 18 +- src/test/clojure/xtdb/operator/scan_test.clj | 3 +- .../clojure/xtdb/temporal/histogram_test.clj | 51 -- .../xtdb/temporal/kd_tree_microbench_test.clj | 109 --- .../clojure/xtdb/temporal/kd_tree_test.clj | 476 ------------- .../chunk-00/temporal.arrow.json | 40 -- .../temporal-snapshots/00.arrow.json | 40 -- .../chunk-00/foo/content-_row_id.arrow.json | 2 +- .../chunk-00/foo/metadata.arrow.json | 4 +- .../chunk-00/temporal.arrow.json | 40 -- .../xt$txs/content-_row_id.arrow.json | 2 +- .../chunk-00/xt$txs/metadata.arrow.json | 6 +- .../chunk-metadata/00.transit.json | 2 +- .../tables/foo/chunks/leaf-c00.arrow.json | 2 +- .../tables/xt$txs/chunks/leaf-c00.arrow.json | 2 +- .../temporal-snapshots/00.arrow.json | 40 -- .../chunk-00/temporal.arrow.json | 40 -- .../temporal-snapshots/00.arrow.json | 40 -- .../chunk-00/temporal.arrow.json | 40 -- .../temporal-snapshots/00.arrow.json | 40 -- .../chunk-00/temporal.arrow.json | 40 -- .../temporal-snapshots/00.arrow.json | 40 -- .../chunk-00/temporal.arrow.json | 40 -- .../temporal-snapshots/00.arrow.json | 40 -- 37 files changed, 76 insertions(+), 4287 deletions(-) delete mode 100644 core/src/main/clojure/xtdb/temporal.clj delete mode 100644 core/src/main/clojure/xtdb/temporal/grid.clj delete mode 100644 core/src/main/clojure/xtdb/temporal/histogram.clj delete mode 100644 core/src/main/clojure/xtdb/temporal/kd_tree.clj delete mode 100644 dev/doc/kd_tree.adoc delete mode 100644 src/test/clojure/xtdb/current_row_ids_test.clj delete mode 100644 src/test/clojure/xtdb/temporal/histogram_test.clj delete mode 100644 src/test/clojure/xtdb/temporal/kd_tree_microbench_test.clj delete mode 100644 src/test/clojure/xtdb/temporal/kd_tree_test.clj delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/temporal.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/temporal-snapshots/00.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/temporal.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/temporal-snapshots/00.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/temporal.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/temporal-snapshots/00.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/temporal.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/temporal-snapshots/00.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/temporal.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/temporal-snapshots/00.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/temporal.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/temporal-snapshots/00.arrow.json diff --git a/core/src/main/clojure/xtdb/indexer.clj b/core/src/main/clojure/xtdb/indexer.clj index 58ebe610a3..519fdaee12 100644 --- a/core/src/main/clojure/xtdb/indexer.clj +++ b/core/src/main/clojure/xtdb/indexer.clj @@ -13,9 +13,8 @@ [xtdb.operator.scan :as scan] [xtdb.rewrite :refer [zmatch]] [xtdb.sql :as sql] - [xtdb.temporal :as temporal] [xtdb.tx-producer :as txp] - [xtdb.types :as t] + [xtdb.types :as types] [xtdb.util :as util] [xtdb.vector :as vec] [xtdb.vector.reader :as vr] @@ -41,8 +40,7 @@ xtdb.object_store.ObjectStore xtdb.operator.IRaQuerySource (xtdb.operator.scan IScanEmitter) - (xtdb.temporal ITemporalManager ITemporalTxIndexer) - (xtdb.vector IRowCopier IVectorReader IVectorWriter RelationReader) + (xtdb.vector IRowCopier IVectorReader RelationReader) (xtdb.watermark IWatermark IWatermarkSource))) (set! *unchecked-math* :warn-on-boxed) @@ -83,8 +81,7 @@ (.digest eid-bytes) (Arrays/copyOfRange 0 16))))) -(defn- ->put-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^ILiveIndexTx live-idx-tx, - ^ITemporalTxIndexer temporal-idxer, ^ILiveChunkTx live-chunk, +(defn- ->put-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk, ^IVectorReader tx-ops-rdr, ^Instant system-time] (let [put-leg (.legReader tx-ops-rdr :put) doc-rdr (.structKeyReader put-leg "document") @@ -124,8 +121,7 @@ (.writeRowId live-table row-id) (.copyRow table-copier tx-op-idx)) - (let [new-entity? (not (.isKnownId iid-mgr table-name eid)) - legacy-iid (.getOrCreateInternalId iid-mgr table-name eid row-id) + (let [legacy-iid (.getOrCreateInternalId iid-mgr table-name eid row-id) valid-from (if (= :null (.getLeg valid-from-rdr tx-op-idx)) system-time-µs (.getLong valid-from-rdr tx-op-idx)) @@ -138,14 +134,11 @@ :valid-to (util/micros->instant valid-to)}))) (let [{:keys [^xtdb.indexer.live_index.ILiveTableTx live-table, ^IRowCopier doc-copier]} live-idx-table] - (.logPut live-table (->iid eid) legacy-iid valid-from valid-to #(.copyRow doc-copier tx-op-idx))) - - (.indexPut temporal-idxer legacy-iid row-id valid-from valid-to new-entity?))) + (.logPut live-table (->iid eid) legacy-iid valid-from valid-to #(.copyRow doc-copier tx-op-idx))))) nil)))) -(defn- ->delete-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^ILiveIndexTx live-idx-tx - ^ITemporalTxIndexer temporal-idxer, ^ILiveChunkTx live-chunk +(defn- ->delete-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^ILiveIndexTx live-idx-tx ^ILiveChunkTx live-chunk ^IVectorReader tx-ops-rdr, ^Instant current-time] (let [delete-leg (.legReader tx-ops-rdr :delete) table-rdr (.structKeyReader delete-leg "table") @@ -158,7 +151,6 @@ (let [row-id (.nextRowId live-chunk) table (.getObject table-rdr tx-op-idx) eid (.getObject id-rdr tx-op-idx) - new-entity? (not (.isKnownId iid-mgr table eid)) legacy-iid (.getOrCreateInternalId iid-mgr table eid row-id) valid-from (if (= :null (.getLeg valid-from-rdr tx-op-idx)) current-time-µs @@ -172,15 +164,11 @@ :valid-to (util/micros->instant valid-to)}))) (-> (.liveTable live-idx-tx table) - (.logDelete (->iid eid) legacy-iid valid-from valid-to)) - - (.indexDelete temporal-idxer legacy-iid row-id valid-from valid-to new-entity?)) + (.logDelete (->iid eid) legacy-iid valid-from valid-to))) nil)))) -(defn- ->evict-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^ILiveIndexTx live-idx-tx - ^ITemporalTxIndexer temporal-idxer, ^ILiveChunkTx live-chunk - ^IVectorReader tx-ops-rdr] +(defn- ->evict-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^ILiveIndexTx live-idx-tx ^ILiveChunkTx live-chunk ^IVectorReader tx-ops-rdr] (let [evict-leg (.legReader tx-ops-rdr :evict) table-rdr (.structKeyReader evict-leg "_table") @@ -193,9 +181,7 @@ legacy-iid (.getOrCreateInternalId iid-mgr table eid row-id)] (-> (.liveTable live-idx-tx table) - (.logEvict (->iid eid) legacy-iid)) - - (.indexEvict temporal-idxer legacy-iid)) + (.logEvict (->iid eid) legacy-iid))) nil)))) @@ -311,8 +297,7 @@ (definterface SqlOpIndexer (^void indexOp [^xtdb.vector.RelationReader inRelation, queryOpts])) -(defn- ->sql-upsert-indexer ^xtdb.indexer.SqlOpIndexer [^IInternalIdManager iid-mgr, ^ILiveIndexTx live-idx-tx - ^ITemporalTxIndexer temporal-idxer, ^ILiveChunkTx live-chunk, +(defn- ->sql-upsert-indexer ^xtdb.indexer.SqlOpIndexer [^IInternalIdManager iid-mgr, ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk, {{:keys [^Instant current-time]} :basis}] (let [current-time-µs (util/instant->micros current-time)] @@ -320,8 +305,7 @@ (indexOp [_ in-rel {:keys [table]}] (let [row-count (.rowCount in-rel) content-rel (vr/rel-reader (->> in-rel - (remove (comp temporal/temporal-column? - #(.getName ^IVectorReader %))) + (remove (comp types/temporal-column? #(.getName ^IVectorReader %))) (map (fn [^IVectorReader vec] (.withName vec (util/str->normal-form-str (.getName vec)))))) (.rowCount in-rel)) @@ -344,7 +328,6 @@ (.copyRow table-copier idx) (let [eid (.getObject id-col idx) - new-entity? (not (.isKnownId iid-mgr table eid)) legacy-iid (.getOrCreateInternalId iid-mgr table eid row-id) valid-from (if (and valid-from-rdr (= :timestamp-tz-micro-utc (.getLeg valid-from-rdr idx))) (.getLong valid-from-ts-rdr idx) @@ -357,10 +340,9 @@ {:valid-from (util/micros->instant valid-from) :valid-to (util/micros->instant valid-to)}))) - (.indexPut temporal-idxer legacy-iid row-id valid-from valid-to new-entity?) (.logPut live-idx-table (->iid eid) legacy-iid valid-from valid-to #(.copyRow live-idx-table-copier idx)))))))))) -(defn- ->sql-delete-indexer ^xtdb.indexer.SqlOpIndexer [^ILiveIndexTx live-idx-tx, ^ITemporalTxIndexer temporal-idxer, ^ILiveChunkTx live-chunk] +(defn- ->sql-delete-indexer ^xtdb.indexer.SqlOpIndexer [^ILiveIndexTx live-idx-tx] (reify SqlOpIndexer (indexOp [_ in-rel {:keys [table]}] (let [table (util/str->normal-form-str table) @@ -370,8 +352,7 @@ valid-from-rdr (.readerForName in-rel "xt$valid_from") valid-to-rdr (.readerForName in-rel "xt$valid_to")] (dotimes [idx row-count] - (let [row-id (.nextRowId live-chunk) - eid (.getObject id-rdr idx) + (let [eid (.getObject id-rdr idx) iid (.getLong iid-rdr idx) valid-from (.getLong valid-from-rdr idx) valid-to (.getLong valid-to-rdr idx)] @@ -381,10 +362,9 @@ :valid-to (util/micros->instant valid-to)}))) (-> (.liveTable live-idx-tx table) - (.logDelete (->iid eid) iid valid-from valid-to)) - (.indexDelete temporal-idxer iid row-id valid-from valid-to false))))))) + (.logDelete (->iid eid) iid valid-from valid-to)))))))) -(defn- ->sql-erase-indexer ^xtdb.indexer.SqlOpIndexer [^ILiveIndexTx live-idx-tx, ^ITemporalTxIndexer temporal-idxer] +(defn- ->sql-erase-indexer ^xtdb.indexer.SqlOpIndexer [^ILiveIndexTx live-idx-tx] (reify SqlOpIndexer (indexOp [_ in-rel {:keys [table]}] (let [table (util/str->normal-form-str table) @@ -395,20 +375,18 @@ (let [eid (.getObject id-rdr idx) iid (.getLong iid-rdr idx)] (-> (.liveTable live-idx-tx table) - (.logEvict (->iid eid) iid)) - (.indexEvict temporal-idxer iid))))))) + (.logEvict (->iid eid) iid)))))))) (defn- ->sql-indexer ^xtdb.indexer.OpIndexer [^BufferAllocator allocator, ^IInternalIdManager iid-mgr - ^ILiveIndexTx live-idx-tx, ^ITemporalTxIndexer temporal-idxer, ^ILiveChunk doc-idxer + ^ILiveIndexTx live-idx-tx, ^ILiveChunk doc-idxer ^IVectorReader tx-ops-rdr, ^IRaQuerySource ra-src, wm-src, ^IScanEmitter scan-emitter {:keys [default-all-valid-time? basis default-tz] :as tx-opts}] (let [sql-leg (.legReader tx-ops-rdr :sql) query-rdr (.structKeyReader sql-leg "query") params-rdr (.structKeyReader sql-leg "params") - upsert-idxer (->sql-upsert-indexer iid-mgr live-idx-tx - temporal-idxer doc-idxer tx-opts) - delete-idxer (->sql-delete-indexer live-idx-tx temporal-idxer doc-idxer) - erase-idxer (->sql-erase-indexer live-idx-tx temporal-idxer)] + upsert-idxer (->sql-upsert-indexer iid-mgr live-idx-tx doc-idxer tx-opts) + delete-idxer (->sql-delete-indexer live-idx-tx) + erase-idxer (->sql-erase-indexer live-idx-tx)] (reify OpIndexer (indexOp [_ tx-op-idx] (letfn [(index-op [^SqlOpIndexer op-idxer {:keys [all-app-time] :as query-opts} inner-query] @@ -466,7 +444,7 @@ (def ^:private ^:const ^String txs-table "xt$txs") -(defn- add-tx-row! [^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk-tx, ^ITemporalTxIndexer temporal-tx, ^IInternalIdManager iid-mgr, ^TransactionInstant tx-key, ^Throwable t] +(defn- add-tx-row! [^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk-tx, ^IInternalIdManager iid-mgr, ^TransactionInstant tx-key, ^Throwable t] (let [tx-id (.tx-id tx-key) system-time-µs (util/instant->micros (.system-time tx-key)) @@ -481,7 +459,7 @@ (doto (.structKeyWriter doc-writer "xt$id" :i64) (.writeLong tx-id)) - (doto (.structKeyWriter doc-writer "xt$tx_time" t/temporal-col-type) + (doto (.structKeyWriter doc-writer "xt$tx_time" types/temporal-col-type) (.writeLong system-time-µs)) (doto (.structKeyWriter doc-writer "xt$committed?" :bool) @@ -503,7 +481,7 @@ (doto (.writerForName doc-writer "xt$id" :i64) (.writeLong tx-id)) - (doto (.writerForName doc-writer "xt$tx_time" t/temporal-col-type) + (doto (.writerForName doc-writer "xt$tx_time" types/temporal-col-type) (.writeLong system-time-µs)) (doto (.writerForName doc-writer "xt$committed?" :bool) @@ -514,14 +492,11 @@ (doto (.writerForType e-wtr :null) (.writeNull nil)) (doto (.writerForType e-wtr :clj-form) - (.writeObject (pr-str t))))) - - (.indexPut temporal-tx legacy-iid row-id system-time-µs util/end-of-time-μs true)))) + (.writeObject (pr-str t)))))))) (deftype Indexer [^BufferAllocator allocator ^ObjectStore object-store ^IScanEmitter scan-emitter - ^ITemporalManager temporal-mgr ^IInternalIdManager iid-mgr ^IRaQuerySource ra-src ^ILiveChunk live-chunk @@ -539,11 +514,9 @@ (let [^DenseUnionVector tx-ops-vec (-> ^ListVector (.getVector tx-root "tx-ops") (.getDataVector)) - temporal-idxer (.startTx temporal-mgr tx-key) - wm-src (reify IWatermarkSource (openWatermark [_ _tx] - (wm/->wm nil (.openWatermark live-chunk-tx) (.openWatermark live-idx-tx) temporal-idxer false))) + (wm/->wm nil (.openWatermark live-chunk-tx) (.openWatermark live-idx-tx)))) tx-opts {:basis {:tx tx-key, :current-time system-time} :default-tz (ZoneId/of (str (-> (.getVector tx-root "default-tz") @@ -554,12 +527,11 @@ (letfn [(index-tx-ops [^DenseUnionVector tx-ops-vec] (let [tx-ops-rdr (vr/vec->reader tx-ops-vec) - !put-idxer (delay (->put-indexer iid-mgr live-idx-tx temporal-idxer live-chunk-tx tx-ops-rdr system-time)) - !delete-idxer (delay (->delete-indexer iid-mgr live-idx-tx temporal-idxer live-chunk-tx tx-ops-rdr system-time)) - !evict-idxer (delay (->evict-indexer iid-mgr live-idx-tx temporal-idxer live-chunk-tx tx-ops-rdr)) + !put-idxer (delay (->put-indexer iid-mgr live-idx-tx live-chunk-tx tx-ops-rdr system-time)) + !delete-idxer (delay (->delete-indexer iid-mgr live-idx-tx live-chunk-tx tx-ops-rdr system-time)) + !evict-idxer (delay (->evict-indexer iid-mgr live-idx-tx live-chunk-tx tx-ops-rdr)) !call-idxer (delay (->call-indexer allocator ra-src wm-src scan-emitter tx-ops-rdr tx-opts)) - !sql-idxer (delay (->sql-indexer allocator iid-mgr live-idx-tx - temporal-idxer live-chunk-tx + !sql-idxer (delay (->sql-indexer allocator iid-mgr live-idx-tx live-chunk-tx tx-ops-rdr ra-src wm-src scan-emitter tx-opts))] (dotimes [tx-op-idx (.valueCount tx-ops-rdr)] (when-let [more-tx-ops (case (.getTypeId tx-ops-rdr tx-op-idx) @@ -578,14 +550,11 @@ (catch xtdb.RuntimeException e e) (catch xtdb.IllegalArgumentException e e) (catch ClosedByInterruptException e - (.abort temporal-idxer) (throw (InterruptedException. (.toString e)))) (catch InterruptedException e - (.abort temporal-idxer) (throw e)) (catch Throwable t (log/error t "error in indexer") - (.abort temporal-idxer) (throw t))) wm-lock-stamp (.writeLock wm-lock)] (try @@ -593,27 +562,18 @@ (do (when (not= e abort-exn) (log/debug e "aborted tx")) - (.abort temporal-idxer) (with-open [live-chunk-tx (.startTx live-chunk) live-idx-tx (.startTx live-idx tx-key)] - (let [temporal-tx (.startTx temporal-mgr tx-key)] - (add-tx-row! live-idx-tx live-chunk-tx temporal-tx iid-mgr tx-key e) - (.commit live-chunk-tx) - (.commit live-idx-tx) - (.commit temporal-tx)))) + (add-tx-row! live-idx-tx live-chunk-tx iid-mgr tx-key e) + (.commit live-chunk-tx) + (.commit live-idx-tx))) (do - (add-tx-row! live-idx-tx live-chunk-tx temporal-idxer iid-mgr tx-key nil) + (add-tx-row! live-idx-tx live-chunk-tx iid-mgr tx-key nil) (.commit live-chunk-tx) - (.commit live-idx-tx) - - (let [evicted-row-ids (.commit temporal-idxer)] - #_{:clj-kondo/ignore [:missing-body-in-when]} - (when-not (.isEmpty evicted-row-ids) - ;; TODO create work item - )))) + (.commit live-idx-tx))) (set! (.-latest-completed-tx this) tx-key) @@ -648,8 +608,7 @@ (or (maybe-existing-wm) (let [^IWatermark old-wm (.shared-wm this)] (try - (let [^IWatermark shared-wm (wm/->wm latest-completed-tx (.openWatermark live-chunk) (.openWatermark live-idx) - (.getTemporalWatermark temporal-mgr) true)] + (let [^IWatermark shared-wm (wm/->wm latest-completed-tx (.openWatermark live-chunk) (.openWatermark live-idx))] (set! (.shared-wm this) shared-wm) (doto shared-wm .retain)) (finally @@ -682,7 +641,6 @@ (finish-chunk! [this] (let [chunk-idx (.chunkIdx live-chunk)] - (.registerNewChunk temporal-mgr chunk-idx) @(.finishChunk live-chunk latest-completed-tx) (.finishChunk live-idx chunk-idx)) @@ -709,7 +667,6 @@ :object-store (ig/ref :xtdb/object-store) :metadata-mgr (ig/ref ::meta/metadata-manager) :scan-emitter (ig/ref :xtdb.operator.scan/scan-emitter) - :temporal-mgr (ig/ref ::temporal/temporal-manager) :internal-id-mgr (ig/ref :xtdb.indexer/internal-id-manager) :live-chunk (ig/ref :xtdb/live-chunk) :live-index (ig/ref :xtdb.indexer/live-index) @@ -717,12 +674,12 @@ opts)) (defmethod ig/init-key :xtdb/indexer - [_ {:keys [allocator object-store metadata-mgr scan-emitter ^ITemporalManager temporal-mgr, ra-src + [_ {:keys [allocator object-store metadata-mgr scan-emitter, ra-src internal-id-mgr live-chunk live-index]}] (let [{:keys [latest-completed-tx]} (meta/latest-chunk-metadata metadata-mgr)] - (Indexer. allocator object-store scan-emitter temporal-mgr internal-id-mgr + (Indexer. allocator object-store scan-emitter internal-id-mgr ra-src live-chunk live-index latest-completed-tx diff --git a/core/src/main/clojure/xtdb/node.clj b/core/src/main/clojure/xtdb/node.clj index 7a1356e6e7..32d7ece725 100644 --- a/core/src/main/clojure/xtdb/node.clj +++ b/core/src/main/clojure/xtdb/node.clj @@ -129,7 +129,6 @@ :xtdb.indexer/live-index {} :xtdb/ingester {} :xtdb.metadata/metadata-manager {} - :xtdb.temporal/temporal-manager {} :xtdb.buffer-pool/buffer-pool {} :xtdb.operator.scan/scan-emitter {} :xtdb.operator/ra-query-source {} diff --git a/core/src/main/clojure/xtdb/operator/scan.clj b/core/src/main/clojure/xtdb/operator/scan.clj index 55e175bb91..9d871ca6d9 100644 --- a/core/src/main/clojure/xtdb/operator/scan.clj +++ b/core/src/main/clojure/xtdb/operator/scan.clj @@ -2,16 +2,13 @@ (:require [clojure.set :as set] [clojure.spec.alpha :as s] [juxt.clojars-mirrors.integrant.core :as ig] - [xtdb.bloom :as bloom] [xtdb.buffer-pool :as bp] [xtdb.expression :as expr] [xtdb.expression.metadata :as expr.meta] - [xtdb.expression.walk :as expr.walk] xtdb.indexer.live-index [xtdb.logical-plan :as lp] [xtdb.metadata :as meta] xtdb.object-store - [xtdb.temporal :as temporal] [xtdb.trie :as trie] [xtdb.types :as types] [xtdb.util :as util] @@ -24,10 +21,8 @@ org.apache.arrow.memory.ArrowBuf org.apache.arrow.memory.BufferAllocator [org.apache.arrow.memory.util ArrowBufPointer] - (org.apache.arrow.vector BigIntVector NullVector VarBinaryVector VectorLoader VectorSchemaRoot) + (org.apache.arrow.vector BigIntVector NullVector VectorLoader VectorSchemaRoot) (org.apache.arrow.vector.complex ListVector StructVector) - (org.roaringbitmap RoaringBitmap) - org.roaringbitmap.buffer.MutableRoaringBitmap xtdb.api.protocols.TransactionInstant xtdb.buffer_pool.IBufferPool xtdb.ICursor @@ -66,8 +61,7 @@ (def ^:dynamic *column->pushdown-bloom* {}) -;; TODO reinstate pushdown blooms -#_{:clj-kondo/ignore [:unused-private-var]} +#_ ; TODO reinstate pushdown blooms (defn- filter-pushdown-bloom-block-idxs [^IMetadataManager metadata-manager chunk-idx ^String table-name ^String col-name ^RoaringBitmap block-idxs] (if-let [^MutableRoaringBitmap pushdown-bloom (get *column->pushdown-bloom* (symbol col-name))] ;; would prefer this `^long` to be on the param but can only have 4 params in a primitive hinted function in Clojure @@ -94,87 +88,6 @@ filtered-block-idxs))))))) block-idxs)) -(defn ->temporal-min-max-range [^RelationReader params, {^TransactionInstant basis-tx :tx}, {:keys [for-valid-time for-system-time]}, selects] - (let [min-range (temporal/->min-range) - max-range (temporal/->max-range)] - (letfn [(apply-bound [f col-name ^long time-μs] - (let [range-idx (temporal/->temporal-column-idx (util/str->normal-form-str (str col-name)))] - (case f - :< (aset max-range range-idx - (min (dec time-μs) (aget max-range range-idx))) - :<= (aset max-range range-idx - (min time-μs (aget max-range range-idx))) - :> (aset min-range range-idx - (max (inc time-μs) (aget min-range range-idx))) - :>= (aset min-range range-idx - (max time-μs (aget min-range range-idx))) - nil))) - - (->time-μs [[tag arg]] - (case tag - :literal (-> arg - (util/sql-temporal->micros (.getZone expr/*clock*))) - :param (-> (-> (.readerForName params (name arg)) - (.getObject 0)) - (util/sql-temporal->micros (.getZone expr/*clock*))) - :now (-> (.instant expr/*clock*) - (util/instant->micros))))] - - (when-let [system-time (some-> basis-tx (.system-time) util/instant->micros)] - (apply-bound :<= "xt$system_from" system-time) - - (when-not for-system-time - (apply-bound :> "xt$system_to" system-time))) - - (letfn [(apply-constraint [constraint start-col end-col] - (when-let [[tag & args] constraint] - (case tag - :at (let [[at] args - at-μs (->time-μs at)] - (apply-bound :<= start-col at-μs) - (apply-bound :> end-col at-μs)) - - ;; overlaps [time-from time-to] - :in (let [[from to] args] - (apply-bound :> end-col (->time-μs (or from [:now]))) - (when to - (apply-bound :< start-col (->time-μs to)))) - - :between (let [[from to] args] - (apply-bound :> end-col (->time-μs (or from [:now]))) - (when to - (apply-bound :<= start-col (->time-μs to)))) - - :all-time nil)))] - - (apply-constraint for-valid-time "xt$valid_from" "xt$valid_to") - (apply-constraint for-system-time "xt$system_from" "xt$system_to")) - - (let [col-types (into {} (map (juxt first #(get temporal/temporal-col-types (util/str->normal-form-str (str (first %)))))) selects) - param-types (expr/->param-types params)] - (doseq [[col-name select-form] selects - :when (temporal/temporal-column? (util/str->normal-form-str (str col-name)))] - (->> (-> (expr/form->expr select-form {:param-types param-types, :col-types col-types}) - (expr/prepare-expr) - (expr.meta/meta-expr {:col-types col-types})) - (expr.walk/prewalk-expr - (fn [{:keys [op] :as expr}] - (case op - :call (when (not= :or (:f expr)) - expr) - - :metadata-vp-call - (let [{:keys [f param-expr]} expr] - (when-let [v (if-let [[_ literal] (find param-expr :literal)] - (when literal (->time-μs [:literal literal])) - (->time-μs [:param (get param-expr :param)]))] - (apply-bound f col-name v))) - - expr))))) - [min-range max-range])) - - [min-range max-range])) - (defn- ->range ^longs [] (let [res (long-array 8)] (doseq [i (range 0 8 2)] @@ -869,7 +782,7 @@ (letfn [(->col-type [[table col-name]] (let [normalized-table (util/str->normal-form-str (str table)) normalized-col-name (util/str->normal-form-str (str col-name))] - (if (temporal/temporal-column? (util/str->normal-form-str (str col-name))) + (if (types/temporal-column? (util/str->normal-form-str (str col-name))) [:timestamp-tz :micro "UTC"] (types/merge-col-types (.columnType metadata-mgr normalized-table normalized-col-name) (some-> (.liveChunk wm) @@ -889,7 +802,7 @@ (distinct)))) {content-col-names false, temporal-col-names true} - (->> col-names (group-by (comp temporal/temporal-column? util/str->normal-form-str str))) + (->> col-names (group-by (comp types/temporal-column? util/str->normal-form-str str))) content-col-names (-> (set (map str content-col-names)) (conj "_row_id")) temporal-col-names (into #{} (map (comp str)) temporal-col-names) @@ -912,7 +825,7 @@ (into {})) metadata-args (vec (for [[col-name select] selects - :when (not (temporal/temporal-column? (util/str->normal-form-str (str col-name))))] + :when (not (types/temporal-column? (util/str->normal-form-str (str col-name))))] select)) row-count (->> (meta/with-all-metadata metadata-mgr normalized-table-name diff --git a/core/src/main/clojure/xtdb/temporal.clj b/core/src/main/clojure/xtdb/temporal.clj deleted file mode 100644 index 83819a86c0..0000000000 --- a/core/src/main/clojure/xtdb/temporal.clj +++ /dev/null @@ -1,648 +0,0 @@ -(ns xtdb.temporal - (:require [clojure.set :as set] - [clojure.tools.logging :as log] - [juxt.clojars-mirrors.integrant.core :as ig] - xtdb.buffer-pool - [xtdb.metadata :as meta] - [xtdb.temporal.grid :as grid] - [xtdb.temporal.kd-tree :as kd] - [xtdb.types :as types] - [xtdb.util :as util] - [xtdb.vector.reader :as vr] - [xtdb.vector.writer :as vw]) - (:import java.lang.AutoCloseable - [java.util ArrayList Arrays Comparator] - [java.util.concurrent CompletableFuture ExecutorService Executors] - [java.util.function LongFunction Predicate ToLongFunction] - java.util.stream.LongStream - [org.apache.arrow.memory ArrowBuf BufferAllocator] - org.apache.arrow.vector.BaseFixedWidthVector - org.roaringbitmap.longlong.Roaring64Bitmap - xtdb.buffer_pool.IBufferPool - xtdb.metadata.IMetadataManager - xtdb.object_store.ObjectStore - [xtdb.temporal.kd_tree IKdTreePointAccess MergedKdTree])) - -;; Temporal proof-of-concept plan: - -;; From a BCDM point of view, core2 (and XTDB) are similar to Jensen's -;; event log approach, that is, we know system-time, and we know the app-time -;; range, but not the actual real state as expressed in the Snodgrass' -;; timestamped tuple approach, which is the relation we want scan to -;; produce. Theoretically, one can map between these via the BCDM, as -;; described in the paper for snapshot equivalent representations, and -;; that serves as a good reference, but not practical. - -;; The only update that needs to happen to the append only data is -;; setting system-time-end to the current system-time when closing -;; rows. Working around this is what the current uni-temporal system-time -;; support does. This fact will help later when and if we decide to -;; store the temporal index per chunk in Arrow and merge between them. - -;; Further, I think we can decide that a put or delete always know its -;; full app-time range, that is, if app-time isn't known it's set to system-time, -;; and if app-time-end isn't know, it's set to end-of-time (at least -;; for the proof-of-concept). - -;; In the temporal index structure, this means that when you do a put -;; (delete) you find any current rows (system-time-end == UC) for the id -;; that overlaps the app-time range, and mark those rows with the -;; system-time-end to current system-time (the part that cannot be done append -;; only). You then insert the new row entry (for put) normally. If the -;; put (delete) didn't fully overlap you copy the start (and/or) end -;; partial row entries forward, referring to the original row-id, -;; updating their app-time-end (for start) and app-time (for end) to -;; match the slice, you also set system-time to that of the current tx, -;; and system-time-end to UC. - -;; We assume that the column store has a 1-to-1 mapping between -;; operations and row-ids, but the temporal index can refer to them -;; more than once in the case of splits. These could also be stored in -;; the column store if we later decide to break the 1-to-1 mapping. - -;; For simplicitly, let's assume that this structure is an in-memory -;; kd-tree for now with 6 dimensions: id, row-id, app-time, -;; app-time-end, system-time, system-time-end. When updating system-time-end, one -;; has a few options, either one deletes the node and reinserts it, or -;; one can have an extra value (not part of the actual index), -;; system-time-delete, which if it exists, supersedes system-time-end when -;; doing the element-level comparision. That would imply that these -;; nodes would needlessly be found by the kd-tree navigation itself, -;; so moving them might be better. But a reason to try to avoid moving -;; nodes is that later this tree can be an implicit kd-tree stored as -;; Arrow, one per chunk, and the query would need to merge them. How -;; to solve this problem well can be saved for later. - -;; Once this structure exists, it could also potentially be used to -;; replace the tombstone check (to see if a row is a deletion) I added -;; as those rows won't sit in the tree. But again, we can postpone -;; that, as this might be superseded by a per-row _op struct. - -(set! *unchecked-math* :warn-on-boxed) - -(def ^:const ^int k 6) - -(defn ->min-range ^longs [] - (long-array k Long/MIN_VALUE)) - -(defn ->max-range ^longs [] - (long-array k Long/MAX_VALUE)) - -(defn ->copy-range ^longs [^longs range] - (some-> range (Arrays/copyOf (alength range)))) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface ITemporalRelationSource - (^xtdb.vector.RelationReader createTemporalRelation [^org.apache.arrow.memory.BufferAllocator allocator - ^java.util.List columns - ^longs temporalMinRange - ^longs temporalMaxRange - ^org.roaringbitmap.longlong.Roaring64Bitmap rowIdBitmap]) - - (^java.util.Set getCurrentRowIds [^long current-time])) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface ITemporalTxIndexer - (^void indexPut [^long iid, ^long rowId, ^long startValidTime, ^long endValidTime, ^boolean newEntity]) - (^void indexDelete [^long iid, ^long rowId, ^long startValidTime, ^long endValidTime, ^boolean newEntity]) - (^void indexEvict [^long iid]) - (^org.roaringbitmap.longlong.Roaring64Bitmap commit []) - (^void abort [])) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface ITemporalManager - (^xtdb.temporal.ITemporalRelationSource getTemporalWatermark []) - (^xtdb.vector.RelationReader createTemporalRelation [^org.apache.arrow.memory.BufferAllocator allocator - ^java.util.List columns - ^longs temporalMinRange - ^longs temporalMaxRange - ^org.roaringbitmap.longlong.Roaring64Bitmap rowIdBitmap]) - (^void registerNewChunk [^long chunkIdx]) - (^xtdb.temporal.ITemporalTxIndexer startTx [^xtdb.api.protocols.TransactionInstant txKey])) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface TemporalManagerPrivate - (^void populateKnownChunks []) - (^Long latestTemporalSnapshotIndex [^int chunk-idx]) - (^void reloadTemporalIndex [^int chunk-idx ^Long snapshot-idx]) - (^void awaitSnapshotBuild []) - (^void buildTemporalSnapshot [^int chunk-idx ^Long snapshot-idx]) - (^AutoCloseable buildStaticTree [^Object base-kd-tree ^int chunk-idx ^Long snapshot-idx])) - -(deftype TemporalCoordinates [^long rowId, ^long iid, - ^long sysTimeStart, ^long sysTimeEnd - ^long appTimeStart, ^long appTimeEnd - ^boolean newEntity, ^boolean tombstone]) - -(def temporal-col-types - {"_iid" :i64, "_row_id" :i64 - "xt$system_from" types/temporal-col-type, "xt$system_to" types/temporal-col-type - "xt$valid_from" types/temporal-col-type, "xt$valid_to" types/temporal-col-type}) - -(defn temporal-column? [col-name] - (contains? temporal-col-types (str col-name))) - -(def ^:const ^int system-time-end-idx 0) -(def ^:const ^int id-idx 1) -(def ^:const ^int system-time-start-idx 2) -(def ^:const ^int row-id-idx 3) -(def ^:const ^int app-time-start-idx 4) -(def ^:const ^int app-time-end-idx 5) - -(def ^:private column->idx {"_iid" id-idx - "_row_id" row-id-idx - "xt$valid_from" app-time-start-idx - "xt$valid_to" app-time-end-idx - "xt$system_from" system-time-start-idx - "xt$system_to" system-time-end-idx}) - -(defn ->temporal-column-idx ^long [col-name] - (long (get column->idx (name col-name)))) - -(defn evict-id [kd-tree, ^BufferAllocator allocator, ^long iid, ^Roaring64Bitmap evicted-row-ids] - (let [min-range (doto (->min-range) - (aset id-idx iid)) - - max-range (doto (->max-range) - (aset id-idx iid)) - - ^IKdTreePointAccess point-access (kd/kd-tree-point-access kd-tree) - - overlap (-> ^LongStream (kd/kd-tree-range-search - kd-tree - min-range - max-range) - (.mapToObj (reify LongFunction - (apply [_ x] - (.getArrayPoint point-access x)))) - (.toArray))] - - (reduce (fn [kd-tree ^longs point] - (.addLong evicted-row-ids (aget point row-id-idx)) - (kd/kd-tree-delete kd-tree allocator (->copy-range point))) - kd-tree - overlap))) - -(defn update-current-row-ids [^clojure.lang.PersistentHashSet current-row-ids removals ^TemporalCoordinates coordinates] - (let [x (apply disj current-row-ids removals)] - (if (.tombstone coordinates) - x - (conj x (.rowId coordinates))))) - -(defn remove-evicted-row-ids [^clojure.lang.PersistentHashSet current-row-ids ^Roaring64Bitmap evicted-row-ids] - (set/difference current-row-ids (set (.toArray evicted-row-ids)))) - -(defn insert-coordinates [kd-tree, ^BufferAllocator allocator, ^TemporalCoordinates coordinates !current-row-ids system-time-μs] - (let [^long system-time-μs system-time-μs - new-entity? (.newEntity coordinates) - row-id (.rowId coordinates) - iid (.iid coordinates) - system-time-start-μs (.sysTimeStart coordinates) - system-time-end-μs (.sysTimeEnd coordinates) - app-time-start-μs (.appTimeStart coordinates) - app-time-end-μs (.appTimeEnd coordinates) - - min-range (doto (->min-range) - (aset id-idx iid) - (aset app-time-end-idx (inc app-time-start-μs)) - (aset system-time-end-idx system-time-start-μs)) - - max-range (doto (->max-range) - (aset id-idx iid) - (aset app-time-start-idx (dec app-time-end-μs)) - (aset system-time-end-idx system-time-end-μs)) - - ^IKdTreePointAccess point-access (kd/kd-tree-point-access kd-tree) - - overlap (when-not new-entity? - (-> ^LongStream (kd/kd-tree-range-search - kd-tree - min-range - max-range) - (.mapToObj (reify LongFunction - (apply [_ x] - (.getArrayPoint point-access x)))) - (.toArray))) - kd-tree (reduce - (fn [kd-tree ^longs point] - (kd/kd-tree-delete kd-tree allocator (->copy-range point))) - kd-tree - overlap) - kd-tree (cond-> kd-tree - (not (.tombstone coordinates)) - (kd/kd-tree-insert allocator - (doto (long-array k) - (aset id-idx iid) - (aset row-id-idx row-id) - (aset app-time-start-idx app-time-start-μs) - (aset app-time-end-idx app-time-end-μs) - (aset system-time-start-idx system-time-start-μs) - (aset system-time-end-idx util/end-of-time-μs))))] - - (when (and - (<= app-time-start-μs system-time-μs) - (> app-time-end-μs system-time-μs)) - (vswap! - !current-row-ids - update-current-row-ids - (map (fn [^longs coord] (aget coord row-id-idx)) overlap) - coordinates)) - - (reduce - (fn [kd-tree ^longs coord] - (cond-> (kd/kd-tree-insert kd-tree allocator (doto (->copy-range coord) - (aset system-time-end-idx system-time-start-μs))) - (< (aget coord app-time-start-idx) app-time-start-μs) - (kd/kd-tree-insert allocator (doto (->copy-range coord) - (aset system-time-start-idx system-time-start-μs) - (aset app-time-end-idx app-time-start-μs))) - - (> (aget coord app-time-end-idx) app-time-end-μs) - (kd/kd-tree-insert allocator (doto (->copy-range coord) - (aset system-time-start-idx system-time-start-μs) - (aset app-time-start-idx app-time-end-μs))))) - kd-tree - overlap))) - -(defn- ->temporal-obj-key [chunk-idx] - (format "chunk-%s/temporal.arrow" (util/->lex-hex-string chunk-idx))) - -(defn- ->temporal-snapshot-obj-key [chunk-idx] - (format "temporal-snapshots/%s.arrow" (util/->lex-hex-string chunk-idx))) - -(defn- temporal-snapshot-obj-key->chunk-idx ^long [obj-key] - (try - (util/<-lex-hex-string (second (re-find #"temporal-snapshots/(\p{XDigit}+)\.arrow" obj-key))) - (catch Throwable t - (log/errorf t "Failed to parse %s" obj-key) - (throw t)))) - -(defn- ->temporal-rel ^xtdb.vector.RelationReader [^BufferAllocator allocator, kd-tree columns temporal-min-range temporal-max-range ^Roaring64Bitmap row-id-bitmap] - (let [^IKdTreePointAccess point-access (kd/kd-tree-point-access kd-tree) - ^LongStream kd-tree-idxs (if (.isEmpty row-id-bitmap) - (LongStream/empty) - (kd/kd-tree-range-search kd-tree temporal-min-range temporal-max-range)) - coordinates (-> kd-tree-idxs - (.mapToObj (reify LongFunction - (apply [_ x] - (.getArrayPoint point-access x)))) - (.filter (reify Predicate - (test [_ x] - (.contains row-id-bitmap (aget ^longs x row-id-idx))))) - - ;; HACK we seem to be creating zero-length app-time ranges, I don't know why, #403 - ;; we filter them out here, but likely best that we don't create them in the first place - (.filter (reify Predicate - (test [_ x] - (not= (aget ^longs x app-time-start-idx) - (aget ^longs x app-time-end-idx))))) - - (.sorted (Comparator/comparingLong (reify ToLongFunction - (applyAsLong [_ x] - (aget ^longs x row-id-idx))))) - (.toArray)) - value-count (alength coordinates)] - (util/with-close-on-catch [cols (ArrayList. (count columns))] - (doseq [col-name columns] - (let [col-idx (->temporal-column-idx col-name) - field (types/col-type->field col-name (get temporal-col-types col-name)) - ^BaseFixedWidthVector temporal-vec (.createVector field allocator) - temporal-vec-wtr (vw/->writer temporal-vec)] - (.allocateNew temporal-vec value-count) - (dotimes [n value-count] - (let [^longs coordinate (aget coordinates n)] - (.writeLong temporal-vec-wtr (aget coordinate col-idx)))) - (.syncValueCount temporal-vec-wtr) - (.add cols (vw/vec-wtr->rdr temporal-vec-wtr)))) - - (vr/rel-reader cols value-count)))) - -(defn row-ids-to-add [kd-tree ^long latest-completed-tx-time ^long current-time] - (let [min-range (doto (->min-range) - (aset app-time-start-idx (inc latest-completed-tx-time)) - (aset app-time-end-idx (inc current-time)) - (aset system-time-end-idx (inc current-time))) - - max-range (doto (->max-range) - (aset app-time-start-idx current-time)) - - ^IKdTreePointAccess point-access (kd/kd-tree-point-access kd-tree) - - overlap (-> ^LongStream (kd/kd-tree-range-search - kd-tree - min-range - max-range) - (.mapToObj (reify LongFunction - (apply [_ x] - (doto (long-array 3) - (aset 0 (.getCoordinate point-access x app-time-start-idx)) - (aset 1 (.getCoordinate point-access x row-id-idx)) - (aset 2 1))))) - (.toArray))] - overlap)) - -(defn row-ids-to-remove [kd-tree ^long latest-completed-tx-time ^long current-time] - (let [min-range (doto (->min-range) - (aset app-time-end-idx (inc latest-completed-tx-time)) - (aset system-time-end-idx (inc current-time))) - - ;; justification here for system-time-end constraint is that if a rows system time end - ;; is before current-time then then that row would have been removed during transaction - ;; processing as there is no way for a system time end to be in the future. - ;; Same applies above for row-ids-to-add - - max-range (doto (->max-range) - (aset app-time-start-idx latest-completed-tx-time) - (aset app-time-end-idx current-time)) - - ^IKdTreePointAccess point-access (kd/kd-tree-point-access kd-tree) - - overlap (-> ^LongStream (kd/kd-tree-range-search - kd-tree - min-range - max-range) - (.mapToObj (reify LongFunction - (apply [_ x] - (doto (long-array 3) - (aset 0 (.getCoordinate point-access x app-time-end-idx)) - (aset 1 (.getCoordinate point-access x row-id-idx)) - (aset 2 0))))) - (.toArray))] - overlap)) - -(defn row-ids-to-from-start [kd-tree ^long current-time] - (let [min-range (doto (->min-range) - (aset app-time-end-idx (inc current-time)) - (aset system-time-end-idx (inc current-time))) - - max-range (doto (->max-range) - (aset app-time-start-idx current-time)) - - ^IKdTreePointAccess point-access (kd/kd-tree-point-access kd-tree) - - overlap (-> ^LongStream (kd/kd-tree-range-search - kd-tree - min-range - max-range) - (.mapToObj (reify LongFunction - (apply [_ x] - (doto (long-array 2) - (aset 0 (.getCoordinate point-access x app-time-start-idx)) - (aset 1 (.getCoordinate point-access x row-id-idx)))))) - (.toArray))] - overlap)) - - -(defn advance-current-row-ids [current-row-ids kd-tree latest-completed-tx-time current-time] - (let [row-ids-to-add (row-ids-to-add kd-tree latest-completed-tx-time current-time) - row-ids-to-remove (row-ids-to-remove kd-tree latest-completed-tx-time current-time) - row-id-changes-by-app-time (sort-by #(aget ^longs % 0) (concat row-ids-to-add row-ids-to-remove))] - (reduce - (fn [current-row-ids-acc ^longs change] - (if (= 1 (aget change 2)) - (conj current-row-ids-acc (aget change 1)) - (disj current-row-ids-acc (aget change 1)))) - current-row-ids - row-id-changes-by-app-time))) - -(defn current-row-ids-from-start [kd-tree current-time] - (let [row-ids-to-add (row-ids-to-from-start kd-tree current-time) - row-id-changes-by-app-time (sort-by #(aget ^longs % 0) row-ids-to-add)] - (reduce - (fn [current-row-ids-acc ^longs change] - (conj current-row-ids-acc (aget change 1))) - #{} - row-id-changes-by-app-time))) - -(deftype TemporalManager [^BufferAllocator allocator - ^ObjectStore object-store - ^IBufferPool buffer-pool - ^IMetadataManager metadata-manager - ^ExecutorService snapshot-pool - ^:volatile-mutable current-row-ids - ^:volatile-mutable ^xtdb.api.protocols.TransactionInstant latest-completed-tx - ^:unsynchronized-mutable snapshot-future - ^:unsynchronized-mutable kd-tree-snapshot-idx - ^:volatile-mutable kd-tree - ^boolean async-snapshot?] - TemporalManagerPrivate - (latestTemporalSnapshotIndex [_ chunk-idx] - (->> (.listObjects object-store "temporal-snapshots/") - (map temporal-snapshot-obj-key->chunk-idx) - (filter #(<= ^long % chunk-idx)) - (last))) - - (buildStaticTree [_ base-kd-tree chunk-idx snapshot-idx] - (let [kd-tree (atom base-kd-tree)] - (try - (let [snapshot-idx (long (or snapshot-idx -1)) - new-chunk-idxs (for [^long idx (distinct (concat (keys (.chunksMetadata metadata-manager)) [chunk-idx])) - :when (> idx snapshot-idx) - :while (<= idx chunk-idx)] - idx) - futs (for [chunk-idx new-chunk-idxs] - (-> (.getBuffer buffer-pool (->temporal-obj-key chunk-idx)) - (util/then-apply util/try-close)))] - @(CompletableFuture/allOf (into-array CompletableFuture futs)) - (doseq [chunk-idx new-chunk-idxs - :let [obj-key (->temporal-obj-key chunk-idx) - chunk-kd-tree (grid/->arrow-buf-grid (-> @(.getBuffer buffer-pool obj-key) - (util/rethrowing-cause)))]] - (swap! kd-tree #(if % - (kd/->merged-kd-tree % chunk-kd-tree) - chunk-kd-tree))) - @kd-tree) - (catch Exception e - (util/try-close @kd-tree) - (throw e))))) - - (reloadTemporalIndex [this chunk-idx snapshot-idx] - (if snapshot-idx - (let [^ArrowBuf temporal-buffer (-> @(.getBuffer buffer-pool (->temporal-snapshot-obj-key snapshot-idx)) - (util/rethrowing-cause))] - (set! (.kd-tree this) (kd/->merged-kd-tree - (.buildStaticTree this - (grid/->arrow-buf-grid temporal-buffer) - chunk-idx - snapshot-idx) - nil)) - (when (and kd-tree-snapshot-idx (not= kd-tree-snapshot-idx snapshot-idx)) - (.evictBuffer buffer-pool (->temporal-snapshot-obj-key kd-tree-snapshot-idx))) - (set! (.kd-tree-snapshot-idx this) snapshot-idx)) - (set! (.kd-tree this) (some-> (.buildStaticTree this nil chunk-idx snapshot-idx) - (kd/->merged-kd-tree nil))))) - - (populateKnownChunks [this] - (when-let [temporal-chunk-idx (last (keys (.chunksMetadata metadata-manager)))] - (.reloadTemporalIndex this temporal-chunk-idx (.latestTemporalSnapshotIndex this temporal-chunk-idx)) - (set! (.current-row-ids this) - (current-row-ids-from-start - (.kd-tree this) - (util/instant->micros (.system-time latest-completed-tx)))))) - - (awaitSnapshotBuild [_] - (-> (some-> snapshot-future (deref)) - (util/rethrowing-cause))) - - (buildTemporalSnapshot [this chunk-idx snapshot-idx] - (let [new-snapshot-obj-key (->temporal-snapshot-obj-key chunk-idx) - path (util/->temp-file "temporal-snapshot" "")] - (try - (if snapshot-idx - (let [^ArrowBuf temporal-buffer (-> @(.getBuffer buffer-pool (->temporal-snapshot-obj-key snapshot-idx)) - (util/rethrowing-cause))] - (with-open [kd-tree (.buildStaticTree this - (grid/->arrow-buf-grid temporal-buffer) - chunk-idx - snapshot-idx)] - (let [temporal-buf (-> (grid/->disk-grid allocator path kd-tree {:k k}) - (util/->mmap-path))] - (-> @(.putObject object-store new-snapshot-obj-key temporal-buf) - (util/rethrowing-cause))))) - (when-let [kd-tree (.buildStaticTree this nil chunk-idx snapshot-idx)] - (with-open [^AutoCloseable kd-tree kd-tree] - (let [temporal-buf (-> (grid/->disk-grid allocator path kd-tree {:k k}) - (util/->mmap-path))] - (-> @(.putObject object-store new-snapshot-obj-key temporal-buf) - (util/rethrowing-cause)))))) - (finally - (util/delete-file path))))) - - ITemporalManager - (getTemporalWatermark [_] - (let [kd-tree (some-> kd-tree (kd/kd-tree-retain allocator)) - latest-completed-tx latest-completed-tx] - (reify - ITemporalRelationSource - (createTemporalRelation [_ allocator columns temporal-min-range temporal-max-range row-id-bitmap] - (->temporal-rel allocator kd-tree columns temporal-min-range temporal-max-range row-id-bitmap)) - - (getCurrentRowIds [_ current-time] - (advance-current-row-ids - current-row-ids - kd-tree - (util/instant->micros (.system-time latest-completed-tx)) - current-time)) - - AutoCloseable - (close [_] - (util/try-close kd-tree))))) - - (registerNewChunk [this chunk-idx] - (when kd-tree - (let [new-temporal-obj-key (->temporal-obj-key chunk-idx) - path (util/->temp-file "temporal-idx" "")] - (try - (let [temporal-buf (-> (grid/->disk-grid allocator - path - (if (instance? MergedKdTree kd-tree) - (.getDynamicKdTree ^MergedKdTree kd-tree) - kd-tree) - {:k k - :cell-size 256 - :deletes? true}) - (util/->mmap-path))] - (-> @(.putObject object-store new-temporal-obj-key temporal-buf) - (util/rethrowing-cause))) - (finally - (util/delete-file path))))) - (.awaitSnapshotBuild this) - (when kd-tree - (with-open [^AutoCloseable _old-kd-tree kd-tree] - (let [snapshot-idx (.latestTemporalSnapshotIndex this chunk-idx) - fut (.submit snapshot-pool ^Runnable #(.buildTemporalSnapshot this chunk-idx snapshot-idx))] - (set! (.snapshot-future this) fut) - (when-not async-snapshot? - @fut) - (.reloadTemporalIndex this chunk-idx snapshot-idx))))) - - (startTx [this-tm tx-key] - (let [system-time-μs (util/instant->micros (.system-time tx-key)) - evicted-row-ids (Roaring64Bitmap.) - !kd-tree (volatile! (kd/kd-tree-retain kd-tree allocator)) - !current-row-ids (volatile! current-row-ids)] - - (when latest-completed-tx - (vswap! - !current-row-ids - advance-current-row-ids - @!kd-tree - (util/instant->micros (.system-time latest-completed-tx)) - system-time-μs)) - - (reify - ITemporalTxIndexer - (indexPut [_ iid row-id start-app-time end-app-time new-entity?] - (vswap! !kd-tree - insert-coordinates allocator (TemporalCoordinates. row-id iid - system-time-μs util/end-of-time-μs - start-app-time end-app-time - new-entity? false) - !current-row-ids - system-time-μs)) - - (indexDelete [_ iid row-id start-app-time end-app-time new-entity?] - (vswap! !kd-tree - insert-coordinates allocator (TemporalCoordinates. row-id iid - system-time-μs util/end-of-time-μs - start-app-time end-app-time - new-entity? true) - !current-row-ids - system-time-μs)) - - (indexEvict [_ iid] - (vswap! !kd-tree evict-id allocator iid evicted-row-ids) - (vswap! !current-row-ids remove-evicted-row-ids evicted-row-ids)) - - (commit [_] - (let [old-kd-tree kd-tree] - (set! (.kd-tree this-tm) @!kd-tree) - (util/try-close old-kd-tree)) - - (set! (.current-row-ids this-tm) @!current-row-ids) - (set! (.latest-completed-tx this-tm) tx-key) - evicted-row-ids) - - (abort [_] (util/try-close @!kd-tree)) - - ITemporalRelationSource - (createTemporalRelation [_ allocator columns temporal-min-range temporal-max-range row-id-bitmap] - (->temporal-rel allocator @!kd-tree columns temporal-min-range temporal-max-range row-id-bitmap)) - - (getCurrentRowIds [_ current-time] - (advance-current-row-ids - @!current-row-ids - @!kd-tree - (util/instant->micros (.system-time latest-completed-tx)) - current-time))))) - - AutoCloseable - (close [this] - (util/shutdown-pool snapshot-pool) - (set! (.snapshot-future this) nil) - (util/try-close kd-tree) - (set! (.kd-tree this) nil) - (set! (.latest-completed-tx this) nil))) - -(defmethod ig/prep-key ::temporal-manager [_ opts] - (merge {:allocator (ig/ref :xtdb/allocator) - :object-store (ig/ref :xtdb/object-store) - :buffer-pool (ig/ref :xtdb.buffer-pool/buffer-pool) - :metadata-mgr (ig/ref :xtdb.metadata/metadata-manager) - :async-snapshot? true} - opts)) - -(defmethod ig/init-key ::temporal-manager - [_ {:keys [^BufferAllocator allocator - ^ObjectStore object-store - ^IBufferPool buffer-pool - ^IMetadataManager metadata-mgr - async-snapshot?]}] - - (let [pool (Executors/newSingleThreadExecutor (util/->prefix-thread-factory "temporal-snapshot-"))] - (doto (TemporalManager. allocator object-store buffer-pool metadata-mgr - pool #{} (:latest-completed-tx (meta/latest-chunk-metadata metadata-mgr)) nil nil nil async-snapshot?) - (.populateKnownChunks)))) - -(defmethod ig/halt-key! ::temporal-manager [_ ^TemporalManager mgr] - (.close mgr)) diff --git a/core/src/main/clojure/xtdb/temporal/grid.clj b/core/src/main/clojure/xtdb/temporal/grid.clj deleted file mode 100644 index 86fbf8b9bd..0000000000 --- a/core/src/main/clojure/xtdb/temporal/grid.clj +++ /dev/null @@ -1,522 +0,0 @@ -(ns xtdb.temporal.grid - (:require [clojure.data.json :as json] - [xtdb.temporal.histogram :as hist] - [xtdb.temporal.kd-tree :as kd] - [xtdb.util :as util]) - (:import (java.io BufferedInputStream BufferedOutputStream Closeable DataInputStream DataOutputStream) - (java.nio.channels Channels) - java.nio.file.Path - (java.util Arrays) - java.util.concurrent.atomic.AtomicInteger - (java.util.function Consumer IntToLongFunction LongConsumer LongFunction LongPredicate LongUnaryOperator UnaryOperator) - (java.util.stream LongStream Stream) - (org.apache.arrow.memory ArrowBuf BufferAllocator) - (org.apache.arrow.vector VectorLoader VectorSchemaRoot) - org.apache.arrow.vector.complex.FixedSizeListVector - org.apache.arrow.vector.ipc.ArrowFileWriter - org.apache.arrow.vector.types.pojo.Schema - xtdb.BitUtil - (xtdb.temporal.histogram IHistogram IMultiDimensionalHistogram) - (xtdb.temporal.kd_tree IKdTreePointAccess KdTreeVectorPointAccess))) - -;; "Learning Multi-dimensional Indexes" -;; https://arxiv.org/pdf/1912.01668.pdf - -(set! *unchecked-math* :warn-on-boxed) - -(defn- cartesian-product-idxs ^java.util.stream.Stream [counts] - (let [total-count (reduce * counts) - end-idxs (int-array (map dec counts)) - len (alength end-idxs)] - (.limit (Stream/iterate (int-array len) - (reify UnaryOperator - (apply [_ as] - (let [^ints as as] - (loop [n 0] - (let [x (aget as n)] - (if (= x (aget end-idxs n)) - (when (not= n len) - (aset as n 0) - (recur (inc n))) - (aset as n (inc x))))) - as)))) - total-count))) - -(defn- three-way-partition ^longs [^IKdTreePointAccess access ^long low ^long hi ^long axis] - (let [pivot (.getCoordinate access (quot (+ low hi) 2) axis)] - (loop [i (int low) - j (int low) - k (inc (int hi))] - (if (< j k) - (let [diff (Long/compare (.getCoordinate access j axis) pivot)] - (cond - (neg? diff) - (do (.swapPoint access i j) - (recur (inc i) (inc j) k)) - - (pos? diff) - (let [k (dec k)] - (.swapPoint access j k) - (recur i j k)) - - :else - (recur i (inc j) k))) - (doto (long-array 2) - (aset 0 i) - (aset 1 (dec k))))))) - -(defn- quick-sort [^IKdTreePointAccess access ^long low ^long hi ^long axis] - (when (< low hi) - (let [^longs left-right (three-way-partition access low hi axis) - left (dec (aget left-right 0)) - right (inc (aget left-right 1))] - (if (< (- left low) (- hi right)) - (do (quick-sort access low left axis) - (recur access right hi axis)) - (do (quick-sort access right hi axis) - (recur access low left axis)))))) - -(defn- binary-search-leftmost ^long [^IntToLongFunction access-fn ^long n ^long idx ^long x] - (loop [l 0 - r n - m (max (min idx (dec n)) 0)] - (if (< l r) - (if (< (.applyAsLong access-fn m) x) - (let [l (inc m)] - (recur l r (BitUtil/unsignedBitShiftRight (+ l r) 1))) - (let [r m] - (recur l r (BitUtil/unsignedBitShiftRight (+ l r) 1)))) - l))) - -(defn- binary-search-rightmost ^long [^IntToLongFunction access-fn ^long n ^long idx ^long x] - (loop [l 0 - r n - m (max (min idx (dec n)) 0)] - (if (< l r) - (if (> (.applyAsLong access-fn m) x) - (let [r m] - (recur l r (BitUtil/unsignedBitShiftRight (+ l r) 1))) - (let [l (inc m)] - (recur l r (BitUtil/unsignedBitShiftRight (+ l r) 1)))) - (dec r)))) - -;; NOTE: slopes and linear scan cannot beat binary search -;; currently. Remove? - -(def ^:private ^:const linear-scan-limit 16) - -(defn- linear-search-leftmost ^long [^IntToLongFunction access-fn ^long n ^long idx ^long x] - (let [m (max (min idx (dec n)) 0)] - (if (< (.applyAsLong access-fn m) x) - (loop [m m - c 0] - (cond - (= linear-scan-limit c) - (binary-search-leftmost access-fn n m x) - - (and (< m n) - (< (.applyAsLong access-fn m) x)) - (recur (inc m) (inc c)) - - :else - m)) - (loop [m m - c 0] - (cond - (= linear-scan-limit c) - (binary-search-leftmost access-fn n m x) - - (and (>= m 0) - (>= (.applyAsLong access-fn m) x)) - (recur (dec m) (inc c)) - - :else - (inc m)))))) - -(defn- linear-search-rightmost ^long [^IntToLongFunction access-fn ^long n ^long idx ^long x] - (let [m (max (min idx (dec n)) 0)] - (if (> (.applyAsLong access-fn m) x) - (loop [m m - c 0] - (cond - (= linear-scan-limit c) - (binary-search-rightmost access-fn n m x) - - (and (>= m 0) - (> (.applyAsLong access-fn m) x)) - (recur (dec m) (inc c)) - - :else - m)) - (loop [m m - c 0] - (cond - (= linear-scan-limit c) - (binary-search-rightmost access-fn n m x) - - (and (< m n) - (<= (.applyAsLong access-fn m) x)) - (recur (inc m) (inc c)) - - :else - (dec m)))))) - -(defn- ->cell-idx ^long [^objects scales ^longs point ^long k-minus-one ^long axis-shift] - (loop [n 0 - idx 0] - (if (= n k-minus-one) - idx - (let [axis-idx (Arrays/binarySearch ^longs (aget scales n) (aget point n)) - ^long axis-idx (if (neg? axis-idx) - (dec (- axis-idx)) - axis-idx)] - (recur (inc n) (bit-or (bit-shift-left idx axis-shift) axis-idx)))))) - -(declare ->grid-point-access) - -(deftype Grid [^ArrowBuf arrow-buf - ^objects scales - ^longs mins - ^longs maxs - ^objects cells - ^doubles k-minus-one-slope+base - ^int k - ^int axis-shift - ^int cell-shift - ^long size - ^long value-count - ^AtomicInteger ref-cnt - ^boolean deletes?] - kd/KdTree - (kd-tree-insert [this allocator point] - (throw (UnsupportedOperationException.))) - (kd-tree-delete [this allocator point] - (throw (UnsupportedOperationException.))) - (kd-tree-range-search [this min-range max-range] - (let [min-range (kd/->longs min-range) - max-range (kd/->longs max-range) - k-minus-one (dec k) - axis-mask (kd/range-bitmask min-range max-range) - axis-idxs+masks (object-array - (for [^long n (range k-minus-one) - :let [min-r (aget min-range n) - max-r (aget max-range n) - min-v (aget mins n) - max-v (aget maxs n)] - :when (BitUtil/bitNot (or (< max-v min-r) (> min-v max-r))) - :let [partial-match-axis? (BitUtil/bitNot (BitUtil/isBitSet axis-mask n)) - ^longs axis-scale (aget scales n) - min-axis-idx (if partial-match-axis? - (int 0) - (Arrays/binarySearch axis-scale min-r)) - min-axis-idx (if (neg? min-axis-idx) - (dec (- min-axis-idx)) - min-axis-idx) - max-axis-idx (if partial-match-axis? - (alength axis-scale) - (Arrays/binarySearch axis-scale max-r)) - max-axis-idx (if (neg? max-axis-idx) - (dec (- max-axis-idx)) - max-axis-idx)] - :let [mask (bit-shift-left (bit-and axis-mask (bit-shift-left 1 n)) Integer/SIZE) - axis-idxs+masks (-> (LongStream/range min-axis-idx (unchecked-inc-int max-axis-idx)) - (.toArray)) - last-idx (dec (alength axis-idxs+masks))]] - (do (aset axis-idxs+masks 0 (bit-or (aget axis-idxs+masks 0) mask)) - (aset axis-idxs+masks last-idx (bit-or (aget axis-idxs+masks last-idx) mask)) - axis-idxs+masks))) - partial-match-last-axis? (BitUtil/bitNot (BitUtil/isBitSet axis-mask k-minus-one)) - min-r (aget min-range k-minus-one) - max-r (aget max-range k-minus-one) - acc (LongStream/builder)] - (when (= k-minus-one (count axis-idxs+masks)) - (.forEach (cartesian-product-idxs (map count axis-idxs+masks)) - (reify Consumer - (accept [_ idxs] - (let [^ints idxs idxs] - (loop [m (int 0) - cell-idx 0 - cell-axis-mask 0] - (if (< m k-minus-one) - (let [axis-idx+mask (aget ^longs (aget axis-idxs+masks m) (aget idxs m))] - (recur (unchecked-inc-int m) - (bit-or (bit-shift-left cell-idx axis-shift) (BitUtil/bitMask axis-idx+mask -1)) - (bit-or cell-axis-mask (BitUtil/unsignedBitShiftRight axis-idx+mask Integer/SIZE)))) - (when-let [^FixedSizeListVector cell (aget cells cell-idx)] - (let [access (KdTreeVectorPointAccess. cell k) - access-fn (reify IntToLongFunction - (applyAsLong [_ idx] - (.getCoordinate access idx k-minus-one))) - n (.getValueCount cell) - slope-idx (bit-shift-left cell-idx 1) - slope (aget k-minus-one-slope+base slope-idx) - base (aget k-minus-one-slope+base (inc slope-idx)) - start-point-idx (bit-shift-left cell-idx cell-shift) - start-idx (if partial-match-last-axis? - 0 - (binary-search-leftmost access-fn n (+ (* slope min-r) base) min-r)) - end-idx (if partial-match-last-axis? - (dec n) - (binary-search-rightmost access-fn n (+ (* slope max-r) base) max-r))] - (if deletes? - (if (zero? cell-axis-mask) - (loop [idx start-idx] - (when (<= idx end-idx) - (when (BitUtil/bitNot (.isDeleted access idx)) - (.add acc (+ start-point-idx idx))) - (recur (inc idx)))) - (loop [idx start-idx] - (when (<= idx end-idx) - (when (and (BitUtil/bitNot (.isDeleted access idx)) - (.isInRange access idx min-range max-range cell-axis-mask)) - (.add acc (+ start-point-idx idx))) - (recur (inc idx))))) - (if (zero? cell-axis-mask) - (loop [idx start-idx] - (when (<= idx end-idx) - (.add acc (+ start-point-idx idx)) - (recur (inc idx)))) - (loop [idx start-idx] - (when (<= idx end-idx) - (when (.isInRange access idx min-range max-range cell-axis-mask) - (.add acc (+ start-point-idx idx))) - (recur (inc idx))))))))))))))) - - (.build acc))) - (kd-tree-points [this deletes?] - (.flatMap (LongStream/range 0 (alength cells)) - (reify LongFunction - (apply [_ cell-idx] - (if-let [^FixedSizeListVector cell (aget cells cell-idx)] - (let [start-point-idx (bit-shift-left cell-idx cell-shift)] - (cond-> (LongStream/range 0 (.getValueCount cell)) - (BitUtil/bitNot deletes?) (.filter (reify LongPredicate - (test [_ x] - (BitUtil/bitNot (.isNull cell x))))) - true (.map (reify LongUnaryOperator - (applyAsLong [_ x] - (+ start-point-idx x)))))) - (LongStream/empty)))))) - (kd-tree-height [_] 1) - (kd-tree-retain [this _] - (when (zero? (.getAndIncrement ref-cnt)) - (.set ref-cnt 0) - (throw (IllegalStateException. "grid closed"))) - this) - (with-point-vec [this _pv] this) - (kd-tree-point-access [this] - (->grid-point-access this)) - (kd-tree-size [_] size) - (kd-tree-value-count [_] value-count) - (kd-tree-dimensions [_] k) - - Closeable - (close [_] - (when (zero? (.decrementAndGet ref-cnt)) - (doseq [cell cells] - (util/try-close cell)) - (util/try-close arrow-buf)))) - -(deftype GridPointAccess [^objects cells ^int cell-shift ^int cell-mask ^int k] - IKdTreePointAccess - (getPoint [this idx] - (.getPoint (KdTreeVectorPointAccess. (aget cells (BitUtil/unsignedBitShiftRight idx cell-shift)) k) - (bit-and idx cell-mask))) - (getArrayPoint [this idx] - (.getArrayPoint (KdTreeVectorPointAccess. (aget cells (BitUtil/unsignedBitShiftRight idx cell-shift)) k) - (bit-and idx cell-mask))) - (getCoordinate [this idx axis] - (.getCoordinate (KdTreeVectorPointAccess. (aget cells (BitUtil/unsignedBitShiftRight idx cell-shift)) k) - (bit-and idx cell-mask) - axis)) - (setCoordinate [_ idx axis value] - (throw (UnsupportedOperationException.))) - (swapPoint [_ from-idx to-idx] - (throw (UnsupportedOperationException.))) - (isDeleted [_ idx] - (.isDeleted (KdTreeVectorPointAccess. (aget cells (BitUtil/unsignedBitShiftRight idx cell-shift)) k) - (bit-and idx cell-mask))) - (isInRange [this idx min-range max-range mask] - (.isInRange (KdTreeVectorPointAccess. (aget cells (BitUtil/unsignedBitShiftRight idx cell-shift)) k) - (bit-and idx cell-mask) min-range max-range mask))) - -(defn- ->grid-point-access ^xtdb.temporal.kd_tree.IKdTreePointAccess [^Grid grid] - (let [cell-shift (.cell-shift grid) - cell-mask (dec (bit-shift-left 1 cell-shift))] - (GridPointAccess. (.cells grid) cell-shift cell-mask (.k grid)))) - -(defn- ->grid-meta-json->grid - ^xtdb.temporal.grid.Grid [arrow-buf - cells - {:keys [scales - mins - maxs - k-minus-one-slope+base - k - axis-shift - cell-shift - size - value-count - deletes?]}] - (Grid. arrow-buf - (object-array (map long-array scales)) - (long-array mins) - (long-array maxs) - cells - (double-array k-minus-one-slope+base) - k - axis-shift - cell-shift - size - value-count - (AtomicInteger. 1) - deletes?)) - -(def ^:private ^:const point-vec-idx 0) - -(defn ->arrow-buf-grid ^xtdb.temporal.grid.Grid [^ArrowBuf arrow-buf] - (let [footer (util/read-arrow-footer arrow-buf) - schema (.getSchema footer) - grid-meta (json/read-str (get (.getCustomMetadata schema) "grid-meta") :key-fn keyword) - allocator (.getAllocator (.getReferenceManager arrow-buf)) - cells (object-array - (for [block (.getRecordBatches footer)] - (with-open [arrow-record-batch (util/->arrow-record-batch-view block arrow-buf) - root (VectorSchemaRoot/create schema allocator)] - (.load (VectorLoader. root) arrow-record-batch) - (when (pos? (.getRowCount root)) - (.getTo (doto (.getTransferPair (.getVector root point-vec-idx) allocator) - (.transfer)))))))] - (->grid-meta-json->grid arrow-buf cells grid-meta))) - -(defn ->mmap-grid ^xtdb.temporal.grid.Grid [^BufferAllocator allocator ^Path path] - (let [nio-buffer (util/->mmap-path path) - arrow-buf (util/->arrow-buf-view allocator nio-buffer)] - (->arrow-buf-grid arrow-buf))) - -(defn ->disk-grid - (^xtdb.temporal.grid.Grid [^BufferAllocator allocator ^Path path points {:keys [^long max-histogram-bins - ^long cell-size - ^long k - deletes?] - :or {max-histogram-bins 128 - cell-size (* 8 1024) - deletes? false}}] - (assert (number? k)) - (util/mkdirs (.getParent path)) - (let [^long total (kd/kd-tree-size points) - _ (assert (= 1 (Long/bitCount cell-size))) - number-of-cells (Math/ceil (/ total cell-size)) - k-minus-one (dec k) - cells-per-dimension (BitUtil/ceilPowerOfTwo (Math/ceil (Math/pow number-of-cells (/ 1 k-minus-one)))) - number-of-cells (long (Math/ceil (Math/pow cells-per-dimension k-minus-one))) - axis-shift (Long/bitCount (dec cells-per-dimension)) - histogram-bins (min max-histogram-bins (* k-minus-one cells-per-dimension)) - ^IMultiDimensionalHistogram histogram (hist/->multidimensional-histogram histogram-bins k-minus-one) - update-histograms-fn (fn [^longs p] - (let [p (double-array k-minus-one p)] - (.update histogram p))) - cell-outs (object-array number-of-cells) - cell-paths (object-array number-of-cells) - ^IKdTreePointAccess access (kd/kd-tree-point-access points)] - (.forEach ^LongStream (kd/kd-tree-points points deletes?) - (reify LongConsumer - (accept [_ x] - (update-histograms-fn (.getArrayPoint access x))))) - (try - (let [histograms (for [n (range k-minus-one)] - (.projectAxis histogram n)) - scales (object-array (for [^IHistogram h histograms - :let [u (.uniform h cells-per-dimension)]] - (long-array (distinct u)))) - mins (long-array (for [^IHistogram h histograms] - (Math/floor (.getMin h)))) - maxs (long-array (for [^IHistogram h histograms] - (Math/ceil (.getMax h)))) - k-minus-one-mins (long-array number-of-cells Long/MAX_VALUE) - k-minus-one-maxs (long-array number-of-cells Long/MIN_VALUE) - k-minus-one-slope+base (double-array (* 2 number-of-cells)) - write-point-fn (fn [^longs p deleted?] - (let [cell-idx (->cell-idx scales p k-minus-one axis-shift) - ^DataOutputStream out (or (aget cell-outs cell-idx) - (let [f (str "." (.getFileName path) (format "_cell_%016x.raw" cell-idx)) - cell-path (.resolveSibling path f)] - (aset cell-paths cell-idx cell-path) - (let [file-ch (util/->file-channel cell-path util/write-truncate-open-opts)] - (try - (doto (DataOutputStream. (BufferedOutputStream. (Channels/newOutputStream file-ch))) - (->> (aset cell-outs cell-idx))) - (catch Exception e - (util/try-close file-ch) - (throw e))))))] - (dotimes [n k] - (let [x (aget p n)] - (when (= n k-minus-one) - (aset k-minus-one-mins cell-idx (min x (aget k-minus-one-mins cell-idx))) - (aset k-minus-one-maxs cell-idx (max x (aget k-minus-one-maxs cell-idx)))) - (.writeLong out x))) - (.writeByte out (if deleted? - 1 - 0))))] - (.forEach ^LongStream (kd/kd-tree-points points deletes?) - (reify LongConsumer - (accept [_ x] - (write-point-fn (.getArrayPoint access x) (.isDeleted access x))))) - (dotimes [n number-of-cells] - (util/try-close (aget cell-outs n)) - (when-let [^Path cell-path (aget cell-paths n)] - (let [min-r (double (aget k-minus-one-mins n)) - max-r (double (aget k-minus-one-maxs n)) - value-count (quot (util/path-size cell-path) (inc (* k Long/BYTES))) - diff (- max-r min-r) - slope (if (zero? diff) - 0.0 - (double (/ value-count diff))) - base (- (* slope min-r)) - slope-idx (* 2 n)] - (aset k-minus-one-slope+base slope-idx slope) - (aset k-minus-one-slope+base (inc slope-idx) base)))) - (let [max-cell-size (reduce max (for [^Path cell-path cell-paths - :when cell-path] - (quot (util/path-size cell-path) (inc (* k Long/BYTES))))) - cell-shift (Long/bitCount (dec (BitUtil/ceilPowerOfTwo max-cell-size))) - grid-meta {:scales scales - :mins mins - :maxs maxs - :k-minus-one-slope+base k-minus-one-slope+base - :k k - :axis-shift axis-shift - :cell-shift cell-shift - :size total - :value-count (bit-shift-left (inc number-of-cells) cell-shift) - :deletes? deletes?} - schema (Schema. [(kd/->point-field k)] {"grid-meta" (json/write-str grid-meta)}) - buf (long-array k)] - (with-open [root (VectorSchemaRoot/create schema allocator) - ch (util/->file-channel path util/write-truncate-open-opts) - out (ArrowFileWriter. root nil ch)] - (let [^FixedSizeListVector point-vec (.getVector root point-vec-idx) - out-access (KdTreeVectorPointAccess. point-vec k)] - (dotimes [n number-of-cells] - (.clear root) - (when-let [^Path cell-path (aget cell-paths n)] - (with-open [in (DataInputStream. (BufferedInputStream. (Channels/newInputStream (util/->file-channel cell-path))))] - (let [value-count (quot (util/path-size cell-path) (inc (* k Long/BYTES)))] - (dotimes [_ value-count] - (dotimes [m k] - (aset buf m (.readLong in))) - (let [deleted? (= (.readByte in) 1) - idx (kd/write-point point-vec out-access buf)] - (when deleted? - (.setNull point-vec idx)))) - (.setRowCount root value-count))) - (util/delete-file cell-path) - (quick-sort out-access 0 (dec (.getRowCount root)) k-minus-one)) - (.writeBatch out)))))) - path - (finally - (doseq [cell cell-outs] - (util/try-close cell)) - (doseq [cell-path cell-paths - :when cell-path] - (util/delete-file cell-path))))))) diff --git a/core/src/main/clojure/xtdb/temporal/histogram.clj b/core/src/main/clojure/xtdb/temporal/histogram.clj deleted file mode 100644 index 1a975d3bbe..0000000000 --- a/core/src/main/clojure/xtdb/temporal/histogram.clj +++ /dev/null @@ -1,353 +0,0 @@ -(ns xtdb.temporal.histogram - (:import (java.util ArrayList Collections Comparator List) - (java.util.function Consumer ToDoubleFunction))) - -;; "A Streaming Parallel Decision Tree Algorithm" -;; https://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf - -;; NOTES, checkout: -;; http://engineering.nyu.edu/~suel/papers/pxp.pdf -;; https://www.researchgate.net/publication/2459804_Approximating_Multi-Dimensional_Aggregate_Range_Queries_Over_Real_Attributes -;; https://ashraf.aboulnaga.me/pubs/sigmod99sthist.pdf - -;; I think adapting the current histogram to store points in the bins -;; and figure out an efficient way to calculate nearest points during -;; updates might be enough, and then use the normal sum/uniform across -;; a single axis at at time on the result. - -;; We might need a way which can accept many empty buckets and -;; maintains a mapping of sparse cell-idxs to dense block-idxs. - -;; Also, check out the Tsunami paper again. - -(set! *unchecked-math* :warn-on-boxed) - -(definterface IHistogram - (^xtdb.temporal.histogram.IHistogram update [^double x]) - (^double sum [^double x]) - (^doubles uniform [^int number-of-buckets]) - (^double getMin []) - (^double getMax []) - (^long getTotal []) - (^java.util.List getBins []) - (^String histogramString [])) - -(definterface IBin - (^double getValue []) - (^long getCount []) - (^void increment [])) - -(deftype Bin [^double value ^:unsynchronized-mutable ^long count] - IBin - (getValue [_] value) - - (getCount [_] count) - - (increment [this] - (set! (.count this) (inc count))) - - (equals [_ other] - (= value (.value ^Bin other))) - - (hashCode [_] - (Double/hashCode value)) - - Comparable - (compareTo [_ other] - (Double/compare value (.value ^Bin other)))) - -(deftype Histogram [^int max-bins - ^:unsynchronized-mutable ^long total - ^:unsynchronized-mutable ^double min-v - ^:unsynchronized-mutable ^double max-v - ^List bins] - IHistogram - (update [this p] - (set! (.total this) (inc total)) - (set! (.min-v this) (min p min-v)) - (set! (.max-v this) (max p max-v)) - - (let [new-bin (Bin. p 1) - idx (Collections/binarySearch bins new-bin)] - (if (neg? idx) - (.add bins (dec (- idx)) new-bin) - (.increment ^IBin (.get bins idx)))) - - (while (> (.size bins) max-bins) - (let [^long min-idx (loop [n 0 - min-idx 0 - delta Double/POSITIVE_INFINITY] - (if (= n (dec (.size bins))) - min-idx - (let [new-delta (- (.getValue ^IBin (.get bins (inc n))) - (.getValue ^IBin (.get bins n)))] - (if (< new-delta delta) - (recur (inc n) n new-delta) - (recur (inc n) min-idx delta))))) - ^IBin bin-i (.get bins min-idx) - ^IBin bin-i+1 (.get bins (inc min-idx)) - qi (.getValue bin-i) - ki (.getCount bin-i) - qi+1 (.getValue bin-i+1) - ki+1 (.getCount bin-i+1) - new-k (+ ki ki+1) - new-q (/ (+ (* qi ki) (* qi+1 ki+1)) new-k) - new-bin (Bin. new-q new-k)] - (doto bins - (.remove (inc min-idx)) - (.set min-idx new-bin)))) - - this) - - (sum [this b] - (let [last-idx (dec (.size bins))] - (cond - (< b (.getValue ^IBin (.get bins 0))) 0 - (>= b (.getValue ^IBin (.get bins last-idx))) total - :else - (let [probe-bin (Bin. b 0) - idx (Collections/binarySearch bins probe-bin) - ^long idx (if (neg? idx) - (- (- idx) 2) - idx)] - (cond - (neg? idx) - 0.0 - - (> idx last-idx) - total - - :else - (let [^IBin bin-i (.get bins idx) - ^IBin bin-i+1 (if (= idx last-idx) - (Bin. (inc max-v) 0) - (.get bins (inc idx))) - pi (.getValue bin-i) - mi (.getCount bin-i) - pi+1 (.getValue bin-i+1) - mi+1 (.getCount bin-i+1) - - mb (+ mi (* (/ (- mi+1 mi) (- pi+1 pi)) - (- b pi))) - s (* (/ (+ mi mb) 2.0) - (/ (- b pi) (- pi+1 pi)))] - (loop [n 0 - s s] - (if (< n idx) - (recur (inc n) (+ s (.getCount ^IBin (.get bins n)))) - (+ s (/ mi 2.0)))))))))) - - (uniform [this number-of-buckets] - (let [last-idx (dec (.size bins)) - number-of-buckets number-of-buckets] - (double-array - (for [^long x (range 1 number-of-buckets) - :let [s (* (double (/ x number-of-buckets)) total) - ^long idx (loop [[^IBin bin & bins] bins - idx 0] - (if-not bin - idx - (if (< (.sum this (.getValue bin)) s) - (recur bins (inc idx)) - (max 0 (dec idx))))) - ^IBin bin-i (.get bins idx) - ^IBin bin-i+1 (if (= idx last-idx) - (Bin. (inc max-v) 0) - (.get bins (inc idx))) - pi (.getValue bin-i) - mi (.getCount bin-i) - pi+1 (.getValue bin-i+1) - mi+1 (.getCount bin-i+1) - d (- s (.sum this pi)) - a (- mi+1 mi) - b (* 2.0 mi) - c (- (* 2.0 d)) - ;; NOTE: unsure if this NaN handling is correct? - z (if (zero? a) - (- (/ c b)) - (/ (+ (- b) (Math/sqrt (Math/abs (- (* b b) (* 4.0 a c))))) - (* 2.0 a)))]] - (+ pi (* (- pi+1 pi) z)))))) - - (getMin [this] - min-v) - - (getMax [this] - max-v) - - (getTotal [this] - total) - - (getBins [this] - bins) - - (histogramString [this] - (str "total: " total " min: " min-v " max: " max-v "\n" - (apply str (for [^IBin b bins - :let [k (.getValue b) - v (.getCount b)]] - (str (format "%10.4f" k) "\t" (apply str (repeat (* 200 (double (/ v total))) "*")) "\n")))))) - -(defn ->histogram ^xtdb.temporal.histogram.Histogram [^long max-bins] - (Histogram. max-bins 0 Double/POSITIVE_INFINITY Double/NEGATIVE_INFINITY (ArrayList. (inc max-bins)))) - -(definterface IMultiDimensionalHistogram - (^xtdb.temporal.histogram.IMultiDimensionalHistogram update [^doubles x]) - (^doubles getMins []) - (^doubles getMaxs []) - (^long getTotal []) - (^java.util.List getBins []) - (^xtdb.temporal.histogram.IHistogram projectAxis [^int axis])) - -(definterface IMultiDimensionalBin - (^doubles getValue []) - (^long getCount []) - (^xtdb.temporal.histogram.IBin projectAxis [^int axis])) - -;; "Fast Hierarchical Clustering and Other Applications of Dynamic -;; Closest Pairs" -;; https://arxiv.org/pdf/cs/9912014.pdf - -(definterface IFastPair - (^double getDistance []) - (^Object getNeighbour []) - (^void updateNeighbour [^Object bin ^double distance])) - -(def ^:private ^Comparator fast-pair-comparator (Comparator/comparingDouble - (reify ToDoubleFunction - (applyAsDouble [_ x] - (.getDistance ^IFastPair x))))) - -(deftype MultiDimensionalBin [^doubles value - ^:unsynchronized-mutable ^long count - ^:unsynchronized-mutable ^double distance - ^:unsynchronized-mutable ^IMultiDimensionalBin neighbour] - IMultiDimensionalBin - (getValue [_] value) - - (getCount [_] count) - - (projectAxis [_ axis] - (Bin. (aget value axis) count)) - - IFastPair - (getDistance [_] - distance) - - (getNeighbour [_] - neighbour) - - (updateNeighbour [this bin distance] - (set! (.distance this) distance) - (set! (.neighbour this) bin))) - -(defn- vec-manhattan-distance ^double [^doubles x ^doubles y] - (let [len (alength x)] - (loop [n 0 - distance 0.0] - (if (= n len) - distance - (recur (inc n) (+ distance (Math/abs (- (aget x n) (aget y n))))))))) - -(defn- vec-squared-euclidean-distance ^double [^doubles x ^doubles y] - (let [len (alength x)] - (loop [n 0 - distance 0.0] - (if (= n len) - distance - (let [diff (- (aget x n) (aget y n))] - (recur (inc n) (+ distance (* diff diff)))))))) - -(defn- vec-euclidean-distance ^double [^doubles x ^doubles y] - (Math/sqrt (vec-squared-euclidean-distance x y))) - -(defn- vec-chessboard-distance ^double [^doubles x ^doubles y] - (let [len (alength x)] - (loop [n 0 - distance 0.0] - (if (= n len) - distance - (recur (inc n) (max distance (Math/abs (- (aget x n) (aget y n))))))))) - -(defn- find-neighbour [^List bins ^MultiDimensionalBin bin] - (let [p (.getValue bin)] - (.updateNeighbour bin nil Double/POSITIVE_INFINITY) - (.forEach bins (reify Consumer - (accept [_ b] - (let [^MultiDimensionalBin b b] - (when-not (identical? b bin) - (let [distance (vec-squared-euclidean-distance p (.getValue b))] - (when (< distance (.getDistance bin)) - (.updateNeighbour bin b distance)))))))))) - -(deftype MultiDimensionalHistogram [^int max-bins - ^int k - ^:unsynchronized-mutable ^long total - ^:unsynchronized-mutable ^doubles min-v - ^:unsynchronized-mutable ^doubles max-v - ^List bins] - IMultiDimensionalHistogram - (update [this p] - (set! (.total this) (inc total)) - (dotimes [n k] - (let [x (aget p n)] - (aset min-v n (min (aget min-v n) x)) - (aset max-v n (max (aget max-v n) x)))) - (let [new-bin (MultiDimensionalBin. p 1 Double/POSITIVE_INFINITY nil)] - (find-neighbour bins new-bin) - (.add bins new-bin) - - (while (> (.size bins) max-bins) - (let [^MultiDimensionalBin bin-i (Collections/min bins fast-pair-comparator) - ^MultiDimensionalBin bin-i+1 (.getNeighbour bin-i) - - qi (.getValue bin-i) - ki (.getCount bin-i) - qi+1 (.getValue bin-i+1) - ki+1 (.getCount bin-i+1) - new-k (+ ki ki+1) - new-q (double-array k) - _ (dotimes [n k] - (aset new-q n (/ (+ (* (aget qi n) ki) - (* (aget qi+1 n) ki+1)) new-k))) - new-bin (MultiDimensionalBin. new-q new-k Double/POSITIVE_INFINITY nil) - - bin-i-idx (.indexOf bins bin-i) - bin-i+1-idx (.indexOf bins bin-i+1) - last-idx (dec (.size bins))] - - (.set bins bin-i-idx new-bin) - (.set bins bin-i+1-idx (.get bins last-idx)) - (.remove bins last-idx) - - (find-neighbour bins new-bin) - - (.forEach bins (reify Consumer - (accept [_ b] - (let [^MultiDimensionalBin b b] - (let [neighbour (.getNeighbour b)] - (when (or (identical? bin-i neighbour) - (identical? bin-i+1 neighbour)) - (find-neighbour bins b)))))))))) - - this) - - (getMins [_] - min-v) - - (getMaxs [_] - max-v) - - (getTotal [_] - total) - - (getBins [_] - bins) - - (projectAxis [_ axis] - (Histogram. max-bins total (aget min-v axis) (aget max-v axis) (ArrayList. ^List (sort (for [^IMultiDimensionalBin b bins] - (.projectAxis b axis))))))) - -(defn ->multidimensional-histogram ^xtdb.temporal.histogram.IMultiDimensionalHistogram [^long max-bins ^long k] - (MultiDimensionalHistogram. max-bins k 0 (double-array k Double/POSITIVE_INFINITY) (double-array k Double/MIN_VALUE) - (ArrayList. (inc max-bins)))) diff --git a/core/src/main/clojure/xtdb/temporal/kd_tree.clj b/core/src/main/clojure/xtdb/temporal/kd_tree.clj deleted file mode 100644 index 6386aa4a41..0000000000 --- a/core/src/main/clojure/xtdb/temporal/kd_tree.clj +++ /dev/null @@ -1,640 +0,0 @@ -(ns xtdb.temporal.kd-tree - (:require [xtdb.types :as t] - [xtdb.util :as util]) - (:import (java.io Closeable) - java.lang.ref.Cleaner - (java.util ArrayDeque List Queue) - java.util.concurrent.LinkedBlockingQueue - (java.util.function LongBinaryOperator LongConsumer LongFunction LongPredicate LongUnaryOperator) - java.util.stream.LongStream - org.apache.arrow.memory.BufferAllocator - org.apache.arrow.vector.BigIntVector - org.apache.arrow.vector.complex.FixedSizeListVector - (org.apache.arrow.vector.types Types$MinorType) - (org.apache.arrow.vector.types.pojo ArrowType$FixedSizeList Field) - org.roaringbitmap.longlong.Roaring64Bitmap - xtdb.BitUtil)) - -(set! *unchecked-math* :warn-on-boxed) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface IKdTreePointAccess - (^java.util.List getPoint [^long idx]) - (^longs getArrayPoint [^long idx]) - (^long getCoordinate [^long idx ^int axis]) - (^void setCoordinate [^long idx ^int axis ^long value]) - (^void swapPoint [^long from-idx ^long to-idx]) - (^boolean isDeleted [^long idx] - "entry is deleted from the kd-tree, not that the entity/row was deleted") - (^boolean isInRange [^long idx ^longs min-range ^longs max-range ^int mask])) - -(defprotocol KdTree - (kd-tree-insert [_ allocator point]) - (kd-tree-delete [_ allocator point]) - (kd-tree-range-search [_ min-range max-range]) - (kd-tree-points [_ deletes?]) - (kd-tree-height [_]) - (kd-tree-retain [_ allocator]) - (kd-tree-point-access [_]) - (kd-tree-size [_]) - (kd-tree-value-count [_]) - (kd-tree-dimensions [_]) - (with-point-vec [_ point-vec])) - -(defn next-axis - {:inline (fn [axis k] - `(let [next-axis# (inc ~axis)] - (if (= ~k next-axis#) - 0 - next-axis#)))} - ^long [^long axis ^long k] - (let [next-axis (inc axis)] - (if (= k next-axis) - 0 - next-axis))) - -(def ^:private ^Class longs-class (Class/forName "[J")) - -(defn ->longs ^longs [xs] - (if (instance? longs-class xs) - xs - (long-array xs))) - -(declare ->node-kd-tree) - -(deftype NilPointAccess [] - IKdTreePointAccess - (getPoint [_ _] - (throw (IndexOutOfBoundsException.))) - (getArrayPoint [_ _] - (throw (IndexOutOfBoundsException.))) - (getCoordinate [_ _ _] - (throw (IndexOutOfBoundsException.))) - (setCoordinate [_ _ _ _] - (throw (UnsupportedOperationException.))) - (swapPoint [_ _ _] - (throw (UnsupportedOperationException.))) - (isDeleted [_ _] - (throw (IndexOutOfBoundsException.))) - (isInRange [_ _ _ _ _] - (throw (IndexOutOfBoundsException.)))) - -(extend-protocol KdTree - nil - (kd-tree-insert [_ allocator point] - (-> (->node-kd-tree allocator (count point)) - (kd-tree-insert allocator point))) - (kd-tree-delete [_ allocator point] - (-> (kd-tree-insert nil allocator point) - (kd-tree-delete allocator point))) - (kd-tree-range-search [_ _ _] - (LongStream/empty)) - (kd-tree-points [_ _deletes?] - (LongStream/empty)) - (kd-tree-height [_] -1) - (kd-tree-retain [_ _]) - (kd-tree-point-access [_] - (NilPointAccess.)) - (kd-tree-size [_] 0) - (kd-tree-value-count [_] 0) - (kd-tree-dimensions [_] 0) - (with-point-vec [_ _])) - -(deftype ListPointAccess [^List list] - IKdTreePointAccess - (getPoint [_ idx] - (.get list idx)) - (getArrayPoint [_ idx] - (->longs (.get list idx))) - (getCoordinate [this idx axis] - (aget ^longs (.getArrayPoint this idx) axis)) - (setCoordinate [_ _ _ _] - (throw (UnsupportedOperationException.))) - (swapPoint [_ _ _] - (throw (UnsupportedOperationException.))) - (isDeleted [_ _] - false) - (isInRange [this idx min-range max-range _axis] - (let [point (.getArrayPoint this idx) - len (alength point)] - (loop [n (int 0)] - (if (= n len) - true - (let [x (aget point n)] - (if (and (<= (aget min-range n) x) - (<= x (aget max-range n))) - (recur (inc n)) - false))))))) - -(extend-protocol KdTree - List - (kd-tree-insert [_ _allocator _point] - (throw (UnsupportedOperationException.))) - (kd-tree-delete [_ _allocator _point] - (throw (UnsupportedOperationException.))) - (kd-tree-range-search [this min-range max-range] - (let [min-range (->longs min-range) - max-range (->longs max-range) - ^IKdTreePointAccess access (kd-tree-point-access this)] - (.filter (LongStream/range 0 (.size this)) - (reify LongPredicate - (test [_ x] - (.isInRange access x min-range max-range -1)))))) - (kd-tree-points [this _deletes?] - (LongStream/range 0 (.size this))) - (kd-tree-height [_] 0) - (kd-tree-retain [this _] - this) - (kd-tree-point-access [this] - (ListPointAccess. this)) - (kd-tree-size [this] (.size this)) - (kd-tree-value-count [this] (.size this)) - (kd-tree-dimensions [this] (count (first this))) - (with-point-vec [this _vec] this)) - -(defn- write-coordinates [^IKdTreePointAccess access ^long idx point] - (if (instance? longs-class point) - (dotimes [n (alength ^longs point)] - (.setCoordinate access idx n (aget ^longs point n))) - (dotimes [n (count point)] - (.setCoordinate access idx n (long (nth point n)))))) - -(defn write-point ^long [^FixedSizeListVector point-vec ^IKdTreePointAccess access point] - (let [idx (.getValueCount point-vec)] - (.startNewValue point-vec idx) - (write-coordinates access idx point) - (.setValueCount point-vec (inc idx)) - idx)) - -(defn ->point-field ^org.apache.arrow.vector.types.pojo.Field [^long k] - (t/->field "point" (ArrowType$FixedSizeList. k) false - (t/->field "coordinates" (.getType Types$MinorType/BIGINT) false))) - -(deftype KdTreeVectorPointAccess [^FixedSizeListVector point-vec ^int k] - IKdTreePointAccess - (getPoint [_ idx] - (.getObject point-vec idx)) - - (getArrayPoint [_ idx] - (let [^BigIntVector coordinates-vec (.getDataVector point-vec) - point (long-array k) - element-start-idx (unchecked-multiply-int idx k)] - (dotimes [n k] - (aset point n (.get coordinates-vec (unchecked-add-int element-start-idx n)))) - point)) - - (getCoordinate [_ idx axis] - (let [^BigIntVector coordinates-vec (.getDataVector point-vec) - element-start-idx (unchecked-multiply-int idx k)] - (.get coordinates-vec (unchecked-add-int element-start-idx axis)))) - - (setCoordinate [_ idx axis value] - (let [^BigIntVector coordinates-vec (.getDataVector point-vec) - element-start-idx (unchecked-multiply-int idx k)] - (.setSafe coordinates-vec (unchecked-add-int element-start-idx axis) value))) - - (swapPoint [_ from-idx to-idx] - (let [^BigIntVector coordinates-vec (.getDataVector point-vec) - tmp (.isNull point-vec to-idx) - _ (if (.isNull point-vec from-idx) - (.setNull point-vec to-idx) - (.setNotNull point-vec to-idx)) - _ (if tmp - (.setNull point-vec from-idx) - (.setNotNull point-vec from-idx)) - from-idx (unchecked-multiply-int from-idx k) - to-idx (unchecked-multiply-int to-idx k)] - - (dotimes [axis k] - (let [from-idx (unchecked-add-int from-idx axis) - to-idx (unchecked-add-int to-idx axis) - tmp (.get coordinates-vec from-idx)] - (.set coordinates-vec from-idx (.get coordinates-vec to-idx)) - (.set coordinates-vec to-idx tmp))))) - - (isDeleted [_ idx] - (.isNull point-vec idx)) - - (isInRange [_ idx min-range max-range mask] - (let [^BigIntVector coordinates-vec (.getDataVector point-vec) - element-start-idx (unchecked-multiply-int idx k)] - (loop [n (int 0)] - (if (= n k) - true - (if (BitUtil/isBitSet mask n) - (let [x (.get coordinates-vec (unchecked-add-int element-start-idx n))] - (if (and (<= (aget min-range n) x) - (<= x (aget max-range n))) - (recur (inc n)) - false)) - (recur (inc n)))))))) - -(defn range-bitmask ^long [^longs min-range ^longs max-range] - (let [len (alength min-range)] - (loop [n 0 - mask 0] - (if (= n len) - mask - (recur (inc n) - (if (and (= Long/MIN_VALUE (aget min-range n)) - (= Long/MAX_VALUE (aget max-range n))) - mask - (bit-or mask (bit-shift-left 1 n)))))))) - -(defn kd-tree->seq - ([kd-tree] - (kd-tree->seq kd-tree (kd-tree-points kd-tree false))) - ([kd-tree ^LongStream stream] - (let [^IKdTreePointAccess point-access (kd-tree-point-access kd-tree)] - (-> stream - (.mapToObj (reify LongFunction - (apply [_ x] - (.getPoint point-access x)))) - (.iterator) - (iterator-seq))))) - -(def ^:private ^:const leaf-size 64) - -(declare ->leaf-node leaf-node-edit) - -(defonce ^:private ^Cleaner leaf-cleaner (Cleaner/create)) - -(deftype LeafNode [^FixedSizeListVector point-vec ^Queue leaf-pool ^long superseded ^int idx ^int axis ^int size ^boolean root? pool-token] - KdTree - (kd-tree-insert [kd-tree allocator point] - (leaf-node-edit kd-tree allocator point false)) - - (kd-tree-delete [kd-tree allocator point] - (leaf-node-edit kd-tree allocator point true)) - - (kd-tree-range-search [_ min-range max-range] - (let [^IKdTreePointAccess access (KdTreeVectorPointAccess. point-vec (.getListSize point-vec)) - min-range (->longs min-range) - max-range (->longs max-range) - axis-mask (range-bitmask min-range max-range) - acc (LongStream/builder)] - (dotimes [n size] - (let [x (+ idx n)] - (when (and (BitUtil/bitNot (BitUtil/isLongBitSet superseded n)) - (BitUtil/bitNot (.isDeleted access x)) - (.isInRange access x min-range max-range axis-mask)) - (.add acc x)))) - (.build acc))) - - (kd-tree-points [_ deletes?] - (let [^IKdTreePointAccess access (KdTreeVectorPointAccess. point-vec (.getListSize point-vec))] - (cond-> (LongStream/range 0 size) - (pos? superseded) (.filter (reify LongPredicate - (test [_ n] - (BitUtil/bitNot (BitUtil/isLongBitSet superseded n))))) - true (.map (reify LongUnaryOperator - (applyAsLong [_ n] - (+ idx n)))) - (BitUtil/bitNot deletes?) (.filter (reify LongPredicate - (test [_ x] - (BitUtil/bitNot (.isDeleted access x)))))))) - - (kd-tree-height [_] 0) - - (kd-tree-retain [_ allocator] - (LeafNode. (.getTo (doto (.getTransferPair point-vec allocator) - (.splitAndTransfer 0 (.getValueCount point-vec)))) - leaf-pool superseded idx axis size root? pool-token)) - - (kd-tree-point-access [_] - (KdTreeVectorPointAccess. point-vec (.getListSize point-vec))) - - (kd-tree-size [kd-tree] - (.count ^LongStream (kd-tree-points kd-tree false))) - - (kd-tree-value-count [_] - (.getValueCount point-vec)) - - (kd-tree-dimensions [_] - (.getListSize point-vec)) - - (with-point-vec [_ point-vec] - (LeafNode. point-vec leaf-pool superseded idx axis size root? pool-token)) - - Closeable - (close [_] - (when root? - (util/try-close point-vec)))) - -(deftype InnerNode [^FixedSizeListVector point-vec ^Queue leaf-pool ^long axis-value ^int axis left right ^boolean root?] - KdTree - (kd-tree-insert [_ allocator point] - (let [point (->longs point)] - (if (< (aget point axis) axis-value) - (InnerNode. point-vec leaf-pool axis-value axis (kd-tree-insert left allocator point) right root?) - (InnerNode. point-vec leaf-pool axis-value axis left (kd-tree-insert right allocator point) root?)))) - - (kd-tree-delete [_ allocator point] - (let [point (->longs point)] - (if (< (aget point axis) axis-value) - (InnerNode. point-vec leaf-pool axis-value axis (kd-tree-delete left allocator point) right root?) - (InnerNode. point-vec leaf-pool axis-value axis left (kd-tree-delete right allocator point) root?)))) - - (kd-tree-range-search [kd-tree min-range max-range] - (let [^IKdTreePointAccess access (KdTreeVectorPointAccess. point-vec (.getListSize point-vec)) - min-range (->longs min-range) - max-range (->longs max-range) - axis-mask (range-bitmask min-range max-range) - acc (LongStream/builder) - stack (ArrayDeque.)] - (loop [node kd-tree] - (cond - (instance? InnerNode node) - (let [^InnerNode node node - axis (.axis node) - axis-value (.axis-value node) - visit-left? (< (aget min-range axis) axis-value) - visit-right? (<= axis-value (aget max-range axis))] - (if visit-left? - (do (when visit-right? - (.push stack (.right node))) - (recur (.left node))) - (if visit-right? - (recur (.right node)) - (recur (.poll stack))))) - - (instance? LeafNode node) - (let [^LeafNode node node - size (.size node) - idx (.idx node) - superseded (.superseded node)] - (if (zero? superseded) - (dotimes [n size] - (let [x (+ idx n)] - (when (and (BitUtil/bitNot (.isDeleted access x)) - (.isInRange access x min-range max-range axis-mask)) - (.add acc x)))) - (dotimes [n size] - (let [x (+ idx n)] - (when (and (BitUtil/bitNot (BitUtil/isLongBitSet superseded n)) - (BitUtil/bitNot (.isDeleted access x)) - (.isInRange access x min-range max-range axis-mask)) - (.add acc x))))) - (recur (.poll stack))) - - :else - (.build acc))))) - - (kd-tree-points [_ deletes?] - (LongStream/concat (kd-tree-points left deletes?) (kd-tree-points right deletes?))) - - (kd-tree-height [_] - (max (inc (long (kd-tree-height left))) - (inc (long (kd-tree-height right))))) - - (kd-tree-retain [_ allocator] - (let [point-vec (.getTo (doto (.getTransferPair point-vec allocator) - (.splitAndTransfer 0 (.getValueCount point-vec))))] - (InnerNode. point-vec - leaf-pool - axis-value - axis - (-> left (with-point-vec point-vec)) - (-> right (with-point-vec point-vec)) - root?))) - - (with-point-vec [_ point-vec] - (InnerNode. point-vec - leaf-pool - axis-value - axis - (-> left (with-point-vec point-vec)) - (-> right (with-point-vec point-vec)) - root?)) - - (kd-tree-point-access [_] - (KdTreeVectorPointAccess. point-vec (.getListSize point-vec))) - - (kd-tree-size [_] - (+ (long (kd-tree-size left)) - (long (kd-tree-size right)))) - - (kd-tree-value-count [_] - (.getValueCount point-vec)) - - (kd-tree-dimensions [_] - (.getListSize point-vec)) - - Closeable - (close [_] - (when root? - (util/try-close point-vec)))) - -(defn- leaf-node-edit [^LeafNode kd-tree ^BufferAllocator allocator point deleted?] - (let [point (->longs point) - ^FixedSizeListVector point-vec (.point-vec kd-tree) - leaf-pool (.leaf-pool kd-tree) - k (.getListSize point-vec) - superseded (.superseded kd-tree) - idx (.idx kd-tree) - axis (.axis kd-tree) - size (.size kd-tree) - root? (.root? kd-tree) - pool-token (.pool-token kd-tree) - ^IKdTreePointAccess access (KdTreeVectorPointAccess. point-vec k)] - (if (< size leaf-size) - (let [point-idx (+ idx size)] - (write-coordinates access point-idx point) - (if deleted? - (.setNull point-vec point-idx) - (.setNotNull point-vec point-idx)) - (let [new-superseded (-> (LongStream/range 0 size) - (.filter (reify LongPredicate - (test [_ n] - (.isInRange access (+ idx n) point point -1)))) - (.reduce superseded (reify LongBinaryOperator - (applyAsLong [_ acc n] - (bit-or acc (bit-shift-left 1 n))))))] - (LeafNode. point-vec leaf-pool new-superseded idx axis (inc size) root? pool-token))) - (let [axis-values (-> (LongStream/range idx (+ idx size)) - (.map (reify LongUnaryOperator - (applyAsLong [_ x] - (.getCoordinate access x axis)))) - (.sorted) - (.toArray)) - axis-value (aget axis-values (BitUtil/unsignedBitShiftRight (alength axis-values) 1)) - next-axis (next-axis axis k) - left (->leaf-node point-vec leaf-pool next-axis) - right (->leaf-node point-vec leaf-pool next-axis)] - (loop [n 0 - acc (InnerNode. point-vec leaf-pool axis-value axis left right root?)] - (cond - (= n leaf-size) - (if deleted? - (kd-tree-delete acc allocator point) - (kd-tree-insert acc allocator point)) - - (BitUtil/isLongBitSet superseded n) - (recur (inc n) acc) - - :else - (let [point-idx (+ idx n) - point (.getArrayPoint access point-idx) - acc (if (.isDeleted access point-idx) - (kd-tree-delete acc allocator point) - (kd-tree-insert acc allocator point))] - (recur (inc n) acc)))))))) - -(defn- ->leaf-node - ([^FixedSizeListVector point-vec ^Queue leaf-pool ^long axis] - (->leaf-node point-vec leaf-pool axis false)) - ([^FixedSizeListVector point-vec ^Queue leaf-pool ^long axis root?] - (let [idx (or (.poll leaf-pool) - (let [idx (.getValueCount point-vec)] - (.setValueCount point-vec (+ idx leaf-size)) - idx)) - pool-token (Object.)] - (.register leaf-cleaner pool-token #(.offer leaf-pool idx)) - (LeafNode. point-vec leaf-pool 0 idx axis 0 root? pool-token)))) - -(defn ->node-kd-tree ^java.io.Closeable [^BufferAllocator allocator ^long k] - (let [^FixedSizeListVector point-vec (.createVector ^Field (->point-field k) allocator) - leaf-pool (LinkedBlockingQueue.)] - (->leaf-node point-vec leaf-pool 0 true))) - -(defn merge-kd-trees ^java.io.Closeable [^BufferAllocator allocator kd-tree-to kd-tree-from] - (let [^IKdTreePointAccess from-access (kd-tree-point-access kd-tree-from) - it (.iterator ^LongStream (kd-tree-points kd-tree-from true))] - (loop [acc kd-tree-to] - (if (.hasNext it) - (let [idx (.nextLong it) - point (.getArrayPoint from-access idx)] - (recur (if (.isDeleted from-access idx) - (kd-tree-delete acc allocator point) - (kd-tree-insert acc allocator point)))) - acc)))) - -(defn build-node-kd-tree ^java.io.Closeable [^BufferAllocator allocator kd-tree-from] - (merge-kd-trees allocator nil kd-tree-from)) - -(deftype MergedKdTreePointAccess [^IKdTreePointAccess static-access ^IKdTreePointAccess dynamic-access - ^Roaring64Bitmap static-delete-bitmap ^long static-value-count] - IKdTreePointAccess - (getPoint [_ idx] - (if (< idx static-value-count) - (.getPoint static-access idx) - (.getPoint dynamic-access (- idx static-value-count)))) - - (getArrayPoint [_ idx] - (if (< idx static-value-count) - (.getArrayPoint static-access idx) - (.getArrayPoint dynamic-access (- idx static-value-count)))) - - (getCoordinate [_ idx axis] - (if (< idx static-value-count) - (.getCoordinate static-access idx axis) - (.getCoordinate dynamic-access (- idx static-value-count) axis))) - - (setCoordinate [_ _ _ _] - (throw (UnsupportedOperationException.))) - - (swapPoint [_ _ _] - (throw (UnsupportedOperationException.))) - - (isDeleted [_ idx] - (if (< idx static-value-count) - (or (.contains static-delete-bitmap idx) (.isDeleted static-access idx)) - (.isDeleted dynamic-access (- idx static-value-count))))) - -#_{:clj-kondo/ignore [:clojure-lsp/unused-public-var]} -(definterface IDynamicKdTreeAccess - (^Object getDynamicKdTree [])) - -(deftype MergedKdTree [static-kd-tree ^:unsynchronized-mutable dynamic-kd-tree ^Roaring64Bitmap static-delete-bitmap ^long static-size ^long static-value-count] - IDynamicKdTreeAccess - (getDynamicKdTree [_] - dynamic-kd-tree) - - KdTree - (kd-tree-insert [this allocator point] - (set! (.dynamic-kd-tree this) (kd-tree-insert dynamic-kd-tree allocator point)) - this) - - (kd-tree-delete [this allocator point] - (.forEach ^LongStream (kd-tree-range-search static-kd-tree point point) - (reify LongConsumer - (accept [_ x] - (.addLong static-delete-bitmap x)))) - (set! (.dynamic-kd-tree this) (kd-tree-delete dynamic-kd-tree allocator point)) - this) - - (kd-tree-range-search [_ min-range max-range] - (LongStream/concat (.filter ^LongStream (kd-tree-range-search static-kd-tree min-range max-range) - (reify LongPredicate - (test [_ x] - (BitUtil/bitNot (.contains static-delete-bitmap x))))) - (.map ^LongStream (kd-tree-range-search dynamic-kd-tree min-range max-range) - (reify LongUnaryOperator - (applyAsLong [_ x] - (+ static-value-count x)))))) - - (kd-tree-points [_ deletes?] - (LongStream/concat (.filter ^LongStream (kd-tree-points static-kd-tree deletes?) - (reify LongPredicate - (test [_ x] - (or deletes? (BitUtil/bitNot (.contains static-delete-bitmap x)))))) - (.map ^LongStream (kd-tree-points dynamic-kd-tree deletes?) - (reify LongUnaryOperator - (applyAsLong [_ x] - (+ static-value-count x)))))) - - (kd-tree-height [_] - (max (long (kd-tree-height static-kd-tree)) - (long (kd-tree-height dynamic-kd-tree)))) - - (kd-tree-retain [_ allocator] - (MergedKdTree. (kd-tree-retain static-kd-tree allocator) - (kd-tree-retain dynamic-kd-tree allocator) - (.clone ^Roaring64Bitmap static-delete-bitmap) - static-size - static-value-count)) - - (with-point-vec [_ point-vec] - (MergedKdTree. (-> static-kd-tree (with-point-vec point-vec)) - (-> dynamic-kd-tree (with-point-vec point-vec)) - (.clone ^Roaring64Bitmap static-delete-bitmap) - static-size - static-value-count)) - - (kd-tree-point-access [_] - (MergedKdTreePointAccess. (kd-tree-point-access static-kd-tree) (kd-tree-point-access dynamic-kd-tree) - static-delete-bitmap static-value-count)) - - (kd-tree-size [_] - (+ (- static-size (.getLongCardinality static-delete-bitmap)) - (long (kd-tree-size dynamic-kd-tree)))) - - (kd-tree-value-count [_] - (+ (long (kd-tree-value-count static-kd-tree)) - (long (kd-tree-value-count dynamic-kd-tree)))) - - (kd-tree-dimensions [_] - (kd-tree-dimensions static-kd-tree)) - - Closeable - (close [_] - (util/try-close static-kd-tree) - (util/try-close dynamic-kd-tree) - (.clear static-delete-bitmap))) - -(defn ->merged-kd-tree - (^xtdb.temporal.kd_tree.MergedKdTree [static-kd-tree] - (->merged-kd-tree static-kd-tree nil)) - (^xtdb.temporal.kd_tree.MergedKdTree [static-kd-tree dynamic-kd-tree] - (let [static-delete-bitmap (Roaring64Bitmap.) - ^IKdTreePointAccess access (kd-tree-point-access dynamic-kd-tree)] - (.forEach ^LongStream (kd-tree-points dynamic-kd-tree true) - (reify LongConsumer - (accept [_ n] - (when (.isDeleted access n) - (let [point (.getArrayPoint access n)] - (.forEach ^LongStream (kd-tree-range-search static-kd-tree point point) - (reify LongConsumer - (accept [_ x] - (.addLong static-delete-bitmap x))))))))) - (MergedKdTree. static-kd-tree dynamic-kd-tree static-delete-bitmap (kd-tree-size static-kd-tree) (kd-tree-value-count static-kd-tree))))) diff --git a/core/src/main/clojure/xtdb/types.clj b/core/src/main/clojure/xtdb/types.clj index 5b6d3ff3bc..a8d5de732a 100644 --- a/core/src/main/clojure/xtdb/types.clj +++ b/core/src/main/clojure/xtdb/types.clj @@ -18,6 +18,14 @@ (def temporal-col-type [:timestamp-tz :micro "UTC"]) (def nullable-temporal-type [:union #{:null temporal-col-type}]) +(def temporal-col-types + {"_iid" :i64, "_row_id" :i64 + "xt$system_from" temporal-col-type, "xt$system_to" temporal-col-type + "xt$valid_from" temporal-col-type, "xt$valid_to" temporal-col-type}) + +(defn temporal-column? [col-name] + (contains? temporal-col-types (str col-name))) + (defn ->field ^org.apache.arrow.vector.types.pojo.Field [^String field-name ^ArrowType arrow-type nullable & children] (Field. field-name (FieldType. nullable arrow-type nil nil) children)) diff --git a/core/src/main/clojure/xtdb/watermark.clj b/core/src/main/clojure/xtdb/watermark.clj index 29eec0d6b4..165f470ed3 100644 --- a/core/src/main/clojure/xtdb/watermark.clj +++ b/core/src/main/clojure/xtdb/watermark.clj @@ -1,22 +1,19 @@ (ns xtdb.watermark (:require [clojure.tools.logging :as log] xtdb.api.protocols - xtdb.live-chunk xtdb.indexer.live-index - xtdb.temporal + xtdb.live-chunk [xtdb.util :as util]) - (:import xtdb.api.protocols.TransactionInstant - xtdb.live_chunk.ILiveChunkWatermark + (:import java.lang.AutoCloseable + java.util.concurrent.atomic.AtomicInteger + xtdb.api.protocols.TransactionInstant xtdb.indexer.live_index.ILiveIndexWatermark - xtdb.temporal.ITemporalRelationSource - java.lang.AutoCloseable - java.util.concurrent.atomic.AtomicInteger)) + xtdb.live_chunk.ILiveChunkWatermark)) #_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} (definterface IWatermark (^xtdb.api.protocols.TransactionInstant txBasis []) (^xtdb.live_chunk.ILiveChunkWatermark liveChunk []) - (^xtdb.temporal.ITemporalRelationSource temporalRootsSource []) (^xtdb.indexer.live_index.ILiveIndexWatermark liveIndex []) (^void retain []) @@ -29,13 +26,10 @@ (^xtdb.watermark.IWatermark openWatermark [^xtdb.api.protocols.TransactionInstant txKey])) (deftype Watermark [^TransactionInstant tx-key, ^ILiveChunkWatermark live-chunk, ^ILiveIndexWatermark live-idx-wm - ^ITemporalRelationSource temporal-roots-src - ^boolean close-temporal-roots? ^AtomicInteger ref-cnt] IWatermark (txBasis [_] tx-key) (liveChunk [_] live-chunk) - (temporalRootsSource [_] temporal-roots-src) (liveIndex [_] live-idx-wm) (retain [this] @@ -50,10 +44,7 @@ (when (zero? (.decrementAndGet ref-cnt)) (log/trace "close wm" (hash this)) (util/try-close live-chunk) - (util/try-close live-idx-wm) - - (when close-temporal-roots? - (util/try-close temporal-roots-src))))) + (util/try-close live-idx-wm)))) -(defn ->wm ^xtdb.watermark.IWatermark [tx-key live-chunk live-idx-wm temporal-roots-src close-temporal-roots?] - (Watermark. tx-key live-chunk live-idx-wm temporal-roots-src close-temporal-roots? (AtomicInteger. 1))) +(defn ->wm ^xtdb.watermark.IWatermark [tx-key live-chunk live-idx-wm] + (Watermark. tx-key live-chunk live-idx-wm (AtomicInteger. 1))) diff --git a/dev/doc/kd_tree.adoc b/dev/doc/kd_tree.adoc deleted file mode 100644 index 7ee22a851b..0000000000 --- a/dev/doc/kd_tree.adoc +++ /dev/null @@ -1,119 +0,0 @@ -= KD-tree - -The KD-tree can not be thought of completely independently of the link:temporal.md[Grid]. -So if you want to understand the whole picture of the temporal structure you will probably need -to read that document too. The KD-tree serves to store temporal data and answer questions -on the temporal data. - -https://en.wikipedia.org/wiki/K-d_tree[Wikipedia] has good page on kd-trees, we suggest that -, if you have not already, have a look at that first. -In short KD-trees store points (with a fixed number of dimensions) -and let you query (by providing sub-ranges in (potentially) several dimensions) for those points efficiently. -It achieves this by building a tree where on -every internal node the points are partitioned according to some dimension. In our case the points -are made up temporal data + some connection to the content. - -Both the KD-tree and the Grid implement an KD-tree interface. When talking about the kd-tree -we referring to the in-memory part of the whole structure. - -In the following we are describing the content and that goes in and out of the KD-tree/Grid. -The KD-tree (currently) stores points in a 6D-space. The six-dimensions are the following. -- `iid` - an internal representation of xt/id -- `row-id` - an id referring to a specific version of a document -- valid-time-start - -- valid-time-end - -- system-time-start - -- system-time-end - - -As a simple example, let's say you put the following two documents: -`[put :foo {:xt/id 1 :bar 1} {:for-valid-time [:in "2020"]}]` -`[put :foo {:xt/id 1 :bar 2} {:for-valid-time ["2021" "2023"]}]` -in two different transactions where system-time is equal to valid-time. -Then the KD-tree stores all the temporal information related to these -two document versions (meaning validity of the first document according to valid-time and system-time etc..). - -After the first transaction the kd-tree would contain the point (according to the order above) -`[1 1 "2020" "+inf" "2020" "+inf"]`. After the second transaction the tree would contain the points. -[source,clojure] ----- -[1 1 "2020" "+inf" "2020" "2021"] ;; old version, see different sys-time -[1 1 "2020" "-2021" "2021" "+inf"] ;; old version, first valid part interval -[1 2 "2021" "2023" "2021" "+inf"] ;; new version -[1 1 "2023+" "+inf" "2021" "+inf"] ;; old version, second vaild part interval ----- - -Beware that the tree does not store the data in the format above (that's just for illustration) but rather -as 64-bit longs. The dimensions might also not appear in this order. The order is quite important -as a highly selective dimension coming earlier is better. This is even more important for the grid -(see the grid for more details). - -In a bitemporal system it is quite obvious why one would need to store the later four dimensions (temporal dimensions). -- `iid`- lets you easily query for all the versions of a document. -- `row-id`- with the row-id you connect content to a validity time range (in both system and valid time) - -One could potentially ask why both ids are needed as one could always just store `row-id` -and go from `iid` to `row-ids`. That is a valid question. Its essentially a tradeoff. -One can (without indirection) ask for all versions + validity of a document if `iid` is in the tree. -Otherwise one has to first go off to the content and retrieve all `row-id` s for a specific `iid`. - -When querying the tree one specifies a min and a max point. It then returns all points where each dimensions falls in -the range of that dimension specified by min and max. For example: -[source,clojure] ----- -[1 "-inf" "-inf" "2020+" "-inf" "2023"] ;; min -[1 "+inf" "2023-" "+inf" "+inf" "+inf"] ;; max ----- -Would return all versions of entity `1` which are valid between 2020 and 2023 looking at the database in 2023. - -The leaves of the KD-tree store the actual coordinates in an arrow `FixedSizeListVector`. These coordinates get -recycled with a java.lang.ref.Cleaner (see the kd_tree.clj for details) mechansim. Arrow vectors can be nulled -even if there is still data at that position. This has the special meaning of a delete (more on this below). - -=== Temporal update - -Let us walk through a temporal update with the example of the -second transaction `[:put :foo {:xt/id 1 :bar 2} {:for-valid-time ["2021" "2023"]}]`. -The temporal coordinate for that put is `[1 2 "2021" "2023" "2021" "+inf"]`. We would then -query for overlapping entity versions (this step can be skipped if we know that we are dealing with a new entity). -The min and max range coordinates would then look as follows: -[source,clojure] ----- -[1 "-inf" "-inf" "2021+" "-inf" "2021"] ;; min -[1 "+inf" "2023-" "+inf" "+inf" "+inf"] ;; max ----- -The restriction on `iid` is because we are only interested in entity id `1`. We don't know anything about the -row-ids of that entity so we are not filtering anything (second column). We are interested in versions that -have a `valid-time-start` before 2023 (third column) and are `valid-time-end` at least a bit after 2021 (fourth column. -We need those columns to be valid in system-time after 2021 (fifth and sixth column). - -We then get back a set of points which intersect with the new update (only `[1 1 "2020" "+inf" "2020" "+inf"]` in this case) -This coordinate is no longer valid after our update. It needs to have it's system-time updated to `2021`. -In practice this means we delete the coordinate and insert a new one (`[1 1 "2020" "+inf" "2020" "2021"]`). - -The second thing that can happen is that the valid-times of the returned coordinates do not strictly lie within -the valid-time of the new version. In that case we need to potentially add two new coordinates (one if there is a valid-time -part that comes before the new version and one that comes after) to the tree. -This is the case in our example. So we need to add the following two updates as well. - -[source,clojure] ----- -[1 1 "2020" "-2021" "2021" "+inf"] ;; old version, first valid part interval -[1 1 "2023+" "+inf" "2021" "+inf"] ;; old version, second vaild part interval ----- - -=== Ingestion process - -The above temporal update happens on the KD-tree in memory. Once a chunk is finished (~1e5 row-ids) the kd-tree gets -merged into the grid (on disk). The grid does not support the update/insert operations. The grid is one file so a -KD-tree flush triggers a build of a new grid file (which is done in some background thread). -There is the notion of a `MergedKDTree` where one part is static (grid) and the other is dynamic (in-memory). -Search operations on this structure just branch off to both grid and kd-tree. Update and insert operations are only -done on the KD-tree. It might happen that the `MergeKDTree` has multiple levels. This happens if the merging of -the single grid-file has not finished yet and there are still multiple kd-trees (not yet merged into the grid) in -memory. - -One issue that arises from this is that one can not update/delete on the grid part of the whole KD-tree. Imagine the case -where we get the second `put` above in the current chunk but the first `put` is in some earlier chunk already in the grid. -For that we store a `delete` in the dynamic part (kd-tree). When searching for coordinates the deletes in the -kd-tree filter out the coordinates coming from the grid. These deletes are stored as usual in the underlying -Arrow Vector, but marked as a delete via nulling that index. diff --git a/modules/bench/src/main/clojure/xtdb/bench/multinode_tpch.clj b/modules/bench/src/main/clojure/xtdb/bench/multinode_tpch.clj index 2e8c8e4fa0..49100b4aa4 100644 --- a/modules/bench/src/main/clojure/xtdb/bench/multinode_tpch.clj +++ b/modules/bench/src/main/clojure/xtdb/bench/multinode_tpch.clj @@ -1,14 +1,12 @@ (ns xtdb.bench.multinode-tpch (:require [clojure.tools.logging :as log] [xtdb.bench :as bench] - [xtdb.ingester :as ingest] - [xtdb.node :as node] [xtdb.datasets.tpch :as tpch] [xtdb.datasets.tpch.ra :as tpch.ra] - [xtdb.temporal :as temporal] + [xtdb.indexer :as idx] + [xtdb.node :as node] [xtdb.test-util :as tu] - [xtdb.util :as util] - [xtdb.indexer :as idx]) + [xtdb.util :as util]) (:import java.nio.file.attribute.FileAttribute java.nio.file.Files java.time.Duration @@ -48,7 +46,6 @@ (test-node k node)) (idx/finish-chunk! (util/component primary-node :xtdb/indexer)) - (.awaitSnapshotBuild ^xtdb.temporal.TemporalManagerPrivate (::temporal/temporal-manager @(:!system primary-node))) (log/info "Starting post finish-chunk node") (with-open [^xtdb.node.Node secondary-node4 (start-node)] diff --git a/src/main/clojure/xtdb/test_util.clj b/src/main/clojure/xtdb/test_util.clj index 3200434f8b..fe3731adb3 100644 --- a/src/main/clojure/xtdb/test_util.clj +++ b/src/main/clojure/xtdb/test_util.clj @@ -11,7 +11,6 @@ xtdb.object-store [xtdb.operator :as op] xtdb.operator.scan - [xtdb.temporal :as temporal] [xtdb.types :as types] [xtdb.util :as util] [xtdb.vector :as vec] @@ -125,13 +124,9 @@ (defn with-mock-clock [f] (with-opts {:xtdb.log/memory-log {:instant-src (->mock-clock)}} f)) -(defn await-temporal-snapshot-build [node] - (.awaitSnapshotBuild ^xtdb.temporal.TemporalManagerPrivate (util/component node ::temporal/temporal-manager))) - (defn finish-chunk! [node] (idx/finish-block! (component node :xtdb/indexer)) - (idx/finish-chunk! (component node :xtdb/indexer)) - (await-temporal-snapshot-build node)) + (idx/finish-chunk! (component node :xtdb/indexer))) (defn open-vec (^org.apache.arrow.vector.ValueVector [col-name vs] diff --git a/src/test/clojure/xtdb/current_row_ids_test.clj b/src/test/clojure/xtdb/current_row_ids_test.clj deleted file mode 100644 index ecd087c764..0000000000 --- a/src/test/clojure/xtdb/current_row_ids_test.clj +++ /dev/null @@ -1,660 +0,0 @@ -(ns xtdb.current-row-ids-test - (:require [clojure.test :as t :refer [deftest]] - [xtdb.api :as xt] - [xtdb.operator.scan :as scan] - [xtdb.temporal :as temporal] - [xtdb.temporal.kd-tree-test :refer [as-micros ->coordinates]] - [xtdb.test-util :as tu] - [xtdb.util :as util]) - (:import org.roaringbitmap.longlong.Roaring64Bitmap - java.io.Closeable - org.apache.arrow.memory.RootAllocator - java.time.Duration)) - -(t/use-fixtures :each tu/with-mock-clock tu/with-node) - -(def - tx1 - '[[:put :xt_docs {:xt/id :ivan, :first-name "Ivan"}] - [:put :xt_docs {:xt/id :petr, :first-name "Petr"} - {:for-valid-time [:in #inst "2020-01-02T12:00:00Z"]}] - [:put :xt_docs {:xt/id :susie, :first-name "Susie"} - {:for-valid-time [:in nil #inst "2020-01-02T13:00:00Z"]}] - [:put :xt_docs {:xt/id :sam, :first-name "Sam"}] - [:put :xt_docs {:xt/id :petr, :first-name "Petr"} - {:for-valid-time [:in #inst "2020-01-04T12:00:00Z"]}] - [:put :xt_docs {:xt/id :jen, :first-name "Jen"} - {:for-valid-time [:in nil #inst "2020-01-04T13:00:00Z"]}] - [:put :xt_docs {:xt/id :james, :first-name "James"} - {:for-valid-time [:in #inst "2020-01-01T12:00:00Z"]}] - [:put :xt_docs {:xt/id :jon, :first-name "Jon"} - {:for-valid-time [:in nil #inst "2020-01-01T12:00:00Z"]}] - [:put :xt_docs {:xt/id :lucy :first-name "Lucy"}]]) - -(deftest test-current-row-ids - (xt/submit-tx - tu/*node* - tx1) - - (xt/submit-tx - tu/*node* - '[[:put :xt_docs {:xt/id :ivan, :first-name "Ivan-2"} - {:for-valid-time [:in #inst "2020-01-02T14:00:00Z"]}] - [:put :xt_docs {:xt/id :ben, :first-name "Ben"} - {:for-valid-time [:in #inst "2020-01-02T14:00:00Z" #inst "2020-01-02T15:00:00Z"]}] - [:evict :xt_docs :lucy]]) - - (t/is (= [{:name "Ivan-2"} - {:name "James"} - {:name "Jen"} - {:name "Petr"} - {:name "Sam"}] - (xt/q tu/*node* - '{:find [name] - :where [(match :xt_docs {:first-name name})] - :order-by [[name :asc]]} - {:basis {:current-time #time/instant "2020-01-03T00:00:00Z"}})))) ;; timing - -(defn valid-ids-at [current-time] - (xt/q tu/*node* - '{:find [id] - :where [(match :xt_docs {:xt/id id})]} - {:basis {:current-time current-time}})) - -(deftest test-current-row-ids-app-time-start-inclusivity - (t/testing "app-time-start" - (xt/submit-tx - tu/*node* - '[[:put :xt_docs {:xt/id 1} - {:for-valid-time [:in #inst "2020-01-01T00:00:02Z"]}]]) - - (t/is (= [] - (valid-ids-at #time/instant "2020-01-01T00:00:01Z"))) - (t/is (= [{:id 1}] - (valid-ids-at #time/instant "2020-01-01T00:00:02Z"))) - (t/is (= [{:id 1}] - (valid-ids-at #time/instant "2020-01-01T00:00:03Z"))))) - -(deftest test-current-row-ids-app-time-end-inclusivity - (t/testing "app-time-start" - (xt/submit-tx - tu/*node* - '[[:put :xt_docs {:xt/id 1} - {:for-valid-time [:in nil #inst "2020-01-01T00:00:02Z"]}]]) - - (t/is (= [{:id 1}] - (valid-ids-at #time/instant "2020-01-01T00:00:01Z"))) - (t/is (= [] - (valid-ids-at #time/instant "2020-01-01T00:00:02Z"))) - (t/is (= [] - (valid-ids-at #time/instant "2020-01-01T00:00:03Z"))))) - - -(deftest remove-evicted-row-ids-test - (t/is - (= #{1 3} - (temporal/remove-evicted-row-ids - #{1 2 3} - (doto - (Roaring64Bitmap.) - (.addLong (long 2))))))) - -(deftest current-row-ids-can-be-built-at-startup - (let [node-dir (util/->path "target/can-build-current-row-ids-at-startup") - expectation [{:name "Ivan"} - {:name "James"} - {:name "Jen"} - {:name "Lucy"} - {:name "Petr"} - {:name "Sam"}]] - (util/delete-dir node-dir) - - (with-open [node (tu/->local-node {:node-dir node-dir})] - - (-> (xt/submit-tx node tx1) - (tu/then-await-tx node (Duration/ofMillis 2000))) - - (tu/finish-chunk! node) - - (t/is (= expectation - (xt/q node - '{:find [name] - :where [(match :xt_docs {:first-name name})] - :order-by [[name :asc]]} - {:basis {:current-time #time/instant "2020-01-03T00:00:00Z"}})))) - - (with-open [node (tu/->local-node {:node-dir node-dir})] - (t/is (= expectation - (xt/q node - '{:find [name] - :where [(match :xt_docs {:first-name name})] - :order-by [[name :asc]]} - {:basis {:current-time #time/instant "2020-01-03T00:00:00Z"}})))))) - -(defn current-rows-for [system-time inserts] - (let [kd-tree nil - !current-row-ids (volatile! #{})] - (with-open [allocator (RootAllocator.) - ^Closeable kd-tree (reduce - (fn [cur-kd-tree coords] - (temporal/insert-coordinates cur-kd-tree - allocator - coords - !current-row-ids - (as-micros system-time))) - kd-tree - inserts)] - @!current-row-ids))) - -(deftest current-row-ids-inserts - (let [system-time #inst "2020-01-02"] - (t/is (= - #{1} - (current-rows-for - system-time - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #inst "2020-01-02" - :new-entity? true})])) - "app-time-start equal to system-time")) - - (let [system-time #inst "2020-01-02"] - (t/is (= - #{1} - (current-rows-for - system-time - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #inst "2020-01-01" - :new-entity? true})])) - "app-time-start before system-time")) - - (let [system-time #inst "2020-01-02"] - (t/is (= - #{} - (current-rows-for - system-time - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #inst "2020-01-03" - :new-entity? true})])) - "app-time-start after system-time")) - - (let [system-time #inst "2020-01-02"] - (t/is (= - #{} - (current-rows-for - system-time - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #inst "2020-01-01" - :app-time-end #inst "2020-01-02" - :new-entity? true})])) - "app-time-start and end before system-time")) - - (let [system-time #inst "2020-01-02"] - (t/is (= - #{1 2} - (current-rows-for - system-time - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #inst "2020-01-02" - :new-entity? true}) - (->coordinates {:id 102 - :row-id 2 - :system-time-start system-time - :app-time-start #inst "2020-01-01" - :new-entity? true})])) - "one row for each entity is added"))) - -(deftest current-row-ids-overlaps - (let [system-time #inst "2020-01-02"] - (t/is (= - #{} - (current-rows-for - system-time - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #inst "2020-01-02" - :new-entity? true}) - (->coordinates {:id 101 - :row-id 2 - :system-time-start system-time - :app-time-start #inst "2020-01-02" - :new-entity? false - :tombstone? true})])) - "delete overlapping at current system-time")) - - (let [system-time #inst "2020-01-02"] - (t/is (= - #{1} - (current-rows-for - system-time - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #inst "2020-01-02" - :app-time-end #inst "2020-01-10" - :new-entity? true}) - (->coordinates {:id 101 - :row-id 2 - :system-time-start system-time - :app-time-start #inst "2020-01-03" - :app-time-end #inst "2020-01-20" - :new-entity? false - :tombstone? true})])) - "delete overlapping after current system-time")) - - (let [system-time #inst "2020-01-02"] - (t/is (= - #{} - (current-rows-for - system-time - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #inst "2020-01-01" - :app-time-end #inst "2020-01-10" - :new-entity? true}) - (->coordinates {:id 101 - :row-id 2 - :system-time-start system-time - :app-time-start #inst "2020-01-02" - :app-time-end #inst "2020-01-04" - :new-entity? false - :tombstone? true})])) - "delete overlapping before current system-time")) - - (let [system-time #inst "2020-01-02"] - (t/is (= - #{2} - (current-rows-for - system-time - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #inst "2020-01-01" - :new-entity? true}) - (->coordinates {:id 101 - :row-id 2 - :system-time-start system-time - :app-time-start #inst "2020-01-02" - :new-entity? false})])) - "new put overlapping at current system-time"))) - -(deftest advance-current-row-ids-add-app-time-start-upper-bound - (let [system-time #time/instant "2020-01-01T00:00:01.000001Z" - kd-tree nil - !current-row-ids (volatile! #{})] - (with-open [allocator (RootAllocator.) - ^Closeable kd-tree (reduce - (fn [cur-kd-tree coords] - (temporal/insert-coordinates cur-kd-tree - allocator - coords - !current-row-ids - (util/instant->micros system-time))) - kd-tree - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #time/instant "2020-01-01T00:00:01.000010Z" - :new-entity? true})])] - (t/is (= #{} - @!current-row-ids)) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000009Z")))) - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000010Z")))) - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000011Z"))))))) - -(deftest advance-current-row-ids-add-app-time-start-lower-bound - (let [system-time #time/instant "2020-01-01T00:00:01.000001Z" - kd-tree nil - !current-row-ids (volatile! #{})] - (with-open [allocator (RootAllocator.) - ^Closeable kd-tree (reduce - (fn [cur-kd-tree coords] - (temporal/insert-coordinates cur-kd-tree - allocator - coords - !current-row-ids - (util/instant->micros system-time))) - kd-tree - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #time/instant "2020-01-01T00:00:01.000002Z" - :new-entity? true})])] - (t/is (= #{} - @!current-row-ids)) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000001Z")))) - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000002Z")))) - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000003Z"))))))) - - -(deftest advance-current-row-ids-add-app-time-end-lower-bound - (let [system-time #time/instant "2020-01-01T00:00:01.000001Z" - kd-tree nil - !current-row-ids (volatile! #{})] - (with-open [allocator (RootAllocator.) - ^Closeable kd-tree (reduce - (fn [cur-kd-tree coords] - (temporal/insert-coordinates cur-kd-tree - allocator - coords - !current-row-ids - (util/instant->micros system-time))) - kd-tree - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #time/instant "2020-01-01T00:00:01.000008Z" - :app-time-end #time/instant "2020-01-01T00:00:01.000010Z" - :new-entity? true})])] - (t/is (= #{} - @!current-row-ids)) - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000009Z")))) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000010Z")))) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000011Z"))))))) - - -(deftest advance-current-row-ids-remove-app-time-end-upper-bound - (let [system-time #time/instant "2020-01-01T00:00:01.000001Z" - kd-tree nil - !current-row-ids (volatile! #{})] - (with-open [allocator (RootAllocator.) - ^Closeable kd-tree (reduce - (fn [cur-kd-tree coords] - (temporal/insert-coordinates cur-kd-tree - allocator - coords - !current-row-ids - (util/instant->micros system-time))) - kd-tree - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-end #time/instant "2020-01-01T00:00:01.000010Z" - :new-entity? true})])] - (t/is (= #{1} - @!current-row-ids)) - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000009Z")))) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000010Z")))) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000011Z"))))))) - -(deftest advance-current-row-ids-remove-app-time-end-lower-bound - (let [system-time #time/instant "2020-01-01T00:00:01.000001Z" - kd-tree nil - !current-row-ids (volatile! #{})] - (with-open [allocator (RootAllocator.) - ^Closeable kd-tree (reduce - (fn [cur-kd-tree coords] - (temporal/insert-coordinates cur-kd-tree - allocator - coords - !current-row-ids - (util/instant->micros system-time))) - kd-tree - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-end #time/instant "2020-01-01T00:00:01.000002Z" - :new-entity? true})])] - (t/is (= #{1} - @!current-row-ids)) - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000001Z")))) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000002Z")))) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000003Z"))))))) - -(deftest advance-current-row-ids-remove-app-time-start-lower-bound - (let [system-time #time/instant "2020-01-01T00:00:01.000001Z" - kd-tree nil - !current-row-ids (volatile! #{})] - (with-open [allocator (RootAllocator.) - ^Closeable kd-tree (reduce - (fn [cur-kd-tree coords] - (temporal/insert-coordinates cur-kd-tree - allocator - coords - !current-row-ids - (util/instant->micros system-time))) - kd-tree - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start #time/instant "2020-01-01T00:00:01.000001Z" - :app-time-end #time/instant "2020-01-01T00:00:01.000002Z" - :new-entity? true})])] - (t/is (= #{1} - @!current-row-ids)) - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000001Z")))) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000002Z")))) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000003Z"))))))) - -(deftest current-row-ids-from-start-test - (let [system-time #time/instant "2020-01-01T00:00:01.000001Z" - kd-tree nil - !current-row-ids (volatile! #{})] - (with-open [allocator (RootAllocator.) - ^Closeable kd-tree (reduce - (fn [cur-kd-tree coords] - (temporal/insert-coordinates cur-kd-tree - allocator - coords - !current-row-ids - (util/instant->micros system-time))) - kd-tree - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start system-time - :new-entity? true}) - (->coordinates {:id 101 - :row-id 2 - :system-time-start system-time - :app-time-start #time/instant "2020-01-01T00:00:01.000010Z" - :new-entity? false})])] - (t/is (= #{1} - (temporal/current-row-ids-from-start - kd-tree - (util/instant->micros #time/instant "2020-01-01T00:00:01.000009Z")))) - - (t/is (= #{2} - (temporal/current-row-ids-from-start - kd-tree - (util/instant->micros #time/instant "2020-01-01T00:00:01.000010Z")))) - - (t/is (= #{2} - (temporal/current-row-ids-from-start - kd-tree - (util/instant->micros #time/instant "2020-01-01T00:00:01.000011Z"))))))) - -(deftest advance-current-row-ids-multiple-periods - ;; test proves the need to sort additions/removals by valid time - (let [system-time #time/instant "2020-01-01T00:00:01.000001Z" - kd-tree nil - !current-row-ids (volatile! #{})] - (with-open [allocator (RootAllocator.) - ^Closeable kd-tree (reduce - (fn [cur-kd-tree coords] - (temporal/insert-coordinates cur-kd-tree - allocator - coords - !current-row-ids - (util/instant->micros system-time))) - kd-tree - [(->coordinates {:id 101 - :row-id 1 - :system-time-start system-time - :app-time-start system-time - :new-entity? true}) - (->coordinates {:id 101 - :row-id 2 - :system-time-start system-time - :app-time-start #time/instant "2020-01-01T00:00:01.000003Z" - :app-time-end #time/instant "2020-01-01T00:00:01.000004Z" - :new-entity? false - :tombstone? true}) - (->coordinates {:id 101 - :row-id 3 - :system-time-start system-time - :app-time-start #time/instant "2020-01-01T00:00:01.000006Z" - :app-time-end #time/instant "2020-01-01T00:00:01.000008Z" - :new-entity? false - :tombstone? true})])] - (t/is (= #{1} - @!current-row-ids)) - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000002Z")))) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000003Z")))) - - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000004Z")))) - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000005Z")))) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000006Z")))) - - (t/is (= #{} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000007Z")))) - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000008Z")))) - - (t/is (= #{1} - (temporal/advance-current-row-ids - @!current-row-ids kd-tree - (util/instant->micros system-time) - (util/instant->micros #time/instant "2020-01-01T00:00:01.000009Z"))))))) - -(deftest test-query-empty-db - (t/is - (= [] - (xt/q - tu/*node* - '{:find [name] - :where [(match :xt_docs {:first-name name})]})) - "watermark for empty db will have no basis so don't use current-row-ids")) diff --git a/src/test/clojure/xtdb/indexer_test.clj b/src/test/clojure/xtdb/indexer_test.clj index 28a49a58b5..17d16dd478 100644 --- a/src/test/clojure/xtdb/indexer_test.clj +++ b/src/test/clojure/xtdb/indexer_test.clj @@ -134,7 +134,7 @@ (let [buffer-name "chunk-00/device_info/metadata.arrow" ^ArrowBuf buffer @(.getBuffer bp buffer-name) footer (util/read-arrow-footer buffer)] - (t/is (= 2 (count (.buffers ^BufferPool bp)))) + (t/is (= 1 (count (.buffers ^BufferPool bp)))) (t/is (instance? ArrowBuf buffer)) (t/is (= 2 (.getRefCount (.getReferenceManager ^ArrowBuf buffer)))) @@ -190,7 +190,7 @@ (t/is (zero? (.getRefCount (.getReferenceManager ^ArrowBuf buffer)))) (t/is (= size (.getSize (.getReferenceManager ^ArrowBuf buffer)))) (t/is (zero? (.getAccountedSize (.getReferenceManager ^ArrowBuf buffer)))) - (t/is (= 1 (count (.buffers ^BufferPool bp))))))))))) + (t/is (= 0 (count (.buffers ^BufferPool bp))))))))))) (t/deftest temporal-watermark-is-immutable-567 (with-open [node (node/start-node {})] @@ -421,8 +421,6 @@ (select-keys [:latest-completed-tx :latest-row-id])))) (let [objs (.listObjects os)] - (t/is (= 4 (count (filter #(re-matches #"^chunk-\p{XDigit}+/temporal\.arrow$" %) objs)))) - (t/is (= 4 (count (filter #(re-matches #"temporal-snapshots/\p{XDigit}+.*" %) objs)))) (t/is (= 1 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_info/metadata\.arrow" %) objs)))) (t/is (= 4 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_readings/metadata\.arrow" %) objs)))) (t/is (= 1 (count (filter #(re-matches #"chunk-.*/device_info/content-api_version\.arrow" %) objs)))) @@ -458,12 +456,7 @@ (t/is (= last-tx-key (tu/then-await-tx last-tx-key node (Duration/ofSeconds 60)))) (t/is (= last-tx-key (tu/latest-completed-tx node))) - (Thread/sleep 1000) ;; TODO for now - (tu/await-temporal-snapshot-build node) - (let [objs (.listObjects os)] - (t/is (= 11 (count (filter #(re-matches #"chunk-\p{XDigit}+/temporal\.arrow" %) objs)))) - (t/is (= 11 (count (filter #(re-matches #"temporal-snapshots/\p{XDigit}+.arrow" %) objs)))) (t/is (= 13 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_(?:info|readings)/metadata.arrow" %) objs)))) (t/is (= 2 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_info/content-api_version\.arrow" %) objs)))) (t/is (= 11 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_readings/content-battery_level\.arrow" %) objs))))))))))) @@ -509,8 +502,6 @@ (t/is (< latest-row-id (count first-half-tx-ops))) (let [objs (.listObjects os)] - (t/is (= 5 (count (filter #(re-matches #"^chunk-\p{XDigit}+/temporal\.arrow$" %) objs)))) - (t/is (= 5 (count (filter #(re-matches #"temporal-snapshots/\p{XDigit}+.*" %) objs)))) (t/is (= 2 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_info/metadata\.arrow" %) objs)))) (t/is (= 5 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_readings/metadata\.arrow" %) objs)))) (t/is (= 2 (count (filter #(re-matches #"chunk-.*/device_info/content-api_version\.arrow" %) objs)))) @@ -549,16 +540,11 @@ (tu/then-await-tx node (Duration/ofSeconds 15))))) (t/is (= second-half-tx-key (tu/latest-completed-tx node)))) - (Thread/sleep 1000) ;; TODO for now - (tu/await-temporal-snapshot-build node) - (doseq [^Node node [new-node node] :let [^ObjectStore os (tu/component node ::os/file-system-object-store) ^IMetadataManager mm (tu/component node ::meta/metadata-manager)]] (let [objs (.listObjects os)] - (t/is (= 11 (count (filter #(re-matches #"^chunk-\p{XDigit}+/temporal\.arrow$" %) objs)))) - (t/is (= 11 (count (filter #(re-matches #"temporal-snapshots/\p{XDigit}+.*" %) objs)))) (t/is (= 2 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_info/metadata\.arrow" %) objs)))) (t/is (= 11 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_readings/metadata\.arrow" %) objs)))) (t/is (= 2 (count (filter #(re-matches #"chunk-.*/device_info/content-api_version\.arrow" %) objs)))) diff --git a/src/test/clojure/xtdb/operator/scan_test.clj b/src/test/clojure/xtdb/operator/scan_test.clj index 5debf258be..762284db29 100644 --- a/src/test/clojure/xtdb/operator/scan_test.clj +++ b/src/test/clojure/xtdb/operator/scan_test.clj @@ -3,7 +3,6 @@ [xtdb.api :as xt] [xtdb.node :as node] [xtdb.operator :as op] - [xtdb.operator.scan :as scan] [xtdb.test-util :as tu] [xtdb.util :as util]) (:import xtdb.operator.IRaQuerySource)) @@ -285,6 +284,8 @@ (t/is (= '{xt/id [:union #{:keyword :utf8}]} (->col-types tx)))))))) +;; TODO adapt for scan/->temporal-range +#_ (t/deftest can-create-temporal-min-max-range (let [μs-2018 (util/instant->micros (util/->instant #inst "2018")) μs-2019 (util/instant->micros (util/->instant #inst "2019"))] diff --git a/src/test/clojure/xtdb/temporal/histogram_test.clj b/src/test/clojure/xtdb/temporal/histogram_test.clj deleted file mode 100644 index 62dd7ed6fc..0000000000 --- a/src/test/clojure/xtdb/temporal/histogram_test.clj +++ /dev/null @@ -1,51 +0,0 @@ -(ns xtdb.temporal.histogram-test - (:require [clojure.test :as t] - [xtdb.temporal.histogram :as hist]) - (:import [xtdb.temporal.histogram IBin IHistogram])) - -(defn- bins->seq [^IHistogram h] - (for [^IBin b (.getBins h)] - [(.getValue b) (.getCount b)])) - -;; See Appendix A. -;; "A Streaming Parallel Decision Tree Algorithm" -;; https://www.jmlr.org/papers/volume11/ben-haim10a/ben-haim10a.pdf - -(t/deftest example-histogram - (let [^IHistogram h (hist/->histogram 5)] - (doseq [v [23 19 10 16 36]] - (.update h v)) - - (t/is (= [[10.0 1] [16.0 1] [19.0 1] [23.0 1] [36.0 1]] - (bins->seq h))) - (t/is (= 5 (.getTotal h))) - (t/is (= 10.0 (.getMin h))) - (t/is (= 36.0 (.getMax h))) - - (.update h 2) - - (t/is (= [[2.0 1] [10.0 1] [17.5 2] [23.0 1] [36.0 1]] - (bins->seq h))) - (t/is (= 6 (.getTotal h))) - - (.update h 9) - - (t/is (= [[2.0 1] [9.5 2] [17.5 2] [23.0 1] [36.0 1]] - (bins->seq h))) - - (doseq [v [32 30 45]] - (.update h v)) - - (t/is (= [[2.0 1] [9.5 2] [19.333333333333332 3] [32.666666666666664 3] [45.0 1]] - (bins->seq h))) - - (t/is (= 3.275064636598679 (.sum h 15))) - (t/is (= 0.0 (.sum h 1))) - (t/is (= 10.0 (.sum h 45))) - - ;; NOTE: this is a bit off from paper. - (t/is (= [15.222890825137508 28.96296296296297] (vec (.uniform h 3)))) - - (t/is (= 10 (.getTotal h))) - (t/is (= 2.0 (.getMin h))) - (t/is (= 45.0 (.getMax h))))) diff --git a/src/test/clojure/xtdb/temporal/kd_tree_microbench_test.clj b/src/test/clojure/xtdb/temporal/kd_tree_microbench_test.clj deleted file mode 100644 index f51194b71c..0000000000 --- a/src/test/clojure/xtdb/temporal/kd_tree_microbench_test.clj +++ /dev/null @@ -1,109 +0,0 @@ -(ns xtdb.temporal.kd-tree-microbench-test - (:require [clojure.test :as t] - [xtdb.util :as util] - [xtdb.temporal.kd-tree :as kd] - [xtdb.temporal.grid :as grid]) - (:import [java.util Collection HashMap Random] - java.util.function.LongSupplier - [java.util.stream LongStream] - java.io.Closeable - [org.apache.arrow.memory RootAllocator])) - -(deftype ZipfRejectionSampler [^Random rng ^long n ^double skew ^double t] - LongSupplier - (getAsLong [_] - (loop [] - (let [b (.nextDouble rng) - inv-b (if (<= (* b t) 1) - (* b t) - (Math/pow (+ (* (* b t) (- 1 skew)) skew) (/ 1 (- 1 skew)))) - sample-x (long (inc inv-b)) - y-rand (.nextDouble rng) - ratio-top (Math/pow sample-x (- skew)) - ratio-bottom (if (<= sample-x 1) - (/ 1 t) - (/ (Math/pow inv-b (- skew)) t)) - ratio (/ ratio-top (* ratio-bottom t))] - (if (< y-rand ratio) - sample-x - (recur)))))) - -(defn- ->zipf-rejection-sampler ^java.util.function.LongSupplier [^Random rng ^double n ^double skew] - (let [t (/ (- (Math/pow n (- 1 skew)) skew) (- 1 skew))] - (ZipfRejectionSampler. rng n skew t))) - -;; TODO: move to JMH. -(t/deftest ^:integration kd-tree-micro-bench - (with-open [allocator (RootAllocator.)] - (doseq [k (range 2 4) - d [:random :zipf]] - (let [rng (Random. 0) - ns 1000000 - qs 100 - next-long-fn (case d - :random #(.nextLong rng) - :zipf (let [^LongSupplier z (->zipf-rejection-sampler rng ns 0.7)] - #(* Long/MAX_VALUE (- (/ (.getAsLong ^LongSupplier z) ns) 0.5)))) - _ (prn :k k) - _ (prn :distribution d) - ts 3 - _ (prn :gen-points ns) - points (time - (->> (repeatedly #(vec (repeatedly k next-long-fn))) - (distinct) - (take ns) - (mapv long-array))) - - _ (prn :gen-queries qs) - queries (time - (vec (for [n (range qs) - :let [min+max-pairs (repeatedly k #(sort [(next-long-fn) - (next-long-fn)]))]] - [n - (long-array (map first min+max-pairs)) - (long-array (map second min+max-pairs))]))) - query->count (HashMap.) - test-dir (util/->path "target/kd-tree-micro-bench")] - (util/delete-dir test-dir) - - (prn :range-queries-scan qs) - (time - (doseq [[query-id ^longs min-range ^longs max-range] queries] - (.put query->count query-id (.count ^LongStream (kd/kd-tree-range-search points min-range max-range))))) - - (prn :average-match-ratio (double (/ (/ (reduce + (vals query->count)) qs) ns))) - - (prn :build-grid) - (with-open [^Closeable grid (time - (->> (grid/->disk-grid allocator (.resolve test-dir (format "grid_%d.arrow" k)) points {:k k}) - (grid/->mmap-grid allocator)))] - (prn :range-queries-grid qs) - (dotimes [_ ts] - (time - (doseq [[query-id min-range max-range] queries] - (t/is (= (.get query->count query-id) - (.count ^LongStream (kd/kd-tree-range-search grid min-range max-range))))))) - - - (prn :build-node-kd-tree ns) - (with-open [^Closeable node-kd-tree (time - (reduce - (fn [acc point] - (kd/kd-tree-insert acc allocator point)) - (kd/->node-kd-tree allocator k ) - points))] - - (prn :range-queries-node-kd-tree qs) - (dotimes [_ ts] - (time - (doseq [[query-id min-range max-range] queries] - (t/is (= (.get query->count query-id) - (.count ^LongStream (kd/kd-tree-range-search node-kd-tree min-range max-range))))))) - - - (let [_ (prn :grid->seq) - grid-seq (time (set (kd/kd-tree->seq grid))) - _ (prn :node-kd-tree->seq) - node-kd-tree-seq (time (set (kd/kd-tree->seq node-kd-tree)))] - - (t/is (= grid-seq node-kd-tree-seq))))))))) diff --git a/src/test/clojure/xtdb/temporal/kd_tree_test.clj b/src/test/clojure/xtdb/temporal/kd_tree_test.clj deleted file mode 100644 index f1c6eb5cd6..0000000000 --- a/src/test/clojure/xtdb/temporal/kd_tree_test.clj +++ /dev/null @@ -1,476 +0,0 @@ -(ns xtdb.temporal.kd-tree-test - (:require [clojure.test :as t] - [xtdb.temporal :as temporal] - [xtdb.temporal.kd-tree :as kd] - [xtdb.util :as util]) - (:import xtdb.temporal.TemporalCoordinates - java.io.Closeable - [java.util Date HashMap List] - org.apache.arrow.memory.RootAllocator)) - -;; NOTE: "Developing Time-Oriented Database Applications in SQL", -;; chapter 10 "Bitemporal Tables". - -;; Uses transaction time splitting, so some rectangles differ, but -;; areas covered are the same. Could or maybe should coalesce. - -(defn- ->row-map [^List point] - (zipmap [:id :row-id :app-time-start :app-time-end :system-time-start :system-time-end] - [(.get point temporal/id-idx) - (.get point temporal/row-id-idx) - (Date/from (util/micros->instant (.get point temporal/app-time-start-idx))) - (Date/from (util/micros->instant (.get point temporal/app-time-end-idx))) - (Date/from (util/micros->instant (.get point temporal/system-time-start-idx))) - (Date/from (util/micros->instant (.get point temporal/system-time-end-idx)))])) - -(defn- temporal-rows [kd-tree row-id->row] - (vec (for [{:keys [row-id] :as row} (->> (map ->row-map (kd/kd-tree->seq kd-tree)) - (sort-by (juxt :system-time-start :row-id)))] - (merge row (get row-id->row row-id))))) - -(defn ->coordinates ^xtdb.temporal.TemporalCoordinates [{:keys [id - ^long row-id - system-time-start - system-time-end - app-time-start - app-time-end - new-entity? tombstone?]}] - (TemporalCoordinates. row-id id - (util/instant->micros (if (instance? Date system-time-start) - (.toInstant ^Date system-time-start) - system-time-start)) - (util/instant->micros (or (if (instance? Date system-time-end) - (some-> ^Date system-time-end .toInstant) - system-time-end) - util/end-of-time)) - (util/instant->micros (let [ats (or app-time-start system-time-start)] - (if (instance? Date ats) - (.toInstant ^Date ats) - ats))) - (util/instant->micros (or (if (instance? Date app-time-end) - (some-> ^Date app-time-end .toInstant) - app-time-end) - util/end-of-time)) - new-entity? (boolean tombstone?))) - - (defn as-micros [^java.util.Date inst] - (util/instant->micros (.toInstant inst))) - -(t/deftest bitemporal-system-time-split-test - (let [kd-tree nil - row-id->row (HashMap.) - !current-row-ids (volatile! #{})] - ;; Current Insert - ;; Eva Nielsen buys the flat at Skovvej 30 in Aalborg on January 10, - ;; 1998. - (with-open [allocator (RootAllocator.) - ^Closeable kd-tree (let [system-time #inst "1998-01-10"] - (temporal/insert-coordinates kd-tree - allocator - (->coordinates {:id 7797 - :row-id 1 - :system-time-start system-time - :new-entity? true}) - !current-row-ids - (as-micros system-time)))] - (.put row-id->row 1 {:customer-number 145}) - (t/is (= [{:id 7797, - :customer-number 145, - :row-id 1, - :app-time-start #inst "1998-01-10T00:00:00.000-00:00", - :app-time-end #inst "9999-12-31T23:59:59.999-00:00", - :system-time-start #inst "1998-01-10T00:00:00.000-00:00", - :system-time-end #inst "9999-12-31T23:59:59.999-00:00"}] - (temporal-rows kd-tree row-id->row))) - (t/is (= #{1} - @!current-row-ids)) - - ;; Current Update - ;; Peter Olsen buys the flat on January 15, 1998. - (let [system-time #inst "1998-01-15" - kd-tree (temporal/insert-coordinates kd-tree - allocator - (->coordinates {:id 7797 - :row-id 2 - :system-time-start system-time - :new-entity? false}) - !current-row-ids - (as-micros system-time))] - (.put row-id->row 2 {:customer-number 827}) - (t/is (= [{:id 7797, - :row-id 1, - :customer-number 145, - :app-time-start #inst "1998-01-10T00:00:00.000-00:00", - :app-time-end #inst "9999-12-31T23:59:59.999-00:00", - :system-time-start #inst "1998-01-10T00:00:00.000-00:00", - :system-time-end #inst "1998-01-15T00:00:00.000-00:00"} - {:id 7797, - :customer-number 145, - :row-id 1, - :app-time-start #inst "1998-01-10T00:00:00.000-00:00", - :app-time-end #inst "1998-01-15T00:00:00.000-00:00", - :system-time-start #inst "1998-01-15T00:00:00.000-00:00", - :system-time-end #inst "9999-12-31T23:59:59.999-00:00"} - {:id 7797, - :row-id 2, - :customer-number 827, - :app-time-start #inst "1998-01-15T00:00:00.000-00:00", - :app-time-end #inst "9999-12-31T23:59:59.999-00:00", - :system-time-start #inst "1998-01-15T00:00:00.000-00:00", - :system-time-end #inst "9999-12-31T23:59:59.999-00:00"}] - (temporal-rows kd-tree row-id->row))) - (t/is (= #{2} - @!current-row-ids)) - - ;; Current Delete - ;; Peter Olsen sells the flat on January 20, 1998. - (let [system-time #inst "1998-01-20" - kd-tree (temporal/insert-coordinates kd-tree - allocator - (->coordinates {:id 7797 - :row-id 3 - :system-time-start system-time - :new-entity? false - :tombstone? true}) - !current-row-ids - (as-micros system-time))] - (.put row-id->row 3 {:customer-number 827}) - (t/is (= [{:id 7797, - :customer-number 145, - :row-id 1, - :app-time-start #inst "1998-01-10T00:00:00.000-00:00", - :app-time-end #inst "9999-12-31T23:59:59.999-00:00", - :system-time-start #inst "1998-01-10T00:00:00.000-00:00", - :system-time-end #inst "1998-01-15T00:00:00.000-00:00"} - {:id 7797, - :customer-number 145, - :row-id 1, - :app-time-start #inst "1998-01-10T00:00:00.000-00:00", - :app-time-end #inst "1998-01-15T00:00:00.000-00:00", - :system-time-start #inst "1998-01-15T00:00:00.000-00:00", - :system-time-end #inst "9999-12-31T23:59:59.999-00:00"} - {:id 7797, - :customer-number 827, - :row-id 2, - :app-time-start #inst "1998-01-15T00:00:00.000-00:00", - :app-time-end #inst "9999-12-31T23:59:59.999-00:00", - :system-time-start #inst "1998-01-15T00:00:00.000-00:00", - :system-time-end #inst "1998-01-20T00:00:00.000-00:00"} - {:id 7797, - :customer-number 827, - :row-id 2, - :app-time-start #inst "1998-01-15T00:00:00.000-00:00", - :app-time-end #inst "1998-01-20T00:00:00.000-00:00", - :system-time-start #inst "1998-01-20T00:00:00.000-00:00", - :system-time-end #inst "9999-12-31T23:59:59.999-00:00"}] - (temporal-rows kd-tree row-id->row))) - (t/is (= #{} - @!current-row-ids)) - - ;; Sequenced Insert - ;; Eva actually purchased the flat on January 3, performed on January 23. - (let [system-time #inst "1998-01-23" - kd-tree (temporal/insert-coordinates kd-tree - allocator - (->coordinates {:id 7797 - :row-id 4 - :system-time-start system-time - :app-time-start #inst "1998-01-03" - :app-time-end #inst "1998-01-15" - :new-entity? false}) - !current-row-ids - (as-micros system-time))] - (.put row-id->row 4 {:customer-number 145}) - (t/is (= [{:id 7797, - :customer-number 145, - :row-id 1, - :app-time-start #inst "1998-01-10T00:00:00.000-00:00", - :app-time-end #inst "9999-12-31T23:59:59.999-00:00", - :system-time-start #inst "1998-01-10T00:00:00.000-00:00", - :system-time-end #inst "1998-01-15T00:00:00.000-00:00"} - {:id 7797, - :customer-number 145, - :row-id 1, - :app-time-start #inst "1998-01-10T00:00:00.000-00:00", - :app-time-end #inst "1998-01-15T00:00:00.000-00:00", - :system-time-start #inst "1998-01-15T00:00:00.000-00:00", - :system-time-end #inst "1998-01-23T00:00:00.000-00:00"} - {:id 7797, - :customer-number 827, - :row-id 2, - :app-time-start #inst "1998-01-15T00:00:00.000-00:00", - :app-time-end #inst "9999-12-31T23:59:59.999-00:00", - :system-time-start #inst "1998-01-15T00:00:00.000-00:00", - :system-time-end #inst "1998-01-20T00:00:00.000-00:00"} - {:id 7797, - :row-id 2, - :customer-number 827, - :app-time-start #inst "1998-01-15T00:00:00.000-00:00", - :app-time-end #inst "1998-01-20T00:00:00.000-00:00", - :system-time-start #inst "1998-01-20T00:00:00.000-00:00", - :system-time-end #inst "9999-12-31T23:59:59.999-00:00"} - {:id 7797, - :customer-number 145, - :row-id 4, - :app-time-start #inst "1998-01-03T00:00:00.000-00:00", - :app-time-end #inst "1998-01-15T00:00:00.000-00:00", - :system-time-start #inst "1998-01-23T00:00:00.000-00:00", - :system-time-end #inst "9999-12-31T23:59:59.999-00:00"}] - (temporal-rows kd-tree row-id->row))) - (t/is (= #{} - @!current-row-ids)) - - ;; NOTE: rows differs from book, but covered area is the same. - ;; Sequenced Delete - ;; A sequenced deletion performed on January 26: Eva actually purchased the flat on January 5. - (let [system-time #inst "1998-01-26" - kd-tree (temporal/insert-coordinates kd-tree - allocator - (->coordinates {:id 7797 - :row-id 5 - :system-time-start system-time - :app-time-start #inst "1998-01-02" - :app-time-end #inst "1998-01-05" - :new-entity? false - :tombstone? true}) - !current-row-ids - (as-micros system-time))] - (.put row-id->row 5 {:customer-number 145}) - (t/is (= [{:id 7797, - :customer-number 145, - :row-id 1, - :app-time-start #inst "1998-01-10T00:00:00.000-00:00", - :app-time-end #inst "9999-12-31T23:59:59.999-00:00", - :system-time-start #inst "1998-01-10T00:00:00.000-00:00", - :system-time-end #inst "1998-01-15T00:00:00.000-00:00"} - {:id 7797, - :customer-number 145, - :row-id 1, - :app-time-start #inst "1998-01-10T00:00:00.000-00:00", - :app-time-end #inst "1998-01-15T00:00:00.000-00:00", - :system-time-start #inst "1998-01-15T00:00:00.000-00:00", - :system-time-end #inst "1998-01-23T00:00:00.000-00:00"} - {:id 7797, - :customer-number 827, - :row-id 2, - :app-time-start #inst "1998-01-15T00:00:00.000-00:00", - :app-time-end #inst "9999-12-31T23:59:59.999-00:00", - :system-time-start #inst "1998-01-15T00:00:00.000-00:00", - :system-time-end #inst "1998-01-20T00:00:00.000-00:00"} - {:id 7797, - :customer-number 827, - :row-id 2, - :app-time-start #inst "1998-01-15T00:00:00.000-00:00", - :app-time-end #inst "1998-01-20T00:00:00.000-00:00", - :system-time-start #inst "1998-01-20T00:00:00.000-00:00", - :system-time-end #inst "9999-12-31T23:59:59.999-00:00"} - {:id 7797, - :customer-number 145, - :row-id 4, - :app-time-start #inst "1998-01-03T00:00:00.000-00:00", - :app-time-end #inst "1998-01-15T00:00:00.000-00:00", - :system-time-start #inst "1998-01-23T00:00:00.000-00:00", - :system-time-end #inst "1998-01-26T00:00:00.000-00:00"} - {:id 7797, - :customer-number 145, - :row-id 4, - :app-time-start #inst "1998-01-05T00:00:00.000-00:00", - :app-time-end #inst "1998-01-15T00:00:00.000-00:00", - :system-time-start #inst "1998-01-26T00:00:00.000-00:00", - :system-time-end #inst "9999-12-31T23:59:59.999-00:00"}] - (temporal-rows kd-tree row-id->row))) - (t/is (= #{} - @!current-row-ids)) - - ;; NOTE: rows differs from book, but covered area is the same. - ;; Sequenced Update - ;; A sequenced update performed on January 28: Peter actually purchased the flat on January 12. - (let [system-time #inst "1998-01-28" - kd-tree (temporal/insert-coordinates kd-tree - allocator - (->coordinates {:id 7797 - :row-id 6 - :system-time-start system-time - :app-time-start #inst "1998-01-12" - :app-time-end #inst "1998-01-15" - :new-entity? false}) - !current-row-ids - (as-micros system-time))] - (.put row-id->row 6 {:customer-number 827}) - (t/is (= [{:id 7797, - :customer-number 145, - :row-id 1, - :app-time-start #inst "1998-01-10T00:00:00.000-00:00", - :app-time-end #inst "9999-12-31T23:59:59.999-00:00", - :system-time-start #inst "1998-01-10T00:00:00.000-00:00", - :system-time-end #inst "1998-01-15T00:00:00.000-00:00"} - {:id 7797, - :customer-number 145, - :row-id 1, - :app-time-start #inst "1998-01-10T00:00:00.000-00:00", - :app-time-end #inst "1998-01-15T00:00:00.000-00:00", - :system-time-start #inst "1998-01-15T00:00:00.000-00:00", - :system-time-end #inst "1998-01-23T00:00:00.000-00:00"} - {:id 7797, - :customer-number 827, - :row-id 2, - :app-time-start #inst "1998-01-15T00:00:00.000-00:00", - :app-time-end #inst "9999-12-31T23:59:59.999-00:00", - :system-time-start #inst "1998-01-15T00:00:00.000-00:00", - :system-time-end #inst "1998-01-20T00:00:00.000-00:00"} - {:id 7797, - :customer-number 827, - :row-id 2, - :app-time-start #inst "1998-01-15T00:00:00.000-00:00", - :app-time-end #inst "1998-01-20T00:00:00.000-00:00", - :system-time-start #inst "1998-01-20T00:00:00.000-00:00", - :system-time-end #inst "9999-12-31T23:59:59.999-00:00"} - {:id 7797, - :customer-number 145, - :row-id 4, - :app-time-start #inst "1998-01-03T00:00:00.000-00:00", - :app-time-end #inst "1998-01-15T00:00:00.000-00:00", - :system-time-start #inst "1998-01-23T00:00:00.000-00:00", - :system-time-end #inst "1998-01-26T00:00:00.000-00:00"} - {:id 7797, - :customer-number 145, - :row-id 4, - :app-time-start #inst "1998-01-05T00:00:00.000-00:00", - :app-time-end #inst "1998-01-15T00:00:00.000-00:00", - :system-time-start #inst "1998-01-26T00:00:00.000-00:00", - :system-time-end #inst "1998-01-28T00:00:00.000-00:00"} - {:id 7797, - :customer-number 145, - :row-id 4, - :app-time-start #inst "1998-01-05T00:00:00.000-00:00", - :app-time-end #inst "1998-01-12T00:00:00.000-00:00", - :system-time-start #inst "1998-01-28T00:00:00.000-00:00", - :system-time-end #inst "9999-12-31T23:59:59.999-00:00"} - {:id 7797, - :customer-number 827, - :row-id 6, - :app-time-start #inst "1998-01-12T00:00:00.000-00:00", - :app-time-end #inst "1998-01-15T00:00:00.000-00:00", - :system-time-start #inst "1998-01-28T00:00:00.000-00:00", - :system-time-end #inst "9999-12-31T23:59:59.999-00:00"}] - (temporal-rows kd-tree row-id->row))) - (t/is (= #{} - @!current-row-ids)) - - - (t/testing "rebuilding tree results in tree with same points" - (let [points (mapv vec (kd/kd-tree->seq kd-tree))] - (with-open [rebuilt-tree (kd/build-node-kd-tree allocator (shuffle points))] - (t/is (= (sort points) - (sort (mapv vec (kd/kd-tree->seq rebuilt-tree)))))))))))))))) - -(t/deftest kd-tree-sanity-check - (let [points [[7 2] [5 4] [9 6] [4 7] [8 1] [2 3]]] - (with-open [allocator (RootAllocator.) - insert-kd-tree (kd/build-node-kd-tree allocator points)] - (t/is (= #{[7 2] [5 4] [2 3] [8 1]} - - (-> insert-kd-tree - (kd/kd-tree-range-search [0 0] [8 4]) - (->> (kd/kd-tree->seq insert-kd-tree) - (map vec) (set)))) - "wikipedia-test") - - (t/testing "seq" - (t/is (= (set points) - (->> (kd/kd-tree->seq insert-kd-tree) - (map vec) (set))))) - - (t/testing "height" - (t/is (= 0 - (kd/kd-tree-height insert-kd-tree)))) - - (t/testing "size" - (t/is (= (count points) - (kd/kd-tree-size insert-kd-tree)))) - - (t/testing "empty tree" - (with-open [^Closeable kd-tree (kd/->node-kd-tree allocator 2)] - (t/is (zero? (kd/kd-tree-size kd-tree)))) - - (with-open [^Closeable kd-tree (kd/kd-tree-insert nil allocator [1 2])] - (t/is (= [[1 2]] (kd/kd-tree->seq kd-tree)))) - - (with-open [^Closeable kd-tree (kd/kd-tree-delete nil allocator [1 2])] - (t/is (empty? (kd/kd-tree->seq kd-tree))) - (t/is (zero? (kd/kd-tree-size kd-tree))))) - - (t/testing "merge" - (with-open [new-tree-with-tombstone (kd/build-node-kd-tree allocator [[4 7] [8 1] [2 3]])] - (let [node-to-delete [2 1] - new-tree-with-tombstone (kd/kd-tree-delete new-tree-with-tombstone allocator node-to-delete)] - (t/is (= 3 (kd/kd-tree-size new-tree-with-tombstone))) - (t/is (= Long/SIZE (kd/kd-tree-value-count new-tree-with-tombstone))) - (with-open [old-tree-with-node-to-be-deleted (kd/build-node-kd-tree allocator [[7 2] [5 4] [9 6] node-to-delete]) - merged-tree (kd/merge-kd-trees allocator old-tree-with-node-to-be-deleted new-tree-with-tombstone) - rebuilt-tree (kd/build-node-kd-tree allocator merged-tree)] - (t/is (= 4 (kd/kd-tree-size old-tree-with-node-to-be-deleted))) - - (t/is (= (set (kd/kd-tree->seq insert-kd-tree)) - (set (kd/kd-tree->seq merged-tree)) - (set (kd/kd-tree->seq rebuilt-tree)))) - - (t/is (= (kd/kd-tree-size insert-kd-tree) - (kd/kd-tree-size merged-tree) - (kd/kd-tree-size rebuilt-tree))) - - (t/testing "merged tree" - (with-open [dynamic-tree (kd/build-node-kd-tree allocator [[4 7] [8 1] [2 3]]) - static-tree (kd/build-node-kd-tree allocator [[7 2] [5 4] [9 6] node-to-delete]) - merged-tree (kd/->merged-kd-tree static-tree dynamic-tree)] - (t/is (= 3 (kd/kd-tree-size dynamic-tree))) - (t/is (= 4 (kd/kd-tree-size static-tree))) - (t/is (= 7 (kd/kd-tree-size merged-tree))) - (t/is (zero? (kd/kd-tree-height merged-tree))) - - (let [unknown-node [0 0] - expected-nodes (set (map vec (kd/kd-tree->seq rebuilt-tree))) - merged-tree (kd/kd-tree-delete merged-tree allocator node-to-delete)] - (t/is (= 6 (kd/kd-tree-size merged-tree))) - (t/is (= 6 (kd/kd-tree-size (kd/kd-tree-delete merged-tree allocator node-to-delete)))) - (t/is (= 6 (kd/kd-tree-size (kd/kd-tree-delete merged-tree allocator unknown-node)))) - (t/is (= expected-nodes (set (map vec (kd/kd-tree->seq merged-tree))))) - - (let [node-to-insert [10 10] - merged-tree (kd/kd-tree-insert merged-tree allocator node-to-insert)] - (t/is (= 7 (kd/kd-tree-size merged-tree))) - (t/is (= (conj expected-nodes node-to-insert) - (set (map vec (kd/kd-tree->seq merged-tree))))) - - (let [merged-tree (kd/kd-tree-delete merged-tree allocator node-to-insert)] - (t/is (= 6 (kd/kd-tree-size merged-tree))) - (t/is (= expected-nodes (set (map vec (kd/kd-tree->seq merged-tree)))))))) - - - (t/testing "layered merged tree" - (let [node-to-insert [10 10] - expected-nodes (set (map vec (kd/kd-tree->seq static-tree)))] - (with-open [^Closeable delta-tree (-> nil - (kd/kd-tree-insert allocator node-to-insert) - (kd/kd-tree-delete allocator node-to-delete)) - static-delta-tree (kd/build-node-kd-tree allocator delta-tree) - merged-tree (kd/->merged-kd-tree (kd/kd-tree-retain static-tree allocator) static-delta-tree)] - (t/is (= 1 (kd/kd-tree-size static-delta-tree))) - (t/is (= Long/SIZE (kd/kd-tree-value-count static-delta-tree))) - - (t/is (= 4 (kd/kd-tree-size merged-tree))) - (t/is (= (* 2 Long/SIZE) (kd/kd-tree-value-count merged-tree))) - - (t/is (= (-> expected-nodes (conj node-to-insert) (disj node-to-delete)) - (set (map vec (kd/kd-tree->seq merged-tree)))))))) - - (t/testing "empty dynamic tree" - (let [node-to-insert [10 10]] - (with-open [merged-tree (kd/->merged-kd-tree (kd/kd-tree-retain static-tree allocator))] - (t/is (= 4 (kd/kd-tree-size merged-tree))) - (t/is (empty? (kd/kd-tree->seq merged-tree (kd/kd-tree-range-search merged-tree node-to-insert node-to-insert)))) - (with-open [^Closeable merged-tree (kd/kd-tree-insert merged-tree allocator node-to-insert)] - (t/is (= 5 (kd/kd-tree-size merged-tree))) - (t/is (= 4 (kd/kd-tree-size static-tree))) - (t/is (= [node-to-insert] - (kd/kd-tree->seq merged-tree (kd/kd-tree-range-search merged-tree node-to-insert node-to-insert))))))))))))))))) diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/temporal.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/temporal.arrow.json deleted file mode 100644 index d127296bb8..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/temporal.arrow.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "point", - "nullable" : false, - "type" : { - "name" : "fixedsizelist", - "listSize" : 6 - }, - "children" : [{ - "name" : "coordinates", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }], - "metadata" : [{ - "value" : "{\"maxs\":[253402300800000000,360287970189639680,1577923200000000,5,1577923200000000],\"value-count\":16,\"axis-shift\":0,\"mins\":[253402300800000000,0,1577836800000000,0,1577836800000000],\"deletes?\":true,\"scales\":[null,null,null,null,null],\"k-minus-one-slope+base\":[0.0,-0.0],\"size\":6,\"k\":6,\"cell-shift\":3}", - "key" : "grid-meta" - }] - }, - "batches" : [{ - "count" : 6, - "columns" : [{ - "name" : "point", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "children" : [{ - "name" : "coordinates", - "count" : 36, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["253402300799999999","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","144115188075855872","1577836800000000","2","1577836800000000","253402300799999999","253402300799999999","216172782113783808","1577923200000000","3","1577923200000000","253402300799999999","253402300799999999","288230376151711744","1577923200000000","4","1577923200000000","253402300799999999","253402300799999999","360287970189639680","1577923200000000","5","1577923200000000","253402300799999999"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/temporal-snapshots/00.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/temporal-snapshots/00.arrow.json deleted file mode 100644 index ab3714f088..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/temporal-snapshots/00.arrow.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "point", - "nullable" : false, - "type" : { - "name" : "fixedsizelist", - "listSize" : 6 - }, - "children" : [{ - "name" : "coordinates", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }], - "metadata" : [{ - "value" : "{\"maxs\":[253402300800000000,360287970189639680,1577923200000000,5,1577923200000000],\"value-count\":16,\"axis-shift\":0,\"mins\":[253402300800000000,0,1577836800000000,0,1577836800000000],\"deletes?\":false,\"scales\":[null,null,null,null,null],\"k-minus-one-slope+base\":[0.0,-0.0],\"size\":6,\"k\":6,\"cell-shift\":3}", - "key" : "grid-meta" - }] - }, - "batches" : [{ - "count" : 6, - "columns" : [{ - "name" : "point", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "children" : [{ - "name" : "coordinates", - "count" : 36, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["253402300799999999","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","144115188075855872","1577836800000000","2","1577836800000000","253402300799999999","253402300799999999","216172782113783808","1577923200000000","3","1577923200000000","253402300799999999","253402300799999999","288230376151711744","1577923200000000","4","1577923200000000","253402300799999999","253402300799999999","360287970189639680","1577923200000000","5","1577923200000000","253402300799999999"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-_row_id.arrow.json index 27c3305b7d..3b6a94f68f 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-_row_id.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-_row_id.arrow.json @@ -17,7 +17,7 @@ "name" : "_row_id", "count" : 3, "VALIDITY" : [1,1,1], - "DATA" : ["8","9","11"] + "DATA" : ["8","9","10"] }] }] } \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/metadata.arrow.json index 9b87796e81..3844c65e4d 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/metadata.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/metadata.arrow.json @@ -164,7 +164,7 @@ "name" : "max", "count" : 8, "VALIDITY" : [1,1,1,0,1,1,1,0], - "DATA" : ["11","2","2","0","11","2","2","0"] + "DATA" : ["10","2","2","0","10","2","2","0"] }] },{ "name" : "utf8", @@ -194,7 +194,7 @@ "count" : 8, "VALIDITY" : [1,1,1,1,1,1,1,1], "OFFSET" : [0,82,134,186,238,320,372,424,476], - "DATA" : ["3a30000007000000000002000500000007000000090000000a0000000b0000000c0000004000000046000000480000004a0000004c0000004e00000050000000080009000b00bc2bb4c921c36f57dee496e1","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a3000000400000000000200040000000a0000000f000000280000002e00000030000000320000001f003e005d00746f44e95cac","3a30000007000000000002000500000007000000090000000a0000000b0000000c0000004000000046000000480000004a0000004c0000004e00000050000000080009000b00bc2bb4c921c36f57dee496e1","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a3000000400000000000200040000000a0000000f000000280000002e00000030000000320000001f003e005d00746f44e95cac"] + "DATA" : ["3a30000007000000000002000500000007000000080000000a0000000b0000000c0000004000000046000000480000004a0000004c0000004e00000050000000080009000a00bc2bb4c910296f57dee48d14","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a3000000400000000000200040000000a0000000f000000280000002e00000030000000320000001f003e005d00746f44e95cac","3a30000007000000000002000500000007000000080000000a0000000b0000000c0000004000000046000000480000004a0000004c0000004e00000050000000080009000a00bc2bb4c910296f57dee48d14","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a3000000400000000000200040000000a0000000f000000280000002e00000030000000320000001f003e005d00746f44e95cac"] }] }] }] diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/temporal.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/temporal.arrow.json deleted file mode 100644 index 8e7339e7ab..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/temporal.arrow.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "point", - "nullable" : false, - "type" : { - "name" : "fixedsizelist", - "listSize" : 6 - }, - "children" : [{ - "name" : "coordinates", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }], - "metadata" : [{ - "value" : "{\"maxs\":[253402300800000000,1008806316530991104,1578268800000000,14,1578268800000000],\"value-count\":64,\"axis-shift\":0,\"mins\":[1577923200000000,0,1577836800000000,0,1577836800000000],\"deletes?\":true,\"scales\":[null,null,null,null,null],\"k-minus-one-slope+base\":[7.147838573671113E-17,-0.11278740315250559],\"size\":10,\"k\":6,\"cell-shift\":5}", - "key" : "grid-meta" - }] - }, - "batches" : [{ - "count" : 18, - "columns" : [{ - "name" : "point", - "count" : 18, - "VALIDITY" : [0,1,1,0,1,0,0,0,1,0,0,1,1,1,0,1,1,1], - "children" : [{ - "name" : "coordinates", - "count" : 108, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["253402300799999999","72057594037927936","1577923200000000","1","1577836800000000","1577923200000000","253402300799999999","0","1577923200000000","0","1577836800000000","1577923200000000","1577923200000000","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","360287970189639680","1577923200000000","5","1577923200000000","253402300799999999","253402300799999999","72057594037927936","1577923200000000","4","1577923200000000","253402300799999999","1577923200000000","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","504403158265495552","1578009600000000","7","1578009600000000","253402300799999999","253402300799999999","576460752303423488","1578096000000000","8","1578096000000000","253402300799999999","253402300799999999","576460752303423488","1578096000000000","9","1578096000000000","253402300799999999","1578096000000000","576460752303423488","1578096000000000","9","1578096000000000","253402300799999999","1578096000000000","576460752303423488","1578096000000000","8","1578096000000000","253402300799999999","253402300799999999","864691128455135232","1578096000000000","12","1578096000000000","253402300799999999","253402300799999999","792633534417207296","1578096000000000","11","1578096000000000","253402300799999999","253402300799999999","936748722493063168","1578182400000000","13","1578182400000000","253402300799999999","253402300799999999","1008806316530991104","1578268800000000","14","1578268800000000","253402300799999999","253402300799999999","144115188075855872","1577836800000000","2","1577836800000000","253402300799999999"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-_row_id.arrow.json index bf60d3d762..b6c3229670 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-_row_id.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-_row_id.arrow.json @@ -17,7 +17,7 @@ "name" : "_row_id", "count" : 6, "VALIDITY" : [1,1,1,1,1,1], - "DATA" : ["2","5","7","12","13","14"] + "DATA" : ["2","5","7","11","12","13"] }] }] } \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/metadata.arrow.json index 574986963b..a9e14e8216 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/metadata.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/metadata.arrow.json @@ -182,7 +182,7 @@ "name" : "max", "count" : 10, "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["14","18145","0","0","0","14","18145","0","0","0"] + "DATA" : ["13","18145","0","0","0","13","18145","0","0","0"] }] },{ "name" : "timestamp-tz-micro-utc", @@ -219,8 +219,8 @@ "name" : "bloom", "count" : 10, "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,100,208,340,368,420,520,628,760,788,840], - "DATA" : ["3a300000070000000000070004000000050001000a0001000b0000000e0000000f000200400000005000000052000000560000005a0000005c0000005e0000000200050007000c000d000e00e014b429d2949e70ce906a4a2fe19521dbd67c0c706b4586","3a300000080000000000070001000100020000000500000008000100090001000a0000000c00000048000000580000005c0000005e0000006000000064000000680000006a00000000006d115a21372ee43ae14667fa29fd97c606da8f8afc2a05a2b0fd5002fe4df8559a9e","3a3000000b0000000000000002000200040001000500010006000100090000000a0000000b0000000c0001000e0001000f0000006000000062000000680000006c000000700000007400000076000000780000007a0000007e00000082000000c402595b6f650af4582a1c3b57f945fbe42583504c016dbb93972a6e309bfa4e6e9008db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000009b003601d1018a6364dfb0e7","3a300000070000000000070004000000050001000a0001000b0000000e0000000f000200400000005000000052000000560000005a0000005c0000005e0000000200050007000c000d000e00e014b429d2949e70ce906a4a2fe19521dbd67c0c706b4586","3a300000080000000000070001000100020000000500000008000100090001000a0000000c00000048000000580000005c0000005e0000006000000064000000680000006a00000000006d115a21372ee43ae14667fa29fd97c606da8f8afc2a05a2b0fd5002fe4df8559a9e","3a3000000b0000000000000002000200040001000500010006000100090000000a0000000b0000000c0001000e0001000f0000006000000062000000680000006c000000700000007400000076000000780000007a0000007e00000082000000c402595b6f650af4582a1c3b57f945fbe42583504c016dbb93972a6e309bfa4e6e9008db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000009b003601d1018a6364dfb0e7"] + "OFFSET" : [0,116,224,356,384,436,552,660,792,820,872], + "DATA" : ["3a30000009000000000007000400000005000100090000000a0001000b0000000c0000000e0000000f00000050000000600000006200000066000000680000006c0000006e00000070000000720000000200050007000b000c000d00e014b429d2949e70ce9021c36a4a2fe1952196e1dbd6706b","3a300000080000000000070001000100020000000500000008000100090001000a0000000c00000048000000580000005c0000005e0000006000000064000000680000006a00000000006d115a21372ee43ae14667fa29fd97c606da8f8afc2a05a2b0fd5002fe4df8559a9e","3a3000000b0000000000000002000200040001000500010006000100090000000a0000000b0000000c0001000e0001000f0000006000000062000000680000006c000000700000007400000076000000780000007a0000007e00000082000000c402595b6f650af4582a1c3b57f945fbe42583504c016dbb93972a6e309bfa4e6e9008db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000009b003601d1018a6364dfb0e7","3a30000009000000000007000400000005000100090000000a0001000b0000000c0000000e0000000f00000050000000600000006200000066000000680000006c0000006e00000070000000720000000200050007000b000c000d00e014b429d2949e70ce9021c36a4a2fe1952196e1dbd6706b","3a300000080000000000070001000100020000000500000008000100090001000a0000000c00000048000000580000005c0000005e0000006000000064000000680000006a00000000006d115a21372ee43ae14667fa29fd97c606da8f8afc2a05a2b0fd5002fe4df8559a9e","3a3000000b0000000000000002000200040001000500010006000100090000000a0000000b0000000c0001000e0001000f0000006000000062000000680000006c000000700000007400000076000000780000007a0000007e00000082000000c402595b6f650af4582a1c3b57f945fbe42583504c016dbb93972a6e309bfa4e6e9008db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000009b003601d1018a6364dfb0e7"] }] }] }] diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-metadata/00.transit.json index 2dba80c78f..dca22a2dcf 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",18145,"~:system-time",["~#time/instant","2020-01-06T00:00:00Z"]]],"~:latest-row-id",14,"~:tables",["^ ","world",["^ ","~:col-types",["^ ","b","~:i64","xt$id","~:uuid"]],"foo",["^ ","^8",["^ ","^:","^9","bar","^9","toto",["~:union",["~#set",["~:utf8","~:absent"]]]]],"hello",["^ ","^8",["^ ","a","^9","^:","^;"]],"xt$txs",["^ ","^8",["^ ","^:","^9","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["^=",["^>",["~:clj-form","~:null"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",18145,"~:system-time",["~#time/instant","2020-01-06T00:00:00Z"]]],"~:latest-row-id",13,"~:tables",["^ ","world",["^ ","~:col-types",["^ ","b","~:i64","xt$id","~:uuid"]],"foo",["^ ","^8",["^ ","^:","^9","bar","^9","toto",["~:union",["~#set",["~:utf8","~:absent"]]]]],"hello",["^ ","^8",["^ ","a","^9","^:","^;"]],"xt$txs",["^ ","^8",["^ ","^:","^9","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["^=",["^>",["~:clj-form","~:null"]]]]]]] \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/foo/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/foo/chunks/leaf-c00.arrow.json index 83e0508707..89d0377d25 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/foo/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/foo/chunks/leaf-c00.arrow.json @@ -178,7 +178,7 @@ "name" : "xt$legacy_iid", "count" : 5, "VALIDITY" : [1,1,1,1,1], - "DATA" : ["792633534417207296","792633534417207296","576460752303423488","576460752303423488","576460752303423488"] + "DATA" : ["720575940379279360","720575940379279360","576460752303423488","576460752303423488","576460752303423488"] },{ "name" : "xt$system_from", "count" : 5, diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/xt$txs/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/xt$txs/chunks/leaf-c00.arrow.json index f5fd8ec2ee..7e78438ce9 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/xt$txs/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/xt$txs/chunks/leaf-c00.arrow.json @@ -151,7 +151,7 @@ "name" : "xt$legacy_iid", "count" : 6, "VALIDITY" : [1,1,1,1,1,1], - "DATA" : ["864691128455135232","936748722493063168","504403158265495552","360287970189639680","144115188075855872","1008806316530991104"] + "DATA" : ["792633534417207296","864691128455135232","504403158265495552","360287970189639680","144115188075855872","936748722493063168"] },{ "name" : "xt$system_from", "count" : 6, diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/temporal-snapshots/00.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/temporal-snapshots/00.arrow.json deleted file mode 100644 index 4840e93563..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/temporal-snapshots/00.arrow.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "point", - "nullable" : false, - "type" : { - "name" : "fixedsizelist", - "listSize" : 6 - }, - "children" : [{ - "name" : "coordinates", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }], - "metadata" : [{ - "value" : "{\"maxs\":[253402300800000000,1008806316530991104,1578268800000000,14,1578268800000000],\"value-count\":32,\"axis-shift\":0,\"mins\":[1577923200000000,0,1577836800000000,0,1577836800000000],\"deletes?\":false,\"scales\":[null,null,null,null,null],\"k-minus-one-slope+base\":[3.9710214298172855E-17,-0.06265966841805867],\"size\":10,\"k\":6,\"cell-shift\":4}", - "key" : "grid-meta" - }] - }, - "batches" : [{ - "count" : 10, - "columns" : [{ - "name" : "point", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "coordinates", - "count" : 60, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["253402300799999999","0","1577923200000000","0","1577836800000000","1577923200000000","1577923200000000","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","360287970189639680","1577923200000000","5","1577923200000000","253402300799999999","253402300799999999","504403158265495552","1578009600000000","7","1578009600000000","253402300799999999","1578096000000000","576460752303423488","1578096000000000","9","1578096000000000","253402300799999999","1578096000000000","576460752303423488","1578096000000000","8","1578096000000000","253402300799999999","253402300799999999","864691128455135232","1578096000000000","12","1578096000000000","253402300799999999","253402300799999999","936748722493063168","1578182400000000","13","1578182400000000","253402300799999999","253402300799999999","1008806316530991104","1578268800000000","14","1578268800000000","253402300799999999","253402300799999999","144115188075855872","1577836800000000","2","1577836800000000","253402300799999999"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/temporal.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/temporal.arrow.json deleted file mode 100644 index 69364f7f77..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/temporal.arrow.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "point", - "nullable" : false, - "type" : { - "name" : "fixedsizelist", - "listSize" : 6 - }, - "children" : [{ - "name" : "coordinates", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }], - "metadata" : [{ - "value" : "{\"maxs\":[253402300800000000,432345564227567616,1577836800000000,6,1577836800000000],\"value-count\":16,\"axis-shift\":0,\"mins\":[253402300800000000,0,1577836800000000,0,1577836800000000],\"deletes?\":true,\"scales\":[null,null,null,null,null],\"k-minus-one-slope+base\":[0.0,-0.0],\"size\":7,\"k\":6,\"cell-shift\":3}", - "key" : "grid-meta" - }] - }, - "batches" : [{ - "count" : 7, - "columns" : [{ - "name" : "point", - "count" : 7, - "VALIDITY" : [1,1,1,1,1,1,1], - "children" : [{ - "name" : "coordinates", - "count" : 42, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["253402300799999999","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","144115188075855872","1577836800000000","2","1577836800000000","253402300799999999","253402300799999999","216172782113783808","1577836800000000","3","1577836800000000","253402300799999999","253402300799999999","288230376151711744","1577836800000000","4","1577836800000000","253402300799999999","253402300799999999","360287970189639680","1577836800000000","5","1577836800000000","253402300799999999","253402300799999999","432345564227567616","1577836800000000","6","1577836800000000","253402300799999999"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/temporal-snapshots/00.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/temporal-snapshots/00.arrow.json deleted file mode 100644 index f46de6a127..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/temporal-snapshots/00.arrow.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "point", - "nullable" : false, - "type" : { - "name" : "fixedsizelist", - "listSize" : 6 - }, - "children" : [{ - "name" : "coordinates", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }], - "metadata" : [{ - "value" : "{\"maxs\":[253402300800000000,432345564227567616,1577836800000000,6,1577836800000000],\"value-count\":16,\"axis-shift\":0,\"mins\":[253402300800000000,0,1577836800000000,0,1577836800000000],\"deletes?\":false,\"scales\":[null,null,null,null,null],\"k-minus-one-slope+base\":[0.0,-0.0],\"size\":7,\"k\":6,\"cell-shift\":3}", - "key" : "grid-meta" - }] - }, - "batches" : [{ - "count" : 7, - "columns" : [{ - "name" : "point", - "count" : 7, - "VALIDITY" : [1,1,1,1,1,1,1], - "children" : [{ - "name" : "coordinates", - "count" : 42, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["253402300799999999","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","144115188075855872","1577836800000000","2","1577836800000000","253402300799999999","253402300799999999","216172782113783808","1577836800000000","3","1577836800000000","253402300799999999","253402300799999999","288230376151711744","1577836800000000","4","1577836800000000","253402300799999999","253402300799999999","360287970189639680","1577836800000000","5","1577836800000000","253402300799999999","253402300799999999","432345564227567616","1577836800000000","6","1577836800000000","253402300799999999"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/temporal.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/temporal.arrow.json deleted file mode 100644 index 381100b4fa..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/temporal.arrow.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "point", - "nullable" : false, - "type" : { - "name" : "fixedsizelist", - "listSize" : 6 - }, - "children" : [{ - "name" : "coordinates", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }], - "metadata" : [{ - "value" : "{\"maxs\":[253402300800000000,144115188075855872,1577836800000000,2,1577836800000000],\"value-count\":8,\"axis-shift\":0,\"mins\":[253402300800000000,0,1577836800000000,0,1577836800000000],\"deletes?\":true,\"scales\":[null,null,null,null,null],\"k-minus-one-slope+base\":[0.0,-0.0],\"size\":3,\"k\":6,\"cell-shift\":2}", - "key" : "grid-meta" - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "point", - "count" : 3, - "VALIDITY" : [1,1,1], - "children" : [{ - "name" : "coordinates", - "count" : 18, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["253402300799999999","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","144115188075855872","1577836800000000","2","1577836800000000","253402300799999999"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/temporal-snapshots/00.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/temporal-snapshots/00.arrow.json deleted file mode 100644 index 7a2c2df806..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/temporal-snapshots/00.arrow.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "point", - "nullable" : false, - "type" : { - "name" : "fixedsizelist", - "listSize" : 6 - }, - "children" : [{ - "name" : "coordinates", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }], - "metadata" : [{ - "value" : "{\"maxs\":[253402300800000000,144115188075855872,1577836800000000,2,1577836800000000],\"value-count\":8,\"axis-shift\":0,\"mins\":[253402300800000000,0,1577836800000000,0,1577836800000000],\"deletes?\":false,\"scales\":[null,null,null,null,null],\"k-minus-one-slope+base\":[0.0,-0.0],\"size\":3,\"k\":6,\"cell-shift\":2}", - "key" : "grid-meta" - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "point", - "count" : 3, - "VALIDITY" : [1,1,1], - "children" : [{ - "name" : "coordinates", - "count" : 18, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["253402300799999999","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","144115188075855872","1577836800000000","2","1577836800000000","253402300799999999"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/temporal.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/temporal.arrow.json deleted file mode 100644 index 04b641c496..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/temporal.arrow.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "point", - "nullable" : false, - "type" : { - "name" : "fixedsizelist", - "listSize" : 6 - }, - "children" : [{ - "name" : "coordinates", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }], - "metadata" : [{ - "value" : "{\"maxs\":[253402300800000000,504403158265495552,1577923200000000,7,1577923200000000],\"value-count\":16,\"axis-shift\":0,\"mins\":[253402300800000000,0,1577836800000000,0,1577836800000000],\"deletes?\":true,\"scales\":[null,null,null,null,null],\"k-minus-one-slope+base\":[0.0,-0.0],\"size\":8,\"k\":6,\"cell-shift\":3}", - "key" : "grid-meta" - }] - }, - "batches" : [{ - "count" : 8, - "columns" : [{ - "name" : "point", - "count" : 8, - "VALIDITY" : [1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "coordinates", - "count" : 48, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["253402300799999999","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","144115188075855872","1577836800000000","2","1577836800000000","253402300799999999","253402300799999999","216172782113783808","1577836800000000","3","1577836800000000","253402300799999999","253402300799999999","288230376151711744","1577836800000000","4","1577836800000000","253402300799999999","253402300799999999","360287970189639680","1577923200000000","5","1577923200000000","253402300799999999","253402300799999999","432345564227567616","1577923200000000","6","1577923200000000","253402300799999999","253402300799999999","504403158265495552","1577923200000000","7","1577923200000000","253402300799999999"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/temporal-snapshots/00.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/temporal-snapshots/00.arrow.json deleted file mode 100644 index d666807e12..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/temporal-snapshots/00.arrow.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "point", - "nullable" : false, - "type" : { - "name" : "fixedsizelist", - "listSize" : 6 - }, - "children" : [{ - "name" : "coordinates", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }], - "metadata" : [{ - "value" : "{\"maxs\":[253402300800000000,504403158265495552,1577923200000000,7,1577923200000000],\"value-count\":16,\"axis-shift\":0,\"mins\":[253402300800000000,0,1577836800000000,0,1577836800000000],\"deletes?\":false,\"scales\":[null,null,null,null,null],\"k-minus-one-slope+base\":[0.0,-0.0],\"size\":8,\"k\":6,\"cell-shift\":3}", - "key" : "grid-meta" - }] - }, - "batches" : [{ - "count" : 8, - "columns" : [{ - "name" : "point", - "count" : 8, - "VALIDITY" : [1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "coordinates", - "count" : 48, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["253402300799999999","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","144115188075855872","1577836800000000","2","1577836800000000","253402300799999999","253402300799999999","216172782113783808","1577836800000000","3","1577836800000000","253402300799999999","253402300799999999","288230376151711744","1577836800000000","4","1577836800000000","253402300799999999","253402300799999999","360287970189639680","1577923200000000","5","1577923200000000","253402300799999999","253402300799999999","432345564227567616","1577923200000000","6","1577923200000000","253402300799999999","253402300799999999","504403158265495552","1577923200000000","7","1577923200000000","253402300799999999"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/temporal.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/temporal.arrow.json deleted file mode 100644 index 659a07ac62..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/temporal.arrow.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "point", - "nullable" : false, - "type" : { - "name" : "fixedsizelist", - "listSize" : 6 - }, - "children" : [{ - "name" : "coordinates", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }], - "metadata" : [{ - "value" : "{\"maxs\":[253402300800000000,432345564227567616,1578009600000000,6,1588291200000000],\"value-count\":32,\"axis-shift\":0,\"mins\":[1578009600000000,0,1577836800000000,0,1577836800000000],\"deletes?\":true,\"scales\":[null,null,null,null,null],\"k-minus-one-slope+base\":[4.368258458778279E-17,-0.0692674394347795],\"size\":9,\"k\":6,\"cell-shift\":4}", - "key" : "grid-meta" - }] - }, - "batches" : [{ - "count" : 11, - "columns" : [{ - "name" : "point", - "count" : 11, - "VALIDITY" : [1,1,1,1,1,0,1,1,0,1,1], - "children" : [{ - "name" : "coordinates", - "count" : 66, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["253402300799999999","0","1578009600000000","0","1577836800000000","1585699200000000","253402300799999999","72057594037927936","1578009600000000","1","1577836800000000","1585699200000000","253402300799999999","72057594037927936","1578009600000000","5","1585699200000000","1588291200000000","1578009600000000","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","144115188075855872","1577836800000000","2","1577836800000000","253402300799999999","253402300799999999","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","216172782113783808","1577923200000000","3","1577923200000000","253402300799999999","1578009600000000","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","72057594037927936","1578009600000000","1","1588291200000000","253402300799999999","253402300799999999","432345564227567616","1578009600000000","6","1578009600000000","253402300799999999"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/temporal-snapshots/00.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/temporal-snapshots/00.arrow.json deleted file mode 100644 index 7a8c6d9b53..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/temporal-snapshots/00.arrow.json +++ /dev/null @@ -1,40 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "point", - "nullable" : false, - "type" : { - "name" : "fixedsizelist", - "listSize" : 6 - }, - "children" : [{ - "name" : "coordinates", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }], - "metadata" : [{ - "value" : "{\"maxs\":[253402300800000000,432345564227567616,1578009600000000,6,1588291200000000],\"value-count\":32,\"axis-shift\":0,\"mins\":[1578009600000000,0,1577836800000000,0,1577836800000000],\"deletes?\":false,\"scales\":[null,null,null,null,null],\"k-minus-one-slope+base\":[3.5740296480913193E-17,-0.05667335953754687],\"size\":9,\"k\":6,\"cell-shift\":4}", - "key" : "grid-meta" - }] - }, - "batches" : [{ - "count" : 9, - "columns" : [{ - "name" : "point", - "count" : 9, - "VALIDITY" : [1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "coordinates", - "count" : 54, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["253402300799999999","0","1578009600000000","0","1577836800000000","1585699200000000","253402300799999999","72057594037927936","1578009600000000","1","1577836800000000","1585699200000000","253402300799999999","72057594037927936","1578009600000000","5","1585699200000000","1588291200000000","1578009600000000","0","1577836800000000","0","1577836800000000","253402300799999999","253402300799999999","144115188075855872","1577836800000000","2","1577836800000000","253402300799999999","253402300799999999","216172782113783808","1577923200000000","3","1577923200000000","253402300799999999","1578009600000000","72057594037927936","1577836800000000","1","1577836800000000","253402300799999999","253402300799999999","72057594037927936","1578009600000000","1","1588291200000000","253402300799999999","253402300799999999","432345564227567616","1578009600000000","6","1578009600000000","253402300799999999"] - }] - }] - }] -} \ No newline at end of file From bea2d8a12e892cacc5828484a1911fc6401f3920 Mon Sep 17 00:00:00 2001 From: James Henderson Date: Fri, 4 Aug 2023 16:13:25 +0100 Subject: [PATCH 3/8] shift row-counter responsibility to the indexer s.t. we can later remove the live-chunk #2663 --- core/src/main/clojure/xtdb/indexer.clj | 96 +++++++++++++++---------- core/src/main/java/xtdb/RowCounter.java | 27 +++++++ src/main/clojure/xtdb/test_util.clj | 2 + src/test/clojure/xtdb/stats_test.clj | 3 +- 4 files changed, 91 insertions(+), 37 deletions(-) create mode 100644 core/src/main/java/xtdb/RowCounter.java diff --git a/core/src/main/clojure/xtdb/indexer.clj b/core/src/main/clojure/xtdb/indexer.clj index 519fdaee12..05302d147d 100644 --- a/core/src/main/clojure/xtdb/indexer.clj +++ b/core/src/main/clojure/xtdb/indexer.clj @@ -33,6 +33,7 @@ (org.apache.arrow.vector BitVector) (org.apache.arrow.vector.complex DenseUnionVector ListVector) (org.apache.arrow.vector.ipc ArrowStreamReader) + xtdb.RowCounter (xtdb.api.protocols ClojureForm TransactionInstant) xtdb.indexer.internal_id_manager.IInternalIdManager (xtdb.indexer.live_index ILiveIndex ILiveIndexTx) @@ -81,7 +82,8 @@ (.digest eid-bytes) (Arrays/copyOfRange 0 16))))) -(defn- ->put-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk, +(defn- ->put-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^RowCounter row-counter, + ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk, ^IVectorReader tx-ops-rdr, ^Instant system-time] (let [put-leg (.legReader tx-ops-rdr :put) doc-rdr (.structKeyReader put-leg "document") @@ -134,11 +136,14 @@ :valid-to (util/micros->instant valid-to)}))) (let [{:keys [^xtdb.indexer.live_index.ILiveTableTx live-table, ^IRowCopier doc-copier]} live-idx-table] - (.logPut live-table (->iid eid) legacy-iid valid-from valid-to #(.copyRow doc-copier tx-op-idx))))) + (.logPut live-table (->iid eid) legacy-iid valid-from valid-to #(.copyRow doc-copier tx-op-idx)))) + + (.addRows row-counter 1)) nil)))) -(defn- ->delete-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^ILiveIndexTx live-idx-tx ^ILiveChunkTx live-chunk +(defn- ->delete-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^RowCounter row-counter, + ^ILiveIndexTx live-idx-tx ^ILiveChunkTx live-chunk ^IVectorReader tx-ops-rdr, ^Instant current-time] (let [delete-leg (.legReader tx-ops-rdr :delete) table-rdr (.structKeyReader delete-leg "table") @@ -164,11 +169,13 @@ :valid-to (util/micros->instant valid-to)}))) (-> (.liveTable live-idx-tx table) - (.logDelete (->iid eid) legacy-iid valid-from valid-to))) + (.logDelete (->iid eid) legacy-iid valid-from valid-to)) + + (.addRows row-counter 1)) nil)))) -(defn- ->evict-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^ILiveIndexTx live-idx-tx ^ILiveChunkTx live-chunk ^IVectorReader tx-ops-rdr] +(defn- ->evict-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^RowCounter row-counter, ^ILiveIndexTx live-idx-tx ^ILiveChunkTx live-chunk ^IVectorReader tx-ops-rdr] (let [evict-leg (.legReader tx-ops-rdr :evict) table-rdr (.structKeyReader evict-leg "_table") @@ -181,7 +188,9 @@ legacy-iid (.getOrCreateInternalId iid-mgr table eid row-id)] (-> (.liveTable live-idx-tx table) - (.logEvict (->iid eid) legacy-iid))) + (.logEvict (->iid eid) legacy-iid)) + + (.addRows row-counter 1)) nil)))) @@ -297,7 +306,8 @@ (definterface SqlOpIndexer (^void indexOp [^xtdb.vector.RelationReader inRelation, queryOpts])) -(defn- ->sql-upsert-indexer ^xtdb.indexer.SqlOpIndexer [^IInternalIdManager iid-mgr, ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk, +(defn- ->sql-upsert-indexer ^xtdb.indexer.SqlOpIndexer [^IInternalIdManager iid-mgr, ^RowCounter row-counter, + ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk, {{:keys [^Instant current-time]} :basis}] (let [current-time-µs (util/instant->micros current-time)] @@ -340,9 +350,11 @@ {:valid-from (util/micros->instant valid-from) :valid-to (util/micros->instant valid-to)}))) - (.logPut live-idx-table (->iid eid) legacy-iid valid-from valid-to #(.copyRow live-idx-table-copier idx)))))))))) + (.logPut live-idx-table (->iid eid) legacy-iid valid-from valid-to #(.copyRow live-idx-table-copier idx))))) + + (.addRows row-counter row-count)))))) -(defn- ->sql-delete-indexer ^xtdb.indexer.SqlOpIndexer [^ILiveIndexTx live-idx-tx] +(defn- ->sql-delete-indexer ^xtdb.indexer.SqlOpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx] (reify SqlOpIndexer (indexOp [_ in-rel {:keys [table]}] (let [table (util/str->normal-form-str table) @@ -362,9 +374,11 @@ :valid-to (util/micros->instant valid-to)}))) (-> (.liveTable live-idx-tx table) - (.logDelete (->iid eid) iid valid-from valid-to)))))))) + (.logDelete (->iid eid) iid valid-from valid-to)))) -(defn- ->sql-erase-indexer ^xtdb.indexer.SqlOpIndexer [^ILiveIndexTx live-idx-tx] + (.addRows row-counter row-count))))) + +(defn- ->sql-erase-indexer ^xtdb.indexer.SqlOpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx] (reify SqlOpIndexer (indexOp [_ in-rel {:keys [table]}] (let [table (util/str->normal-form-str table) @@ -375,18 +389,20 @@ (let [eid (.getObject id-rdr idx) iid (.getLong iid-rdr idx)] (-> (.liveTable live-idx-tx table) - (.logEvict (->iid eid) iid)))))))) + (.logEvict (->iid eid) iid)))) + + (.addRows row-counter row-count))))) -(defn- ->sql-indexer ^xtdb.indexer.OpIndexer [^BufferAllocator allocator, ^IInternalIdManager iid-mgr +(defn- ->sql-indexer ^xtdb.indexer.OpIndexer [^BufferAllocator allocator, ^IInternalIdManager iid-mgr, ^RowCounter row-counter ^ILiveIndexTx live-idx-tx, ^ILiveChunk doc-idxer ^IVectorReader tx-ops-rdr, ^IRaQuerySource ra-src, wm-src, ^IScanEmitter scan-emitter {:keys [default-all-valid-time? basis default-tz] :as tx-opts}] (let [sql-leg (.legReader tx-ops-rdr :sql) query-rdr (.structKeyReader sql-leg "query") params-rdr (.structKeyReader sql-leg "params") - upsert-idxer (->sql-upsert-indexer iid-mgr live-idx-tx doc-idxer tx-opts) - delete-idxer (->sql-delete-indexer live-idx-tx) - erase-idxer (->sql-erase-indexer live-idx-tx)] + upsert-idxer (->sql-upsert-indexer iid-mgr row-counter live-idx-tx doc-idxer tx-opts) + delete-idxer (->sql-delete-indexer row-counter live-idx-tx) + erase-idxer (->sql-erase-indexer row-counter live-idx-tx)] (reify OpIndexer (indexOp [_ tx-op-idx] (letfn [(index-op [^SqlOpIndexer op-idxer {:keys [all-app-time] :as query-opts} inner-query] @@ -444,7 +460,7 @@ (def ^:private ^:const ^String txs-table "xt$txs") -(defn- add-tx-row! [^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk-tx, ^IInternalIdManager iid-mgr, ^TransactionInstant tx-key, ^Throwable t] +(defn- add-tx-row! [^IInternalIdManager iid-mgr, ^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk-tx, ^TransactionInstant tx-key, ^Throwable t] (let [tx-id (.tx-id tx-key) system-time-µs (util/instant->micros (.system-time tx-key)) @@ -492,7 +508,9 @@ (doto (.writerForType e-wtr :null) (.writeNull nil)) (doto (.writerForType e-wtr :clj-form) - (.writeObject (pr-str t)))))))) + (.writeObject (pr-str t)))))) + + (.addRows row-counter 1))) (deftype Indexer [^BufferAllocator allocator ^ObjectStore object-store @@ -503,6 +521,8 @@ ^ILiveIndex live-idx ^:volatile-mutable ^TransactionInstant latest-completed-tx + ^RowCounter row-counter + ^long rows-per-chunk ^:volatile-mutable ^IWatermark shared-wm ^StampedLock wm-lock] @@ -527,11 +547,11 @@ (letfn [(index-tx-ops [^DenseUnionVector tx-ops-vec] (let [tx-ops-rdr (vr/vec->reader tx-ops-vec) - !put-idxer (delay (->put-indexer iid-mgr live-idx-tx live-chunk-tx tx-ops-rdr system-time)) - !delete-idxer (delay (->delete-indexer iid-mgr live-idx-tx live-chunk-tx tx-ops-rdr system-time)) - !evict-idxer (delay (->evict-indexer iid-mgr live-idx-tx live-chunk-tx tx-ops-rdr)) + !put-idxer (delay (->put-indexer iid-mgr row-counter live-idx-tx live-chunk-tx tx-ops-rdr system-time)) + !delete-idxer (delay (->delete-indexer iid-mgr row-counter live-idx-tx live-chunk-tx tx-ops-rdr system-time)) + !evict-idxer (delay (->evict-indexer iid-mgr row-counter live-idx-tx live-chunk-tx tx-ops-rdr)) !call-idxer (delay (->call-indexer allocator ra-src wm-src scan-emitter tx-ops-rdr tx-opts)) - !sql-idxer (delay (->sql-indexer allocator iid-mgr live-idx-tx live-chunk-tx + !sql-idxer (delay (->sql-indexer allocator iid-mgr row-counter live-idx-tx live-chunk-tx tx-ops-rdr ra-src wm-src scan-emitter tx-opts))] (dotimes [tx-op-idx (.valueCount tx-ops-rdr)] (when-let [more-tx-ops (case (.getTypeId tx-ops-rdr tx-op-idx) @@ -565,12 +585,12 @@ (with-open [live-chunk-tx (.startTx live-chunk) live-idx-tx (.startTx live-idx tx-key)] - (add-tx-row! live-idx-tx live-chunk-tx iid-mgr tx-key e) + (add-tx-row! iid-mgr row-counter live-idx-tx live-chunk-tx tx-key e) (.commit live-chunk-tx) (.commit live-idx-tx))) (do - (add-tx-row! live-idx-tx live-chunk-tx iid-mgr tx-key nil) + (add-tx-row! iid-mgr row-counter live-idx-tx live-chunk-tx tx-key nil) (.commit live-chunk-tx) (.commit live-idx-tx))) @@ -583,7 +603,7 @@ (while (.isBlockFull live-chunk) (finish-block! this)) - (when (.isChunkFull live-chunk) + (when (>= (.getChunkRowCount row-counter) rows-per-chunk) (finish-chunk! this)) tx-key))) @@ -640,10 +660,10 @@ (throw t)))) (finish-chunk! [this] - (let [chunk-idx (.chunkIdx live-chunk)] - @(.finishChunk live-chunk latest-completed-tx) + @(.finishChunk live-chunk latest-completed-tx) - (.finishChunk live-idx chunk-idx)) + (.finishChunk live-idx (.getChunkIdx row-counter)) + (.nextChunk row-counter) (let [wm-lock-stamp (.writeLock wm-lock)] (try @@ -670,22 +690,26 @@ :internal-id-mgr (ig/ref :xtdb.indexer/internal-id-manager) :live-chunk (ig/ref :xtdb/live-chunk) :live-index (ig/ref :xtdb.indexer/live-index) - :ra-src (ig/ref ::op/ra-query-source)} + :ra-src (ig/ref ::op/ra-query-source) + :rows-per-chunk 102400} opts)) (defmethod ig/init-key :xtdb/indexer [_ {:keys [allocator object-store metadata-mgr scan-emitter, ra-src - internal-id-mgr live-chunk live-index]}] + internal-id-mgr live-chunk live-index, rows-per-chunk]}] - (let [{:keys [latest-completed-tx]} (meta/latest-chunk-metadata metadata-mgr)] + (let [{:keys [latest-completed-tx latest-row-id], :or {latest-row-id -1}} (meta/latest-chunk-metadata metadata-mgr)] - (Indexer. allocator object-store scan-emitter internal-id-mgr - ra-src live-chunk live-index + (assert live-chunk) + (->Indexer allocator object-store scan-emitter internal-id-mgr + ra-src live-chunk live-index - latest-completed-tx + latest-completed-tx + (RowCounter. (inc ^long latest-row-id)) + rows-per-chunk - nil ; watermark - (StampedLock.)))) + nil ; watermark + (StampedLock.)))) (defmethod ig/halt-key! :xtdb/indexer [_ ^AutoCloseable indexer] (.close indexer)) diff --git a/core/src/main/java/xtdb/RowCounter.java b/core/src/main/java/xtdb/RowCounter.java new file mode 100644 index 0000000000..fa31b2db5c --- /dev/null +++ b/core/src/main/java/xtdb/RowCounter.java @@ -0,0 +1,27 @@ +package xtdb; + +public class RowCounter { + private long chunkIdx; + private long chunkRowCount; + + public RowCounter(long chunkIdx) { + this.chunkIdx = chunkIdx; + } + + public void nextChunk() { + chunkIdx += chunkRowCount; + chunkRowCount = 0; + } + + public void addRows(int rowCount) { + chunkRowCount += rowCount; + } + + public long getChunkRowCount() { + return chunkRowCount; + } + + public long getChunkIdx() { + return chunkIdx; + } +} diff --git a/src/main/clojure/xtdb/test_util.clj b/src/main/clojure/xtdb/test_util.clj index fe3731adb3..d45cc5dc56 100644 --- a/src/main/clojure/xtdb/test_util.clj +++ b/src/main/clojure/xtdb/test_util.clj @@ -245,6 +245,8 @@ :xtdb.tx-producer/tx-producer {:instant-src instant-src} :xtdb.buffer-pool/buffer-pool {:cache-path (.resolve node-dir buffers-dir)} :xtdb.object-store/file-system-object-store {:root-path (.resolve node-dir "objects")} + :xtdb/indexer (->> {:rows-per-chunk rows-per-chunk} + (into {} (filter val))) :xtdb/live-chunk (->> {:rows-per-block rows-per-block :rows-per-chunk rows-per-chunk} (into {} (filter val)))}))) diff --git a/src/test/clojure/xtdb/stats_test.clj b/src/test/clojure/xtdb/stats_test.clj index f028cc2a9b..40a417869d 100644 --- a/src/test/clojure/xtdb/stats_test.clj +++ b/src/test/clojure/xtdb/stats_test.clj @@ -9,7 +9,8 @@ (t/use-fixtures :each tu/with-allocator) (deftest test-scan - (with-open [node (node/start-node {:xtdb/live-chunk {:rows-per-block 2 , :rows-per-chunk 2}})] + (with-open [node (node/start-node {:xtdb/indexer {:rows-per-chunk 2} + :xtdb/live-chunk {:rows-per-block 2 , :rows-per-chunk 2}})] (let [scan-emitter (util/component node :xtdb.operator.scan/scan-emitter)] (xt/submit-tx node [[:put :foo {:xt/id "foo1"}] [:put :bar {:xt/id "bar1"}]]) From a86aa80a22dabebf27caa944ffbf9d4dfc659ae8 Mon Sep 17 00:00:00 2001 From: James Henderson Date: Fri, 4 Aug 2023 16:47:08 +0100 Subject: [PATCH 4/8] live index maintains col-types (previously in live-chunk) #2663 --- .../main/clojure/xtdb/indexer/live_index.clj | 42 ++++++++++++++----- core/src/main/clojure/xtdb/operator/scan.clj | 6 +-- src/test/clojure/xtdb/operator/scan_test.clj | 3 +- 3 files changed, 36 insertions(+), 15 deletions(-) diff --git a/core/src/main/clojure/xtdb/indexer/live_index.clj b/core/src/main/clojure/xtdb/indexer/live_index.clj index 31646e2dc6..29fb4eea2c 100644 --- a/core/src/main/clojure/xtdb/indexer/live_index.clj +++ b/core/src/main/clojure/xtdb/indexer/live_index.clj @@ -3,15 +3,18 @@ [xtdb.buffer-pool] [xtdb.object-store] [xtdb.trie :as trie] + [xtdb.types :as types] [xtdb.util :as util] [xtdb.vector.reader :as vr] [xtdb.vector.writer :as vw]) - (:import (java.lang AutoCloseable) + (:import [clojure.lang MapEntry] + (java.lang AutoCloseable) (java.nio ByteBuffer) (java.util ArrayList HashMap Map) (java.util.concurrent CompletableFuture) (java.util.function Function) (org.apache.arrow.memory BufferAllocator) + [org.apache.arrow.vector.types.pojo Field] (xtdb.object_store ObjectStore) (xtdb.trie LiveHashTrie) (xtdb.vector IRelationWriter IVectorWriter))) @@ -19,6 +22,7 @@ ;; #_{:clj-kondo/ignore [:clojure-lsp/unused-public-var]} (definterface ILiveTableWatermark + (^java.util.Map columnTypes []) (^xtdb.vector.RelationReader liveRelation []) (^xtdb.trie.LiveHashTrie liveTrie [])) @@ -41,6 +45,7 @@ #_{:clj-kondo/ignore [:clojure-lsp/unused-public-var]} (definterface ILiveIndexWatermark + (^java.util.Map allColumnTypes []) (^xtdb.indexer.live_index.ILiveTableWatermark liveTable [^String tableName])) #_{:clj-kondo/ignore [:clojure-lsp/unused-public-var]} @@ -62,6 +67,15 @@ (^xtdb.trie.LiveHashTrie live-trie [test-live-table]) (^xtdb.vector.IRelationWriter live-rel [test-live-table])) +(defn- live-rel->col-types [^IRelationWriter live-rel] + (-> (.writerForName live-rel "op") + (.writerForTypeId (byte 0)) + (.structKeyWriter "xt$doc") + (.getVector) (.getField) (.getChildren) + (->> (into {} (map (fn [^Field child-field] + (MapEntry/create (.getName child-field) + (types/field->col-type child-field)))))))) + (defn- open-wm-live-rel ^xtdb.vector.RelationReader [^IRelationWriter rel, retain?] (let [out-cols (ArrayList.)] (try @@ -133,9 +147,11 @@ (openWatermark [_ retain?] (locking this-table - (let [wm-live-rel (open-wm-live-rel live-rel retain?) + (let [col-types (live-rel->col-types live-rel) + wm-live-rel (open-wm-live-rel live-rel retain?) wm-live-trie (.compactLogs ^LiveHashTrie @!transient-trie)] (reify ILiveTableWatermark + (columnTypes [_] col-types) (liveRelation [_] wm-live-rel) (liveTrie [_] wm-live-trie) @@ -157,9 +173,12 @@ (openWatermark [this retain?] (locking this - (let [wm-live-rel (open-wm-live-rel live-rel retain?) + (let [col-types (live-rel->col-types live-rel) + wm-live-rel (open-wm-live-rel live-rel retain?) wm-live-trie (.compactLogs live-trie)] + (reify ILiveTableWatermark + (columnTypes [_] col-types) (liveRelation [_] wm-live-rel) (liveTrie [_] wm-live-trie) @@ -182,12 +201,12 @@ op-wtr (.writerForName rel "op") put-wtr (.writerForField op-wtr trie/put-field) delete-wtr (.writerForField op-wtr trie/delete-field)] - (LiveTable. allocator object-store table-name rel - (LiveHashTrie/emptyTrie (.getVector iid-wtr)) - iid-wtr (.writerForName rel "xt$legacy_iid") (.writerForName rel "xt$system_from") - put-wtr (.structKeyWriter put-wtr "xt$valid_from") (.structKeyWriter put-wtr "xt$valid_to") (.structKeyWriter put-wtr "xt$doc") - delete-wtr (.structKeyWriter delete-wtr "xt$valid_from") (.structKeyWriter delete-wtr "xt$valid_to") - (.writerForField op-wtr trie/evict-field))))) + (->LiveTable allocator object-store table-name rel + (LiveHashTrie/emptyTrie (.getVector iid-wtr)) + iid-wtr (.writerForName rel "xt$legacy_iid") (.writerForName rel "xt$system_from") + put-wtr (.structKeyWriter put-wtr "xt$valid_from") (.structKeyWriter put-wtr "xt$valid_to") (.structKeyWriter put-wtr "xt$doc") + delete-wtr (.structKeyWriter delete-wtr "xt$valid_from") (.structKeyWriter delete-wtr "xt$valid_to") + (.writerForField op-wtr trie/evict-field))))) (defrecord LiveIndex [^BufferAllocator allocator, ^ObjectStore object-store, ^Map tables] ILiveIndex @@ -221,6 +240,7 @@ (util/->jfn (fn [_] (.openWatermark live-table false))))) (reify ILiveIndexWatermark + (allColumnTypes [_] (update-vals wms #(.columnTypes ^ILiveTableWatermark %))) (liveTable [_ table-name] (.get wms table-name)) AutoCloseable @@ -236,6 +256,8 @@ (.put wms table-name (.openWatermark live-table true))) (reify ILiveIndexWatermark + (allColumnTypes [_] (update-vals wms #(.columnTypes ^ILiveTableWatermark %))) + (liveTable [_ table-name] (.get wms table-name)) AutoCloseable @@ -262,7 +284,7 @@ opts)) (defmethod ig/init-key :xtdb.indexer/live-index [_ {:keys [allocator object-store]}] - (LiveIndex. allocator object-store (HashMap.))) + (->LiveIndex allocator object-store (HashMap.))) (defmethod ig/halt-key! :xtdb.indexer/live-index [_ live-idx] (util/close live-idx)) diff --git a/core/src/main/clojure/xtdb/operator/scan.clj b/core/src/main/clojure/xtdb/operator/scan.clj index 9d871ca6d9..4cfa557699 100644 --- a/core/src/main/clojure/xtdb/operator/scan.clj +++ b/core/src/main/clojure/xtdb/operator/scan.clj @@ -765,7 +765,7 @@ (tableColNames [_ wm table-name] (let [normalized-table (util/str->normal-form-str table-name)] (into #{} cat [(keys (.columnTypes metadata-mgr normalized-table)) - (some-> (.liveChunk wm) + (some-> (.liveIndex wm) (.liveTable normalized-table) (.columnTypes) keys)]))) @@ -774,7 +774,7 @@ (merge-with set/union (update-vals (.allColumnTypes metadata-mgr) (comp set keys)) - (update-vals (some-> (.liveChunk wm) + (update-vals (some-> (.liveIndex wm) (.allColumnTypes)) (comp set keys)))) @@ -785,7 +785,7 @@ (if (types/temporal-column? (util/str->normal-form-str (str col-name))) [:timestamp-tz :micro "UTC"] (types/merge-col-types (.columnType metadata-mgr normalized-table normalized-col-name) - (some-> (.liveChunk wm) + (some-> (.liveIndex wm) (.liveTable normalized-table) (.columnTypes) (get normalized-col-name))))))] diff --git a/src/test/clojure/xtdb/operator/scan_test.clj b/src/test/clojure/xtdb/operator/scan_test.clj index 762284db29..5c942725bd 100644 --- a/src/test/clojure/xtdb/operator/scan_test.clj +++ b/src/test/clojure/xtdb/operator/scan_test.clj @@ -284,8 +284,7 @@ (t/is (= '{xt/id [:union #{:keyword :utf8}]} (->col-types tx)))))))) -;; TODO adapt for scan/->temporal-range -#_ +#_ ; TODO adapt for scan/->temporal-range (t/deftest can-create-temporal-min-max-range (let [μs-2018 (util/instant->micros (util/->instant #inst "2018")) μs-2019 (util/instant->micros (util/->instant #inst "2019"))] From 987957f7376d9b4d08b1da5b562f6d94be8e305b Mon Sep 17 00:00:00 2001 From: James Henderson Date: Fri, 4 Aug 2023 17:32:07 +0100 Subject: [PATCH 5/8] indexer/live-index taking over responsibility for uploading the chunk metadata, #2663 --- core/src/main/clojure/xtdb/indexer.clj | 19 +++-- .../main/clojure/xtdb/indexer/live_index.clj | 58 ++++++++------ core/src/main/clojure/xtdb/live_chunk.clj | 75 ++++++++----------- core/src/main/clojure/xtdb/metadata.clj | 8 -- core/src/main/clojure/xtdb/util.clj | 4 - src/test/clojure/xtdb/indexer_test.clj | 12 +-- .../clojure/xtdb/ts_devices_small_test.clj | 4 +- .../chunk-metadata/00.transit.json | 2 +- .../chunk-metadata/00.transit.json | 2 +- .../chunk-metadata/00.transit.json | 2 +- .../chunk-metadata/00.transit.json | 2 +- .../chunk-metadata/00.transit.json | 2 +- .../chunk-metadata/00.transit.json | 2 +- 13 files changed, 93 insertions(+), 99 deletions(-) diff --git a/core/src/main/clojure/xtdb/indexer.clj b/core/src/main/clojure/xtdb/indexer.clj index 05302d147d..7a6e36acc7 100644 --- a/core/src/main/clojure/xtdb/indexer.clj +++ b/core/src/main/clojure/xtdb/indexer.clj @@ -38,6 +38,7 @@ xtdb.indexer.internal_id_manager.IInternalIdManager (xtdb.indexer.live_index ILiveIndex ILiveIndexTx) (xtdb.live_chunk ILiveChunk ILiveChunkTx ILiveTableTx) + xtdb.metadata.IMetadataManager xtdb.object_store.ObjectStore xtdb.operator.IRaQuerySource (xtdb.operator.scan IScanEmitter) @@ -514,6 +515,7 @@ (deftype Indexer [^BufferAllocator allocator ^ObjectStore object-store + ^IMetadataManager metadata-mgr ^IScanEmitter scan-emitter ^IInternalIdManager iid-mgr ^IRaQuerySource ra-src @@ -660,9 +662,16 @@ (throw t)))) (finish-chunk! [this] - @(.finishChunk live-chunk latest-completed-tx) + @(.finishChunk live-chunk) + + (let [chunk-idx (.getChunkIdx row-counter) + table-metadata (.finishChunk live-idx chunk-idx)] + + (.finishChunk metadata-mgr chunk-idx + {:latest-completed-tx latest-completed-tx + :next-chunk-idx (+ chunk-idx (.getChunkRowCount row-counter)) + :tables table-metadata})) - (.finishChunk live-idx (.getChunkIdx row-counter)) (.nextChunk row-counter) (let [wm-lock-stamp (.writeLock wm-lock)] @@ -698,14 +707,14 @@ [_ {:keys [allocator object-store metadata-mgr scan-emitter, ra-src internal-id-mgr live-chunk live-index, rows-per-chunk]}] - (let [{:keys [latest-completed-tx latest-row-id], :or {latest-row-id -1}} (meta/latest-chunk-metadata metadata-mgr)] + (let [{:keys [latest-completed-tx next-chunk-idx], :or {next-chunk-idx 0}} (meta/latest-chunk-metadata metadata-mgr)] (assert live-chunk) - (->Indexer allocator object-store scan-emitter internal-id-mgr + (->Indexer allocator object-store metadata-mgr scan-emitter internal-id-mgr ra-src live-chunk live-index latest-completed-tx - (RowCounter. (inc ^long latest-row-id)) + (RowCounter. next-chunk-idx) rows-per-chunk nil ; watermark diff --git a/core/src/main/clojure/xtdb/indexer/live_index.clj b/core/src/main/clojure/xtdb/indexer/live_index.clj index 29fb4eea2c..a74572412a 100644 --- a/core/src/main/clojure/xtdb/indexer/live_index.clj +++ b/core/src/main/clojure/xtdb/indexer/live_index.clj @@ -1,6 +1,7 @@ (ns xtdb.indexer.live-index (:require [juxt.clojars-mirrors.integrant.core :as ig] [xtdb.buffer-pool] + [xtdb.metadata :as meta] [xtdb.object-store] [xtdb.trie :as trie] [xtdb.types :as types] @@ -17,7 +18,7 @@ [org.apache.arrow.vector.types.pojo Field] (xtdb.object_store ObjectStore) (xtdb.trie LiveHashTrie) - (xtdb.vector IRelationWriter IVectorWriter))) + (xtdb.vector IRelationWriter IVectorWriter RelationReader))) ;; #_{:clj-kondo/ignore [:clojure-lsp/unused-public-var]} @@ -40,7 +41,7 @@ (definterface ILiveTable (^xtdb.indexer.live_index.ILiveTableTx startTx [^xtdb.api.protocols.TransactionInstant txKey]) (^xtdb.indexer.live_index.ILiveTableWatermark openWatermark [^boolean retain]) - (^java.util.concurrent.CompletableFuture #_ finishChunk [^long chunkIdx]) + (^java.util.concurrent.CompletableFuture #_> finishChunk [^long chunkIdx]) (^void close [])) #_{:clj-kondo/ignore [:clojure-lsp/unused-public-var]} @@ -60,21 +61,21 @@ (^xtdb.indexer.live_index.ILiveTable liveTable [^String tableName]) (^xtdb.indexer.live_index.ILiveIndexTx startTx [^xtdb.api.protocols.TransactionInstant txKey]) (^xtdb.indexer.live_index.ILiveIndexWatermark openWatermark []) - (^void finishChunk [^long chunkIdx]) + (^java.util.Map finishChunk [^long chunkIdx]) (^void close [])) (defprotocol TestLiveTable (^xtdb.trie.LiveHashTrie live-trie [test-live-table]) (^xtdb.vector.IRelationWriter live-rel [test-live-table])) -(defn- live-rel->col-types [^IRelationWriter live-rel] - (-> (.writerForName live-rel "op") - (.writerForTypeId (byte 0)) - (.structKeyWriter "xt$doc") - (.getVector) (.getField) (.getChildren) - (->> (into {} (map (fn [^Field child-field] - (MapEntry/create (.getName child-field) - (types/field->col-type child-field)))))))) +(defn- live-rel->col-types [^RelationReader live-rel] + (->> (for [^Field child-field (-> (.readerForName live-rel "op") + (.legReader :put) + (.structKeyReader "xt$doc") + (.getField) + (.getChildren))] + (MapEntry/create (.getName child-field) (types/field->col-type child-field))) + (into {}))) (defn- open-wm-live-rel ^xtdb.vector.RelationReader [^IRelationWriter rel, retain?] (let [out-cols (ArrayList.)] @@ -147,8 +148,8 @@ (openWatermark [_ retain?] (locking this-table - (let [col-types (live-rel->col-types live-rel) - wm-live-rel (open-wm-live-rel live-rel retain?) + (let [wm-live-rel (open-wm-live-rel live-rel retain?) + col-types (live-rel->col-types wm-live-rel) wm-live-trie (.compactLogs ^LiveHashTrie @!transient-trie)] (reify ILiveTableWatermark (columnTypes [_] col-types) @@ -167,14 +168,19 @@ (close [_])))) (finishChunk [_ chunk-idx] - (when-let [bufs (trie/live-trie->bufs allocator (-> live-trie (.compactLogs)) (vw/rel-wtr->rdr live-rel))] - (let [chunk-idx-str (util/->lex-hex-string chunk-idx)] - (trie/write-trie-bufs! obj-store (format "tables/%s/chunks" table-name) chunk-idx-str bufs)))) + (let [live-rel-rdr (vw/rel-wtr->rdr live-rel)] + (when-let [bufs (trie/live-trie->bufs allocator (-> live-trie (.compactLogs)) live-rel-rdr)] + (let [chunk-idx-str (util/->lex-hex-string chunk-idx) + !fut (trie/write-trie-bufs! obj-store (format "tables/%s/chunks" table-name) chunk-idx-str bufs) + table-metadata (MapEntry/create table-name + {:col-types (live-rel->col-types live-rel-rdr)})] + (-> !fut + (util/then-apply (fn [_] table-metadata))))))) (openWatermark [this retain?] (locking this - (let [col-types (live-rel->col-types live-rel) - wm-live-rel (open-wm-live-rel live-rel retain?) + (let [wm-live-rel (open-wm-live-rel live-rel retain?) + col-types (live-rel->col-types wm-live-rel) wm-live-trie (.compactLogs live-trie)] (reify ILiveTableWatermark @@ -264,15 +270,19 @@ (close [_] (util/close wms))))) (finishChunk [_ chunk-idx] - @(CompletableFuture/allOf (->> (for [^ILiveTable table (.values tables)] - (.finishChunk table chunk-idx)) + (let [futs (->> (for [^ILiveTable table (.values tables)] + (.finishChunk table chunk-idx)) - (remove nil?) + (remove nil?) + (into-array CompletableFuture))] - (into-array CompletableFuture))) + @(CompletableFuture/allOf futs) - (util/close tables) - (.clear tables)) + (util/close tables) + (.clear tables) + + (-> (into {} (keep deref) futs) + (util/rethrowing-cause)))) AutoCloseable (close [_] diff --git a/core/src/main/clojure/xtdb/live_chunk.clj b/core/src/main/clojure/xtdb/live_chunk.clj index 32b94d3aa2..33eab273dd 100644 --- a/core/src/main/clojure/xtdb/live_chunk.clj +++ b/core/src/main/clojure/xtdb/live_chunk.clj @@ -123,7 +123,7 @@ (^void finishBlock []) (^void nextBlock []) - (^java.util.concurrent.CompletableFuture finishChunk [^xtdb.api.protocols.TransactionInstant latestCompletedTx]) + (^java.util.concurrent.CompletableFuture finishChunk []) (^void nextChunk []) (^void close [])) @@ -278,38 +278,33 @@ (finishChunk [_] (let [row-counts (.blockRowCounts row-counter) - block-meta-wtr (.writeBlockMetadata table-metadata-writer -1) - - !fut (-> (CompletableFuture/allOf - (->> (cons row-id-vec (map #(.getVector ^IVectorWriter %) (vals static-rel))) - (map (fn [^ValueVector live-vec] - (let [live-root (VectorSchemaRoot/of (into-array [live-vec]))] - (.writeMetadata block-meta-wtr (vr/vec->reader live-vec)) - - (.putObject object-store (meta/->chunk-obj-key chunk-idx table-name (.getName live-vec)) - (with-open [write-root (VectorSchemaRoot/create (.getSchema live-root) allocator)] - (let [loader (VectorLoader. write-root)] - (with-open [^ICursor slices (blocks/->slices live-root row-counts)] - (let [buf (util/build-arrow-ipc-byte-buffer write-root :file - (fn [write-batch!] - (.forEachRemaining slices - (reify Consumer - (accept [_ sliced-root] - (with-open [arb (.getRecordBatch (VectorUnloader. sliced-root))] - (.load loader arb) - (write-batch!)))))))] - (.nextChunk row-counter) - buf)))))))) - (into-array CompletableFuture))) - - (util/then-compose - (fn [_] - (.endBlock block-meta-wtr) - (.finishChunk table-metadata-writer)))) - - chunk-metadata (meta/live-rel->chunk-metadata table-name (vw/rel-wtr->rdr static-rel))] - (-> !fut - (util/then-apply (fn [_] chunk-metadata))))) + block-meta-wtr (.writeBlockMetadata table-metadata-writer -1)] + (-> (CompletableFuture/allOf + (->> (cons row-id-vec (map #(.getVector ^IVectorWriter %) (vals static-rel))) + (map (fn [^ValueVector live-vec] + (let [live-root (VectorSchemaRoot/of (into-array [live-vec]))] + (.writeMetadata block-meta-wtr (vr/vec->reader live-vec)) + + (.putObject object-store (meta/->chunk-obj-key chunk-idx table-name (.getName live-vec)) + (with-open [write-root (VectorSchemaRoot/create (.getSchema live-root) allocator)] + (let [loader (VectorLoader. write-root)] + (with-open [^ICursor slices (blocks/->slices live-root row-counts)] + (let [buf (util/build-arrow-ipc-byte-buffer write-root :file + (fn [write-batch!] + (.forEachRemaining slices + (reify Consumer + (accept [_ sliced-root] + (with-open [arb (.getRecordBatch (VectorUnloader. sliced-root))] + (.load loader arb) + (write-batch!)))))))] + (.nextChunk row-counter) + buf)))))))) + (into-array CompletableFuture))) + + (util/then-compose + (fn [_] + (.endBlock block-meta-wtr) + (.finishChunk table-metadata-writer)))))) AutoCloseable (close [_] @@ -410,17 +405,11 @@ (nextBlock [_] (.nextBlock row-counter)) - (finishChunk [_ latest-completed-tx] + (finishChunk [_] (let [futs (for [^ILiveTable live-table (.values live-tables)] (.finishChunk live-table))] - (-> (CompletableFuture/allOf (into-array CompletableFuture futs)) - (util/then-apply (fn [_] - (.finishChunk metadata-mgr chunk-idx - {:latest-completed-tx latest-completed-tx - :latest-row-id (dec (+ chunk-idx (.chunkRowCount row-counter))) - :tables (-> (into {} (keep deref) futs) - (util/rethrowing-cause))})))))) + (CompletableFuture/allOf (into-array CompletableFuture futs)))) (nextChunk [this] (run! util/try-close (.values live-tables)) @@ -445,9 +434,7 @@ opts)) (defmethod ig/init-key :xtdb/live-chunk [_ {:keys [allocator object-store metadata-mgr ^long rows-per-block ^long rows-per-chunk]}] - (let [chunk-idx (if-let [{:keys [^long latest-row-id]} (meta/latest-chunk-metadata metadata-mgr)] - (inc latest-row-id) - 0) + (let [^long chunk-idx (:next-chunk-idx (meta/latest-chunk-metadata metadata-mgr) 0) bloom-false-positive-probability (bloom/bloom-false-positive-probability? rows-per-chunk)] (when (> bloom-false-positive-probability 0.05) diff --git a/core/src/main/clojure/xtdb/metadata.clj b/core/src/main/clojure/xtdb/metadata.clj index 9de2f6dfd9..00879544e0 100644 --- a/core/src/main/clojure/xtdb/metadata.clj +++ b/core/src/main/clojure/xtdb/metadata.clj @@ -118,14 +118,6 @@ (defn- ->chunk-metadata-obj-key [chunk-idx] (format "chunk-metadata/%s.transit.json" (util/->lex-hex-string chunk-idx))) -(defn live-rel->chunk-metadata [^String table-name, ^RelationReader live-rel] - (when (pos? (.rowCount live-rel)) - (MapEntry/create table-name - {:col-types (->> (for [^IVectorReader live-col live-rel] - (MapEntry/create (.getName live-col) - (types/field->col-type (.getField live-col)))) - (into {}))}))) - (defn- write-chunk-metadata ^java.nio.ByteBuffer [chunk-meta] (with-open [os (ByteArrayOutputStream.)] (let [w (transit/writer os :json {:handlers xt.transit/tj-write-handlers})] diff --git a/core/src/main/clojure/xtdb/util.clj b/core/src/main/clojure/xtdb/util.clj index 33e3e5e7ab..25f2d08357 100644 --- a/core/src/main/clojure/xtdb/util.clj +++ b/core/src/main/clojure/xtdb/util.clj @@ -110,10 +110,6 @@ (.putLong (.getLeastSignificantBits uuid)))] (.array bb))) -(defn bytes->uuid ^UUID [^bytes bytes] - (let [bb (ByteBuffer/wrap bytes)] - (UUID. (.getLong bb) (.getLong bb)))) - (defn ->lex-hex-string "Turn a long into a lexicographically-sortable hex string by prepending the length" [^long l] diff --git a/src/test/clojure/xtdb/indexer_test.clj b/src/test/clojure/xtdb/indexer_test.clj index 17d16dd478..ea5bcbcf4e 100644 --- a/src/test/clojure/xtdb/indexer_test.clj +++ b/src/test/clojure/xtdb/indexer_test.clj @@ -119,9 +119,9 @@ (tu/finish-chunk! node) (t/is (= {:latest-completed-tx last-tx-key - :latest-row-id 5} + :next-chunk-idx 6} (-> (meta/latest-chunk-metadata mm) - (select-keys [:latest-completed-tx :latest-row-id])))) + (select-keys [:latest-completed-tx :next-chunk-idx])))) (let [objects-list (->> (.listObjects os "chunk-00/device_info") (filter #(str/ends-with? % "/metadata.arrow")))] (t/is (= 1 (count objects-list))) @@ -416,9 +416,9 @@ (tu/finish-chunk! node) (t/is (= {:latest-completed-tx last-tx-key - :latest-row-id 11109} + :next-chunk-idx 11110} (-> (meta/latest-chunk-metadata mm) - (select-keys [:latest-completed-tx :latest-row-id])))) + (select-keys [:latest-completed-tx :next-chunk-idx])))) (let [objs (.listObjects os)] (t/is (= 1 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_info/metadata\.arrow" %) objs)))) @@ -495,11 +495,11 @@ (tu/then-await-tx node (Duration/ofSeconds 10))))) (t/is (= first-half-tx-key (tu/latest-completed-tx node))) - (let [{:keys [^TransactionInstant latest-completed-tx, latest-row-id]} + (let [{:keys [^TransactionInstant latest-completed-tx, next-chunk-idx]} (meta/latest-chunk-metadata mm)] (t/is (< (:tx-id latest-completed-tx) (:tx-id first-half-tx-key))) - (t/is (< latest-row-id (count first-half-tx-ops))) + (t/is (< next-chunk-idx (count first-half-tx-ops))) (let [objs (.listObjects os)] (t/is (= 2 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_info/metadata\.arrow" %) objs)))) diff --git a/src/test/clojure/xtdb/ts_devices_small_test.clj b/src/test/clojure/xtdb/ts_devices_small_test.clj index e64d6eb0e3..2b2b1a272d 100644 --- a/src/test/clojure/xtdb/ts_devices_small_test.clj +++ b/src/test/clojure/xtdb/ts_devices_small_test.clj @@ -26,9 +26,9 @@ (tu/finish-chunk! node) (t/is (= {:latest-completed-tx last-tx-key - :latest-row-id (dec 1001000)} + :next-chunk-idx 1001000} (-> (meta/latest-chunk-metadata (tu/component ::meta/metadata-manager)) - (select-keys [:latest-completed-tx :latest-row-id]))))) + (select-keys [:latest-completed-tx :next-chunk-idx]))))) (f)))))) diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-metadata/00.transit.json index ffea3df898..e94df4560a 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",8165,"~:system-time",["~#time/instant","2020-01-02T00:00:00Z"]]],"~:latest-row-id",5,"~:tables",["^ ","device_info",["^ ","~:col-types",["^ ","xt$id","~:utf8","os_name","^:","model","^:","api_version","^:","manufacturer","^:"]],"device_readings",["^ ","^8",["^ ","cpu_avg_1min","~:f64","cpu_avg_15min","^A","device_id","^:","battery_level","^A","mem_used","^A","battery_status","^:","mem_free","^A","time",["~:timestamp-tz","~:micro","UTC"],"ssid","^:","rssi","^A","cpu_avg_5min","^A","battery_temperature","^A","^9","^:","bssid","^:"]],"xt$txs",["^ ","^8",["^ ","^9","~:i64","xt$tx_time",["^I","^J","UTC"],"xt$committed?","~:bool","xt$error",["~:union",["~#set",["~:clj-form","~:null"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",8165,"~:system-time",["~#time/instant","2020-01-02T00:00:00Z"]]],"~:next-chunk-idx",6,"~:tables",["^ ","device_info",["^ ","~:col-types",["^ ","xt$id","~:utf8","os_name","^:","model","^:","api_version","^:","manufacturer","^:"]],"device_readings",["^ ","^8",["^ ","cpu_avg_1min","~:f64","cpu_avg_15min","^A","device_id","^:","battery_level","^A","mem_used","^A","battery_status","^:","mem_free","^A","time",["~:timestamp-tz","~:micro","UTC"],"ssid","^:","rssi","^A","cpu_avg_5min","^A","battery_temperature","^A","^9","^:","bssid","^:"]],"xt$txs",["^ ","^8",["^ ","^9","~:i64","xt$tx_time",["^I","^J","UTC"],"xt$committed?","~:bool","xt$error",["~:union",["~#set",["~:clj-form","~:null"]]]]]]] \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-metadata/00.transit.json index dca22a2dcf..1417014675 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",18145,"~:system-time",["~#time/instant","2020-01-06T00:00:00Z"]]],"~:latest-row-id",13,"~:tables",["^ ","world",["^ ","~:col-types",["^ ","b","~:i64","xt$id","~:uuid"]],"foo",["^ ","^8",["^ ","^:","^9","bar","^9","toto",["~:union",["~#set",["~:utf8","~:absent"]]]]],"hello",["^ ","^8",["^ ","a","^9","^:","^;"]],"xt$txs",["^ ","^8",["^ ","^:","^9","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["^=",["^>",["~:clj-form","~:null"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",18145,"~:system-time",["~#time/instant","2020-01-06T00:00:00Z"]]],"~:next-chunk-idx",16,"~:tables",["^ ","world",["^ ","~:col-types",["^ ","b","~:i64","xt$id","~:uuid"]],"foo",["^ ","^8",["^ ","^:","^9","bar","^9","toto",["~:union",["~#set",["~:utf8","~:absent"]]]]],"hello",["^ ","^8",["^ ","a","^9","^:","^;"]],"xt$txs",["^ ","^8",["^ ","^:","^9","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["^=",["^>",["~:clj-form","~:null"]]]]]]] \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-metadata/00.transit.json index 5dd1485b86..11a71645fd 100644 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",0,"~:system-time",["~#time/instant","2020-01-01T00:00:00Z"]]],"~:latest-row-id",6,"~:tables",["^ ","xt_docs",["^ ","~:col-types",["^ ","struct",["~:union",["~#set",[["~:struct",["^ ","~$a",["^:",["^;",["~:bool","~:i64"]]],"~$b",["^:",["^;",["~:utf8","~:absent"]]],"~$c",["^:",["^;",["^?","^@"]]]]],"^@"]]],"xt$id",["^:",["^;",["^?","~:keyword","^>"]]],"list",["^:",["^;",[["~:list",["^:",["^;",["~:f64","^?",["~:timestamp-tz","~:micro","UTC"],"^="]]]],"^@"]]]]],"xt$txs",["^ ","^8",["^ ","^A","^>","xt$tx_time",["^F","^G","UTC"],"xt$committed?","^=","xt$error",["^:",["^;",["~:clj-form","~:null"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",0,"~:system-time",["~#time/instant","2020-01-01T00:00:00Z"]]],"~:next-chunk-idx",7,"~:tables",["^ ","xt_docs",["^ ","~:col-types",["^ ","struct",["~:union",["~#set",[["~:struct",["^ ","~$a",["^:",["^;",["~:bool","~:i64"]]],"~$b",["^:",["^;",["~:utf8","~:absent"]]],"~$c",["^:",["^;",["^?","^@"]]]]],"^@"]]],"xt$id",["^:",["^;",["^?","~:keyword","^>"]]],"list",["^:",["^;",[["~:list",["^:",["^;",["~:f64","^?",["~:timestamp-tz","~:micro","UTC"],"^="]]]],"^@"]]]]],"xt$txs",["^ ","^8",["^ ","^A","^>","xt$tx_time",["^F","^G","UTC"],"xt$committed?","^=","xt$error",["^:",["^;",["~:clj-form","~:null"]]]]]]] \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-metadata/00.transit.json index d9df50c69c..aca0f0f4ac 100644 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",0,"~:system-time",["~#time/instant","2020-01-01T00:00:00Z"]]],"~:latest-row-id",2,"~:tables",["^ ","xt$txs",["^ ","~:col-types",["^ ","xt$id","~:i64","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["~:union",["~#set",["~:clj-form","~:null"]]]]],"table",["^ ","^8",["^ ","^9","^:","foo","^:","bar","~:utf8","baz",["^A",["^B",["~:f64","^:"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",0,"~:system-time",["~#time/instant","2020-01-01T00:00:00Z"]]],"~:next-chunk-idx",3,"~:tables",["^ ","xt$txs",["^ ","~:col-types",["^ ","xt$id","~:i64","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["~:union",["~#set",["~:clj-form","~:null"]]]]],"table",["^ ","^8",["^ ","^9","^:","foo","^:","bar","~:utf8","baz",["^A",["^B",["~:f64","^:"]]]]]]] \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-metadata/00.transit.json index fcc98df1d5..c98474d7da 100644 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",5845,"~:system-time",["~#time/instant","2020-01-02T00:00:00Z"]]],"~:latest-row-id",7,"~:tables",["^ ","xt_docs",["^ ","~:col-types",["^ ","struct",["~:union",["~#set",["~:absent",["~:struct",["^ ","~$a",["^:",["^;",["~:bool","~:i64"]]],"~$b",["^:",["^;",["~:utf8",["^=",["^ ","~$c","^@","~$d","^@"]]]]]]]]]],"xt$id",["^:",["^;",["^@","~:keyword","^?"]]],"list",["^:",["^;",[["~:list",["^:",["^;",["~:f64","^@",["~:timestamp-tz","~:micro","UTC"],"^>"]]]],"^<"]]]]],"xt$txs",["^ ","^8",["^ ","^A","^?","xt$tx_time",["^F","^G","UTC"],"xt$committed?","^>","xt$error",["^:",["^;",["~:clj-form","~:null"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",5845,"~:system-time",["~#time/instant","2020-01-02T00:00:00Z"]]],"~:next-chunk-idx",8,"~:tables",["^ ","xt_docs",["^ ","~:col-types",["^ ","struct",["~:union",["~#set",["~:absent",["~:struct",["^ ","~$a",["^:",["^;",["~:bool","~:i64"]]],"~$b",["^:",["^;",["~:utf8",["^=",["^ ","~$c","^@","~$d","^@"]]]]]]]]]],"xt$id",["^:",["^;",["^@","~:keyword","^?"]]],"list",["^:",["^;",[["~:list",["^:",["^;",["~:f64","^@",["~:timestamp-tz","~:micro","UTC"],"^>"]]]],"^<"]]]]],"xt$txs",["^ ","^8",["^ ","^A","^?","xt$tx_time",["^F","^G","UTC"],"xt$committed?","^>","xt$error",["^:",["^;",["~:clj-form","~:null"]]]]]]] \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-metadata/00.transit.json index ece0b3eed8..96d097d6e3 100644 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",6554,"~:system-time",["~#time/instant","2020-01-03T00:00:00Z"]]],"~:latest-row-id",6,"~:tables",["^ ","xt_docs",["^ ","~:col-types",["^ ","xt$id","~:utf8","month",["~:union",["~#set",["^:","~:absent"]]]]],"xt$txs",["^ ","^8",["^ ","^9","~:i64","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["^<",["^=",["~:clj-form","~:null"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",6554,"~:system-time",["~#time/instant","2020-01-03T00:00:00Z"]]],"~:next-chunk-idx",7,"~:tables",["^ ","xt_docs",["^ ","~:col-types",["^ ","xt$id","~:utf8","month",["~:union",["~#set",["^:","~:absent"]]]]],"xt$txs",["^ ","^8",["^ ","^9","~:i64","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["^<",["^=",["~:clj-form","~:null"]]]]]]] \ No newline at end of file From c95f43eec419cb7f1a394ea06d05683e94dd807e Mon Sep 17 00:00:00 2001 From: James Henderson Date: Fri, 4 Aug 2023 21:39:58 +0100 Subject: [PATCH 6/8] SQL DML uses new IIDs; remove internal-id-mgr, `_iid`, #2663 --- core/src/main/clojure/xtdb/indexer.clj | 108 +++++++----------- .../xtdb/indexer/internal_id_manager.clj | 67 ----------- .../main/clojure/xtdb/indexer/live_index.clj | 25 ++-- core/src/main/clojure/xtdb/live_chunk.clj | 1 - core/src/main/clojure/xtdb/node.clj | 1 - core/src/main/clojure/xtdb/operator/scan.clj | 14 +-- core/src/main/clojure/xtdb/sql/analyze.clj | 14 +-- core/src/main/clojure/xtdb/trie.clj | 1 - core/src/main/clojure/xtdb/types.clj | 2 +- .../clojure/xtdb/indexer/live_index_test.clj | 5 +- src/test/clojure/xtdb/indexer_test.clj | 13 +-- .../device_info/chunks/leaf-c00.arrow.json | 14 --- .../chunks/leaf-c00.arrow.json | 14 --- .../tables/xt$txs/chunks/leaf-c00.arrow.json | 14 --- .../chunk-00/foo/content-_row_id.arrow.json | 2 +- .../chunk-00/foo/metadata.arrow.json | 8 +- .../chunk-00/world/content-_row_id.arrow.json | 2 +- .../chunk-00/world/metadata.arrow.json | 6 +- .../xt$txs/content-_row_id.arrow.json | 2 +- .../chunk-00/xt$txs/metadata.arrow.json | 6 +- .../tables/foo/chunks/leaf-c00.arrow.json | 14 --- .../tables/hello/chunks/leaf-c00.arrow.json | 14 --- .../tables/world/chunks/leaf-c00.arrow.json | 14 --- .../tables/xt$txs/chunks/leaf-c00.arrow.json | 14 --- .../tables/xt$txs/chunks/leaf-c00.arrow.json | 14 --- .../tables/xt_docs/chunks/leaf-c00.arrow.json | 14 --- .../tables/table/chunks/leaf-c00.arrow.json | 14 --- .../tables/xt$txs/chunks/leaf-c00.arrow.json | 14 --- .../tables/xt$txs/chunks/leaf-c00.arrow.json | 14 --- .../tables/xt_docs/chunks/leaf-c00.arrow.json | 14 --- .../xt$txs/content-_row_id.arrow.json | 2 +- .../chunk-00/xt$txs/metadata.arrow.json | 6 +- .../xt_docs/content-_row_id.arrow.json | 2 +- .../chunk-00/xt_docs/metadata.arrow.json | 4 +- .../tables/xt$txs/chunks/leaf-c00.arrow.json | 14 --- .../tables/xt_docs/chunks/leaf-c00.arrow.json | 14 --- ...dynamic-parameters-103-update-app-time.edn | 34 +++--- ...ynamic-parameters-103-update-set-value.edn | 30 ++--- .../test-for-all-valid-time-387-delete.edn | 28 ++--- .../test-for-all-valid-time-387-update.edn | 37 ++---- .../test-sql-delete-plan.edn | 30 ++--- ...sql-update-plan-with-column-references.edn | 24 ++-- ...sql-update-plan-with-period-references.edn | 30 ++--- .../test-sql-update-plan.edn | 34 +++--- 44 files changed, 175 insertions(+), 573 deletions(-) delete mode 100644 core/src/main/clojure/xtdb/indexer/internal_id_manager.clj diff --git a/core/src/main/clojure/xtdb/indexer.clj b/core/src/main/clojure/xtdb/indexer.clj index 7a6e36acc7..4ad8bf483d 100644 --- a/core/src/main/clojure/xtdb/indexer.clj +++ b/core/src/main/clojure/xtdb/indexer.clj @@ -4,7 +4,6 @@ [sci.core :as sci] [xtdb.datalog :as d] [xtdb.error :as err] - xtdb.indexer.internal-id-manager xtdb.indexer.live-index xtdb.live-chunk [xtdb.metadata :as meta] @@ -35,7 +34,6 @@ (org.apache.arrow.vector.ipc ArrowStreamReader) xtdb.RowCounter (xtdb.api.protocols ClojureForm TransactionInstant) - xtdb.indexer.internal_id_manager.IInternalIdManager (xtdb.indexer.live_index ILiveIndex ILiveIndexTx) (xtdb.live_chunk ILiveChunk ILiveChunkTx ILiveTableTx) xtdb.metadata.IMetadataManager @@ -71,20 +69,20 @@ (MessageDigest/getInstance "SHA-256"))))) (defn- ->iid ^bytes [eid] - (if (uuid? eid) - (util/uuid->bytes eid) - - (let [^bytes eid-bytes (cond - (string? eid) (.getBytes (str "s" eid)) - (keyword? eid) (.getBytes (str "k" eid)) - (integer? eid) (.getBytes (str "i" eid)) - :else (throw (UnsupportedOperationException. (pr-str (class eid)))))] - (-> ^MessageDigest (.get !msg-digest) - (.digest eid-bytes) - (Arrays/copyOfRange 0 16))))) - -(defn- ->put-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^RowCounter row-counter, - ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk, + (ByteBuffer/wrap + (if (uuid? eid) + (util/uuid->bytes eid) + + (let [^bytes eid-bytes (cond + (string? eid) (.getBytes (str "s" eid)) + (keyword? eid) (.getBytes (str "k" eid)) + (integer? eid) (.getBytes (str "i" eid)) + :else (throw (UnsupportedOperationException. (pr-str (class eid)))))] + (-> ^MessageDigest (.get !msg-digest) + (.digest eid-bytes) + (Arrays/copyOfRange 0 16)))))) + +(defn- ->put-indexer ^xtdb.indexer.OpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk, ^IVectorReader tx-ops-rdr, ^Instant system-time] (let [put-leg (.legReader tx-ops-rdr :put) doc-rdr (.structKeyReader put-leg "document") @@ -115,7 +113,7 @@ (indexOp [_ tx-op-idx] (let [row-id (.nextRowId live-chunk) - {:keys [table-name, ^IVectorReader id-rdr, live-chunk-table, live-idx-table]} + {:keys [^IVectorReader id-rdr, live-chunk-table, live-idx-table]} (nth tables (.getTypeId doc-rdr tx-op-idx)) eid (.getObject id-rdr tx-op-idx)] @@ -124,8 +122,7 @@ (.writeRowId live-table row-id) (.copyRow table-copier tx-op-idx)) - (let [legacy-iid (.getOrCreateInternalId iid-mgr table-name eid row-id) - valid-from (if (= :null (.getLeg valid-from-rdr tx-op-idx)) + (let [valid-from (if (= :null (.getLeg valid-from-rdr tx-op-idx)) system-time-µs (.getLong valid-from-rdr tx-op-idx)) valid-to (if (= :null (.getLeg valid-to-rdr tx-op-idx)) @@ -137,14 +134,13 @@ :valid-to (util/micros->instant valid-to)}))) (let [{:keys [^xtdb.indexer.live_index.ILiveTableTx live-table, ^IRowCopier doc-copier]} live-idx-table] - (.logPut live-table (->iid eid) legacy-iid valid-from valid-to #(.copyRow doc-copier tx-op-idx)))) + (.logPut live-table (->iid eid) valid-from valid-to #(.copyRow doc-copier tx-op-idx)))) (.addRows row-counter 1)) nil)))) -(defn- ->delete-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^RowCounter row-counter, - ^ILiveIndexTx live-idx-tx ^ILiveChunkTx live-chunk +(defn- ->delete-indexer ^xtdb.indexer.OpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx ^ILiveChunkTx live-chunk ^IVectorReader tx-ops-rdr, ^Instant current-time] (let [delete-leg (.legReader tx-ops-rdr :delete) table-rdr (.structKeyReader delete-leg "table") @@ -154,10 +150,8 @@ current-time-µs (util/instant->micros current-time)] (reify OpIndexer (indexOp [_ tx-op-idx] - (let [row-id (.nextRowId live-chunk) - table (.getObject table-rdr tx-op-idx) + (let [table (.getObject table-rdr tx-op-idx) eid (.getObject id-rdr tx-op-idx) - legacy-iid (.getOrCreateInternalId iid-mgr table eid row-id) valid-from (if (= :null (.getLeg valid-from-rdr tx-op-idx)) current-time-µs (.getLong valid-from-rdr tx-op-idx)) @@ -170,13 +164,13 @@ :valid-to (util/micros->instant valid-to)}))) (-> (.liveTable live-idx-tx table) - (.logDelete (->iid eid) legacy-iid valid-from valid-to)) + (.logDelete (->iid eid) valid-from valid-to)) (.addRows row-counter 1)) nil)))) -(defn- ->evict-indexer ^xtdb.indexer.OpIndexer [^IInternalIdManager iid-mgr, ^RowCounter row-counter, ^ILiveIndexTx live-idx-tx ^ILiveChunkTx live-chunk ^IVectorReader tx-ops-rdr] +(defn- ->evict-indexer ^xtdb.indexer.OpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx ^ILiveChunkTx live-chunk ^IVectorReader tx-ops-rdr] (let [evict-leg (.legReader tx-ops-rdr :evict) table-rdr (.structKeyReader evict-leg "_table") @@ -185,11 +179,10 @@ (indexOp [_ tx-op-idx] (let [row-id (.nextRowId live-chunk) table (.getObject table-rdr tx-op-idx) - eid (.getObject id-rdr tx-op-idx) - legacy-iid (.getOrCreateInternalId iid-mgr table eid row-id)] + eid (.getObject id-rdr tx-op-idx)] (-> (.liveTable live-idx-tx table) - (.logEvict (->iid eid) legacy-iid)) + (.logEvict (->iid eid))) (.addRows row-counter 1)) @@ -307,8 +300,7 @@ (definterface SqlOpIndexer (^void indexOp [^xtdb.vector.RelationReader inRelation, queryOpts])) -(defn- ->sql-upsert-indexer ^xtdb.indexer.SqlOpIndexer [^IInternalIdManager iid-mgr, ^RowCounter row-counter, - ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk, +(defn- ->sql-upsert-indexer ^xtdb.indexer.SqlOpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk, {{:keys [^Instant current-time]} :basis}] (let [current-time-µs (util/instant->micros current-time)] @@ -339,7 +331,6 @@ (.copyRow table-copier idx) (let [eid (.getObject id-col idx) - legacy-iid (.getOrCreateInternalId iid-mgr table eid row-id) valid-from (if (and valid-from-rdr (= :timestamp-tz-micro-utc (.getLeg valid-from-rdr idx))) (.getLong valid-from-ts-rdr idx) current-time-µs) @@ -351,7 +342,7 @@ {:valid-from (util/micros->instant valid-from) :valid-to (util/micros->instant valid-to)}))) - (.logPut live-idx-table (->iid eid) legacy-iid valid-from valid-to #(.copyRow live-idx-table-copier idx))))) + (.logPut live-idx-table (->iid eid) valid-from valid-to #(.copyRow live-idx-table-copier idx))))) (.addRows row-counter row-count)))))) @@ -360,13 +351,11 @@ (indexOp [_ in-rel {:keys [table]}] (let [table (util/str->normal-form-str table) row-count (.rowCount in-rel) - id-rdr (.readerForName in-rel "xt$id") - iid-rdr (.readerForName in-rel "_iid") + iid-rdr (.readerForName in-rel "xt$iid") valid-from-rdr (.readerForName in-rel "xt$valid_from") valid-to-rdr (.readerForName in-rel "xt$valid_to")] (dotimes [idx row-count] - (let [eid (.getObject id-rdr idx) - iid (.getLong iid-rdr idx) + (let [iid (.getBytes iid-rdr idx) valid-from (.getLong valid-from-rdr idx) valid-to (.getLong valid-to-rdr idx)] (when (> valid-from valid-to) @@ -375,7 +364,7 @@ :valid-to (util/micros->instant valid-to)}))) (-> (.liveTable live-idx-tx table) - (.logDelete (->iid eid) iid valid-from valid-to)))) + (.logDelete iid valid-from valid-to)))) (.addRows row-counter row-count))))) @@ -384,24 +373,21 @@ (indexOp [_ in-rel {:keys [table]}] (let [table (util/str->normal-form-str table) row-count (.rowCount in-rel) - id-rdr (.readerForName in-rel "xt$id") - iid-rdr (.readerForName in-rel "_iid")] + iid-rdr (.readerForName in-rel "xt$iid")] (dotimes [idx row-count] - (let [eid (.getObject id-rdr idx) - iid (.getLong iid-rdr idx)] + (let [iid (.getBytes iid-rdr idx)] (-> (.liveTable live-idx-tx table) - (.logEvict (->iid eid) iid)))) + (.logEvict iid)))) (.addRows row-counter row-count))))) -(defn- ->sql-indexer ^xtdb.indexer.OpIndexer [^BufferAllocator allocator, ^IInternalIdManager iid-mgr, ^RowCounter row-counter - ^ILiveIndexTx live-idx-tx, ^ILiveChunk doc-idxer +(defn- ->sql-indexer ^xtdb.indexer.OpIndexer [^BufferAllocator allocator, ^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^ILiveChunk doc-idxer ^IVectorReader tx-ops-rdr, ^IRaQuerySource ra-src, wm-src, ^IScanEmitter scan-emitter {:keys [default-all-valid-time? basis default-tz] :as tx-opts}] (let [sql-leg (.legReader tx-ops-rdr :sql) query-rdr (.structKeyReader sql-leg "query") params-rdr (.structKeyReader sql-leg "params") - upsert-idxer (->sql-upsert-indexer iid-mgr row-counter live-idx-tx doc-idxer tx-opts) + upsert-idxer (->sql-upsert-indexer row-counter live-idx-tx doc-idxer tx-opts) delete-idxer (->sql-delete-indexer row-counter live-idx-tx) erase-idxer (->sql-erase-indexer row-counter live-idx-tx)] (reify OpIndexer @@ -461,16 +447,15 @@ (def ^:private ^:const ^String txs-table "xt$txs") -(defn- add-tx-row! [^IInternalIdManager iid-mgr, ^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk-tx, ^TransactionInstant tx-key, ^Throwable t] +(defn- add-tx-row! [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk-tx, ^TransactionInstant tx-key, ^Throwable t] (let [tx-id (.tx-id tx-key) system-time-µs (util/instant->micros (.system-time tx-key)) - row-id (.nextRowId live-chunk-tx) - legacy-iid (.getOrCreateInternalId iid-mgr txs-table tx-id row-id)] + row-id (.nextRowId live-chunk-tx)] (let [live-table (.liveTable live-idx-tx txs-table) doc-writer (.docWriter live-table)] - (.logPut live-table (->iid tx-id) legacy-iid system-time-µs util/end-of-time-μs + (.logPut live-table (->iid tx-id) system-time-µs util/end-of-time-μs (fn write-doc! [] (.startStruct doc-writer) (doto (.structKeyWriter doc-writer "xt$id" :i64) @@ -517,7 +502,6 @@ ^ObjectStore object-store ^IMetadataManager metadata-mgr ^IScanEmitter scan-emitter - ^IInternalIdManager iid-mgr ^IRaQuerySource ra-src ^ILiveChunk live-chunk ^ILiveIndex live-idx @@ -549,11 +533,11 @@ (letfn [(index-tx-ops [^DenseUnionVector tx-ops-vec] (let [tx-ops-rdr (vr/vec->reader tx-ops-vec) - !put-idxer (delay (->put-indexer iid-mgr row-counter live-idx-tx live-chunk-tx tx-ops-rdr system-time)) - !delete-idxer (delay (->delete-indexer iid-mgr row-counter live-idx-tx live-chunk-tx tx-ops-rdr system-time)) - !evict-idxer (delay (->evict-indexer iid-mgr row-counter live-idx-tx live-chunk-tx tx-ops-rdr)) + !put-idxer (delay (->put-indexer row-counter live-idx-tx live-chunk-tx tx-ops-rdr system-time)) + !delete-idxer (delay (->delete-indexer row-counter live-idx-tx live-chunk-tx tx-ops-rdr system-time)) + !evict-idxer (delay (->evict-indexer row-counter live-idx-tx live-chunk-tx tx-ops-rdr)) !call-idxer (delay (->call-indexer allocator ra-src wm-src scan-emitter tx-ops-rdr tx-opts)) - !sql-idxer (delay (->sql-indexer allocator iid-mgr row-counter live-idx-tx live-chunk-tx + !sql-idxer (delay (->sql-indexer allocator row-counter live-idx-tx live-chunk-tx tx-ops-rdr ra-src wm-src scan-emitter tx-opts))] (dotimes [tx-op-idx (.valueCount tx-ops-rdr)] (when-let [more-tx-ops (case (.getTypeId tx-ops-rdr tx-op-idx) @@ -587,12 +571,12 @@ (with-open [live-chunk-tx (.startTx live-chunk) live-idx-tx (.startTx live-idx tx-key)] - (add-tx-row! iid-mgr row-counter live-idx-tx live-chunk-tx tx-key e) + (add-tx-row! row-counter live-idx-tx live-chunk-tx tx-key e) (.commit live-chunk-tx) (.commit live-idx-tx))) (do - (add-tx-row! iid-mgr row-counter live-idx-tx live-chunk-tx tx-key nil) + (add-tx-row! row-counter live-idx-tx live-chunk-tx tx-key nil) (.commit live-chunk-tx) (.commit live-idx-tx))) @@ -696,7 +680,6 @@ :object-store (ig/ref :xtdb/object-store) :metadata-mgr (ig/ref ::meta/metadata-manager) :scan-emitter (ig/ref :xtdb.operator.scan/scan-emitter) - :internal-id-mgr (ig/ref :xtdb.indexer/internal-id-manager) :live-chunk (ig/ref :xtdb/live-chunk) :live-index (ig/ref :xtdb.indexer/live-index) :ra-src (ig/ref ::op/ra-query-source) @@ -705,13 +688,10 @@ (defmethod ig/init-key :xtdb/indexer [_ {:keys [allocator object-store metadata-mgr scan-emitter, ra-src - internal-id-mgr live-chunk live-index, rows-per-chunk]}] + live-chunk live-index, rows-per-chunk]}] (let [{:keys [latest-completed-tx next-chunk-idx], :or {next-chunk-idx 0}} (meta/latest-chunk-metadata metadata-mgr)] - - (assert live-chunk) - (->Indexer allocator object-store metadata-mgr scan-emitter internal-id-mgr - ra-src live-chunk live-index + (->Indexer allocator object-store metadata-mgr scan-emitter ra-src live-chunk live-index latest-completed-tx (RowCounter. next-chunk-idx) diff --git a/core/src/main/clojure/xtdb/indexer/internal_id_manager.clj b/core/src/main/clojure/xtdb/indexer/internal_id_manager.clj deleted file mode 100644 index 8ea4d6388f..0000000000 --- a/core/src/main/clojure/xtdb/indexer/internal_id_manager.clj +++ /dev/null @@ -1,67 +0,0 @@ -(ns xtdb.indexer.internal-id-manager - (:require [juxt.clojars-mirrors.integrant.core :as ig] - [xtdb.buffer-pool] - [xtdb.metadata :as meta] - [xtdb.util :as util] - [xtdb.vector.reader :as vr]) - (:import (java.io Closeable) - java.nio.ByteBuffer - (java.util Map) - (java.util.concurrent ConcurrentHashMap) - (java.util.function Consumer Function) - xtdb.buffer_pool.IBufferPool - xtdb.metadata.IMetadataManager)) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface IInternalIdManager - (^long getOrCreateInternalId [^String table, ^Object id, ^long row-id]) - (^boolean isKnownId [^String table, ^Object id])) - -(defn- normalize-id [id] - (cond-> id - (bytes? id) (ByteBuffer/wrap))) - -(defmethod ig/prep-key :xtdb.indexer/internal-id-manager [_ opts] - (merge {:metadata-mgr (ig/ref ::meta/metadata-manager) - :buffer-pool (ig/ref :xtdb.buffer-pool/buffer-pool) - :allocator (ig/ref :xtdb/allocator)} - opts)) - -(deftype InternalIdManager [^Map id->internal-id] - IInternalIdManager - (getOrCreateInternalId [_ table id row-id] - (.computeIfAbsent id->internal-id - [table (normalize-id id)] - (reify Function - (apply [_ _] - ;; big endian for index distribution - (Long/reverseBytes row-id))))) - - (isKnownId [_ table id] - (.containsKey id->internal-id [table (normalize-id id)])) - - Closeable - (close [_] - (.clear id->internal-id))) - -(defmethod ig/init-key :xtdb.indexer/internal-id-manager [_ {:keys [^IBufferPool buffer-pool, ^IMetadataManager metadata-mgr]}] - (let [iid-mgr (InternalIdManager. (ConcurrentHashMap.))] - (doseq [[chunk-idx chunk-metadata] (.chunksMetadata metadata-mgr) - table (keys (:tables chunk-metadata))] - (with-open [id-chunks (-> @(.getBuffer buffer-pool (meta/->chunk-obj-key chunk-idx table "xt$id")) - (util/rethrowing-cause) - (util/->chunks {:close-buffer? true})) - row-id-chunks (-> @(.getBuffer buffer-pool (meta/->chunk-obj-key chunk-idx table "_row_id")) - (util/->chunks {:close-buffer? true}))] - (-> (util/combine-col-cursors {"_row_id" row-id-chunks, "xt$id" id-chunks}) - (.forEachRemaining - (reify Consumer - (accept [_ root] - (let [rel (vr/<-root root) - id-rdr (.readerForName rel "xt$id") - row-id-rdr (.readerForName rel "_row_id")] - (dotimes [idx (.rowCount rel)] - (.getOrCreateInternalId iid-mgr table - (.getObject id-rdr idx) - (.getLong row-id-rdr idx)))))))))) - iid-mgr)) diff --git a/core/src/main/clojure/xtdb/indexer/live_index.clj b/core/src/main/clojure/xtdb/indexer/live_index.clj index a74572412a..76437271e5 100644 --- a/core/src/main/clojure/xtdb/indexer/live_index.clj +++ b/core/src/main/clojure/xtdb/indexer/live_index.clj @@ -31,9 +31,9 @@ (definterface ILiveTableTx (^xtdb.indexer.live_index.ILiveTableWatermark openWatermark [^boolean retain]) (^xtdb.vector.IVectorWriter docWriter []) - (^void logPut [^bytes iid, ^long legacyIid, ^long validFrom, ^long validTo, writeDocFn]) - (^void logDelete [^bytes iid, ^long legacyIid, ^long validFrom, ^long validTo]) - (^void logEvict [^bytes iid, ^long legacyIid]) + (^void logPut [^java.nio.ByteBuffer iid, ^long validFrom, ^long validTo, writeDocFn]) + (^void logDelete [^java.nio.ByteBuffer iid, ^long validFrom, ^long validTo]) + (^void logEvict [^java.nio.ByteBuffer iid]) (^void commit []) (^void close [])) @@ -93,7 +93,7 @@ (deftype LiveTable [^BufferAllocator allocator, ^ObjectStore obj-store, ^String table-name ^IRelationWriter live-rel, ^:unsynchronized-mutable ^LiveHashTrie live-trie - ^IVectorWriter iid-wtr, ^IVectorWriter legacy-iid-wtr, ^IVectorWriter system-from-wtr + ^IVectorWriter iid-wtr, ^IVectorWriter system-from-wtr ^IVectorWriter put-wtr, ^IVectorWriter put-valid-from-wtr, ^IVectorWriter put-valid-to-wtr, ^IVectorWriter put-doc-wtr ^IVectorWriter delete-wtr, ^IVectorWriter delete-valid-from-wtr, ^IVectorWriter delete-valid-to-wtr ^IVectorWriter evict-wtr] @@ -104,11 +104,10 @@ (reify ILiveTableTx (docWriter [_] put-doc-wtr) - (logPut [_ iid legacy-iid valid-from valid-to write-doc!] + (logPut [_ iid valid-from valid-to write-doc!] (.startRow live-rel) - (.writeBytes iid-wtr (ByteBuffer/wrap iid)) - (.writeLong legacy-iid-wtr legacy-iid) + (.writeBytes iid-wtr iid) (.writeLong system-from-wtr system-from-µs) (.startStruct put-wtr) @@ -121,9 +120,8 @@ (swap! !transient-trie #(.add ^LiveHashTrie % (dec (.getPosition (.writerPosition live-rel)))))) - (logDelete [_ iid legacy-iid valid-from valid-to] - (.writeBytes iid-wtr (ByteBuffer/wrap iid)) - (.writeLong legacy-iid-wtr legacy-iid) + (logDelete [_ iid valid-from valid-to] + (.writeBytes iid-wtr iid) (.writeLong system-from-wtr system-from-µs) (.startStruct delete-wtr) @@ -135,9 +133,8 @@ (swap! !transient-trie #(.add ^LiveHashTrie % (dec (.getPosition (.writerPosition live-rel)))))) - (logEvict [_ iid legacy-iid] - (.writeBytes iid-wtr (ByteBuffer/wrap iid)) - (.writeLong legacy-iid-wtr legacy-iid) + (logEvict [_ iid] + (.writeBytes iid-wtr iid) (.writeLong system-from-wtr system-from-µs) (.writeNull evict-wtr nil) @@ -209,7 +206,7 @@ delete-wtr (.writerForField op-wtr trie/delete-field)] (->LiveTable allocator object-store table-name rel (LiveHashTrie/emptyTrie (.getVector iid-wtr)) - iid-wtr (.writerForName rel "xt$legacy_iid") (.writerForName rel "xt$system_from") + iid-wtr (.writerForName rel "xt$system_from") put-wtr (.structKeyWriter put-wtr "xt$valid_from") (.structKeyWriter put-wtr "xt$valid_to") (.structKeyWriter put-wtr "xt$doc") delete-wtr (.structKeyWriter delete-wtr "xt$valid_from") (.structKeyWriter delete-wtr "xt$valid_to") (.writerForField op-wtr trie/evict-field))))) diff --git a/core/src/main/clojure/xtdb/live_chunk.clj b/core/src/main/clojure/xtdb/live_chunk.clj index 33eab273dd..d1285578bd 100644 --- a/core/src/main/clojure/xtdb/live_chunk.clj +++ b/core/src/main/clojure/xtdb/live_chunk.clj @@ -3,7 +3,6 @@ [juxt.clojars-mirrors.integrant.core :as ig] [xtdb.blocks :as blocks] [xtdb.bloom :as bloom] - xtdb.indexer.internal-id-manager [xtdb.metadata :as meta] xtdb.object-store [xtdb.types :as types] diff --git a/core/src/main/clojure/xtdb/node.clj b/core/src/main/clojure/xtdb/node.clj index 32d7ece725..d7b8fcfd0e 100644 --- a/core/src/main/clojure/xtdb/node.clj +++ b/core/src/main/clojure/xtdb/node.clj @@ -124,7 +124,6 @@ :xtdb/allocator {} :xtdb/default-tz nil :xtdb/indexer {} - :xtdb.indexer/internal-id-manager {} :xtdb/live-chunk {} :xtdb.indexer/live-index {} :xtdb/ingester {} diff --git a/core/src/main/clojure/xtdb/operator/scan.clj b/core/src/main/clojure/xtdb/operator/scan.clj index 4cfa557699..10125759a7 100644 --- a/core/src/main/clojure/xtdb/operator/scan.clj +++ b/core/src/main/clojure/xtdb/operator/scan.clj @@ -218,7 +218,7 @@ (for [col-name col-names :let [normalized-name (util/str->normal-form-str col-name) ^IVectorReader rdr (case normalized-name - "_iid" (.readerForName leaf-rel "xt$legacy_iid") + "xt$iid" (.readerForName leaf-rel "xt$iid") "xt$system_from" sys-from-rdr "xt$system_to" (vr/vec->reader (doto (NullVector. "xt$system_to") @@ -229,7 +229,7 @@ :when rdr] (.rowCopier rdr (case normalized-name - "_iid" (.writerForName out-rel col-name :i64) + "xt$iid" (.writerForName out-rel col-name [:fixed-size-binary 16]) "xt$system_from" (.writerForName out-rel col-name types/temporal-col-type) "xt$system_to" (.writerForName out-rel col-name [:union #{:null types/temporal-col-type}]) @@ -342,7 +342,7 @@ (for [col-name col-names :let [normalized-name (util/str->normal-form-str col-name) ^IVectorReader rdr (case normalized-name - "_iid" (.readerForName leaf-rel "xt$legacy_iid") + "xt$iid" (.readerForName leaf-rel "xt$iid") "xt$system_from" nil "xt$system_to" (vr/vec->reader (doto (NullVector. "xt$system_to") @@ -353,7 +353,7 @@ :when rdr] (.rowCopier rdr (case normalized-name - "_iid" (.writerForName out-rel col-name :i64) + "xt$iid" (.writerForName out-rel col-name [:fixed-size-binary 16]) "xt$system_to" (.writerForName out-rel col-name [:union #{:null types/temporal-col-type}]) (.writerForName out-rel col-name))))) @@ -498,7 +498,7 @@ (for [col-name col-names :let [normalized-name (util/str->normal-form-str col-name) ^IVectorReader rdr (case normalized-name - "_iid" (.readerForName leaf-rel "xt$legacy_iid") + "xt$iid" (.readerForName leaf-rel "xt$iid") "xt$system_from" nil "xt$system_to" nil "xt$valid_from" nil @@ -507,7 +507,7 @@ :when rdr] (.rowCopier rdr (case normalized-name - "_iid" (.writerForName out-rel col-name :i64) + "xt$iid" (.writerForName out-rel col-name [:fixed-size-binary 16]) (.writerForName out-rel col-name))))) valid-from-wtrs (vec @@ -804,7 +804,7 @@ {content-col-names false, temporal-col-names true} (->> col-names (group-by (comp types/temporal-column? util/str->normal-form-str str))) - content-col-names (-> (set (map str content-col-names)) (conj "_row_id")) + content-col-names (set (map str content-col-names)) temporal-col-names (into #{} (map (comp str)) temporal-col-names) normalized-table-name (util/str->normal-form-str (str table)) diff --git a/core/src/main/clojure/xtdb/sql/analyze.clj b/core/src/main/clojure/xtdb/sql/analyze.clj index 334bdf9d40..e099af4f67 100644 --- a/core/src/main/clojure/xtdb/sql/analyze.clj +++ b/core/src/main/clojure/xtdb/sql/analyze.clj @@ -661,11 +661,7 @@ :when (= table-id column-table-id)] (last identifiers)) - ;; TODO _iid (and possibly _row_id) can go once we remove the old temporal-indexer - ;; also see below - #{"xt$id" "_iid" "_row_id" - "xt$valid_from" "xt$valid_to" - "xt$system_from" "xt$system_to"} + #{"xt$iid" "xt$valid_from" "xt$valid_to" "xt$system_from" "xt$system_to"} (when (= :update_statement__searched (r/ctor (r/parent ag))) (set/difference known-columns @@ -695,9 +691,7 @@ [(let [{:keys [correlation-name known-columns], :as table} (table ag) updated-cols (update-set-cols ag)] (vec - (concat (->> (for [col-name (set/union #{"xt$id" "_iid" "_row_id" - "xt$valid_from" "xt$valid_to" - "xt$system_from" "xt$system_to"} + (concat (->> (for [col-name (set/union #{"xt$iid" "xt$valid_from" "xt$valid_to" "xt$system_from" "xt$system_to"} (when (= :update_statement__searched (r/ctor ag)) (set/difference known-columns (into #{} (map :identifier) updated-cols))))] @@ -708,9 +702,7 @@ :erase_statement__searched [(let [{:keys [correlation-name], :as table} (table ag)] - (->> (for [col-name ["xt$id" "_iid" "_row_id" - "xt$valid_from" "xt$valid_to" - "xt$system_from" "xt$system_to"]] + (->> (for [col-name ["xt$iid" "xt$valid_from" "xt$valid_to" "xt$system_from" "xt$system_to"]] {:identifier col-name :qualified-column [correlation-name col-name]}) (into [] (map #(vary-meta % assoc :table table)))))] diff --git a/core/src/main/clojure/xtdb/trie.clj b/core/src/main/clojure/xtdb/trie.clj index fcb471c7da..9e80de7ffd 100644 --- a/core/src/main/clojure/xtdb/trie.clj +++ b/core/src/main/clojure/xtdb/trie.clj @@ -38,7 +38,6 @@ (def ^org.apache.arrow.vector.types.pojo.Schema log-leaf-schema (Schema. [(types/col-type->field "xt$iid" [:fixed-size-binary 16]) - (types/col-type->field "xt$legacy_iid" :i64) (types/col-type->field "xt$system_from" types/temporal-col-type) (types/->field "op" (ArrowType$Union. UnionMode/Dense (int-array (range 3))) false put-field delete-field evict-field)])) diff --git a/core/src/main/clojure/xtdb/types.clj b/core/src/main/clojure/xtdb/types.clj index a8d5de732a..fb58858893 100644 --- a/core/src/main/clojure/xtdb/types.clj +++ b/core/src/main/clojure/xtdb/types.clj @@ -19,7 +19,7 @@ (def nullable-temporal-type [:union #{:null temporal-col-type}]) (def temporal-col-types - {"_iid" :i64, "_row_id" :i64 + {"xt$iid" [:fixed-size-binary 16], "xt$system_from" temporal-col-type, "xt$system_to" temporal-col-type "xt$valid_from" temporal-col-type, "xt$valid_to" temporal-col-type}) diff --git a/src/test/clojure/xtdb/indexer/live_index_test.clj b/src/test/clojure/xtdb/indexer/live_index_test.clj index 6c40ccaba1..37691b7518 100644 --- a/src/test/clojure/xtdb/indexer/live_index_test.clj +++ b/src/test/clojure/xtdb/indexer/live_index_test.clj @@ -9,7 +9,8 @@ [xtdb.test-util :as tu] [xtdb.util :as util] [xtdb.vector.reader :as vr]) - (:import java.time.Duration + (:import [java.nio ByteBuffer] + java.time.Duration [java.util Random UUID] [org.apache.arrow.memory BufferAllocator RootAllocator] [org.apache.arrow.vector FixedSizeBinaryVector] @@ -42,7 +43,7 @@ live-table-tx (.liveTable live-idx-tx "my-table")] (let [wp (IVectorPosition/build)] (doseq [^UUID iid iids] - (.logPut live-table-tx (util/uuid->bytes iid) (.getMostSignificantBits iid) 0 0 + (.logPut live-table-tx (ByteBuffer/wrap (util/uuid->bytes iid)) 0 0 #(.getPositionAndIncrement wp)))) (.commit live-idx-tx) diff --git a/src/test/clojure/xtdb/indexer_test.clj b/src/test/clojure/xtdb/indexer_test.clj index ea5bcbcf4e..565b592ce1 100644 --- a/src/test/clojure/xtdb/indexer_test.clj +++ b/src/test/clojure/xtdb/indexer_test.clj @@ -9,7 +9,6 @@ [xtdb.buffer-pool :as bp] [xtdb.expression.metadata :as expr.meta] [xtdb.indexer :as idx] - xtdb.indexer.internal-id-manager [xtdb.metadata :as meta] [xtdb.node :as node] [xtdb.object-store :as os] @@ -28,7 +27,6 @@ [org.apache.arrow.vector.complex ListVector StructVector] xtdb.api.protocols.TransactionInstant [xtdb.buffer_pool BufferPool IBufferPool] - (xtdb.indexer.internal_id_manager InternalIdManager) (xtdb.metadata IMetadataManager) xtdb.node.Node xtdb.object_store.ObjectStore @@ -488,7 +486,6 @@ (with-open [node (tu/->local-node (assoc node-opts :buffers-dir "buffers-1"))] (let [^ObjectStore os (util/component node ::os/file-system-object-store) - ^InternalIdManager iid-mgr (util/component node :xtdb.indexer/internal-id-manager) ^IMetadataManager mm (util/component node ::meta/metadata-manager)] (t/is (= first-half-tx-key (-> first-half-tx-key @@ -505,9 +502,7 @@ (t/is (= 2 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_info/metadata\.arrow" %) objs)))) (t/is (= 5 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_readings/metadata\.arrow" %) objs)))) (t/is (= 2 (count (filter #(re-matches #"chunk-.*/device_info/content-api_version\.arrow" %) objs)))) - (t/is (= 5 (count (filter #(re-matches #"chunk-.*/device_readings/content-battery_level\.arrow" %) objs))))) - - (t/is (= 2055 (count (.id->internal-id iid-mgr))))) + (t/is (= 5 (count (filter #(re-matches #"chunk-.*/device_readings/content-battery_level\.arrow" %) objs)))))) (t/is (= :utf8 (.columnType mm "device_readings" "xt$id"))) @@ -531,8 +526,6 @@ (tu/then-await-tx node (Duration/ofSeconds 10)))) (:tx-id second-half-tx-key))) - (t/is (>= (count (.id->internal-id iid-mgr)) 2055)) - (t/is (= :utf8 (.columnType mm "device_info" "xt$id")))) (doseq [^Node node [new-node node]] @@ -550,9 +543,7 @@ (t/is (= 2 (count (filter #(re-matches #"chunk-.*/device_info/content-api_version\.arrow" %) objs)))) (t/is (= 11 (count (filter #(re-matches #"chunk-.*/device_readings/content-battery_level\.arrow" %) objs))))) - (t/is (= :utf8 (.columnType mm "device_info" "xt$id"))) - - (t/is (= 2110 (count (.id->internal-id iid-mgr)))))))))))))) + (t/is (= :utf8 (.columnType mm "device_info" "xt$id"))))))))))))) (t/deftest merges-column-fields-on-restart (let [node-dir (util/->path "target/merges-column-fields") diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/tables/device_info/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/tables/device_info/chunks/leaf-c00.arrow.json index c063de4a41..ed38e63d0b 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/tables/device_info/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/tables/device_info/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -188,11 +179,6 @@ "count" : 2, "VALIDITY" : [1,1], "DATA" : ["248daaa010bf702848523e4fa63f996c","ef4f71005524e9af20ffaca545cde6e1"] - },{ - "name" : "xt$legacy_iid", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["216172782113783808","0"] },{ "name" : "xt$system_from", "count" : 2, diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/tables/device_readings/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/tables/device_readings/chunks/leaf-c00.arrow.json index be685cddcc..8ee9f1d213 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/tables/device_readings/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/tables/device_readings/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -342,11 +333,6 @@ "count" : 2, "VALIDITY" : [1,1], "DATA" : ["58941814a63f68d05acf4177ec17d3ba","6bd5602dd9300d63410ca07b677f0041"] - },{ - "name" : "xt$legacy_iid", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["72057594037927936","288230376151711744"] },{ "name" : "xt$system_from", "count" : 2, diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/tables/xt$txs/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/tables/xt$txs/chunks/leaf-c00.arrow.json index ab43eebfc5..aae2410731 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/tables/xt$txs/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/tables/xt$txs/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -147,11 +138,6 @@ "count" : 2, "VALIDITY" : [1,1], "DATA" : ["6b2c5e88c38aa669787f711205999504","a4e167a76a05add8a8654c169b07b044"] - },{ - "name" : "xt$legacy_iid", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["360287970189639680","144115188075855872"] },{ "name" : "xt$system_from", "count" : 2, diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-_row_id.arrow.json index 3b6a94f68f..1060f12edf 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-_row_id.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-_row_id.arrow.json @@ -17,7 +17,7 @@ "name" : "_row_id", "count" : 3, "VALIDITY" : [1,1,1], - "DATA" : ["8","9","10"] + "DATA" : ["7","8","9"] }] }] } \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/metadata.arrow.json index 3844c65e4d..97b8a66dfb 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/metadata.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/metadata.arrow.json @@ -159,12 +159,12 @@ "name" : "min", "count" : 8, "VALIDITY" : [1,1,1,0,1,1,1,0], - "DATA" : ["8","1","1","0","8","1","1","0"] + "DATA" : ["7","1","1","0","7","1","1","0"] },{ "name" : "max", "count" : 8, "VALIDITY" : [1,1,1,0,1,1,1,0], - "DATA" : ["10","2","2","0","10","2","2","0"] + "DATA" : ["9","2","2","0","9","2","2","0"] }] },{ "name" : "utf8", @@ -193,8 +193,8 @@ "name" : "bloom", "count" : 8, "VALIDITY" : [1,1,1,1,1,1,1,1], - "OFFSET" : [0,82,134,186,238,320,372,424,476], - "DATA" : ["3a30000007000000000002000500000007000000080000000a0000000b0000000c0000004000000046000000480000004a0000004c0000004e00000050000000080009000a00bc2bb4c910296f57dee48d14","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a3000000400000000000200040000000a0000000f000000280000002e00000030000000320000001f003e005d00746f44e95cac","3a30000007000000000002000500000007000000080000000a0000000b0000000c0000004000000046000000480000004a0000004c0000004e00000050000000080009000a00bc2bb4c910296f57dee48d14","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a3000000400000000000200040000000a0000000f000000280000002e00000030000000320000001f003e005d00746f44e95cac"] + "OFFSET" : [0,66,118,170,222,288,340,392,444], + "DATA" : ["3a300000050000000000020005000100070000000a0000000b00010030000000360000003a0000003c0000003e000000070008000900bc2bce90b4c96f579521dee4","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a3000000400000000000200040000000a0000000f000000280000002e00000030000000320000001f003e005d00746f44e95cac","3a300000050000000000020005000100070000000a0000000b00010030000000360000003a0000003c0000003e000000070008000900bc2bce90b4c96f579521dee4","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a3000000400000000000200040000000a0000000f000000280000002e00000030000000320000001f003e005d00746f44e95cac"] }] }] }] diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-_row_id.arrow.json index c985d14f76..2dca7cbc33 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-_row_id.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-_row_id.arrow.json @@ -17,7 +17,7 @@ "name" : "_row_id", "count" : 2, "VALIDITY" : [1,1], - "DATA" : ["1","4"] + "DATA" : ["1","3"] }] }] } \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/metadata.arrow.json index dff064b31d..19f28b45fc 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/metadata.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/metadata.arrow.json @@ -171,7 +171,7 @@ "name" : "max", "count" : 6, "VALIDITY" : [1,1,0,1,1,0], - "DATA" : ["4","3","0","4","3","0"] + "DATA" : ["3","3","0","3","3","0"] }] },{ "name" : "uuid", @@ -193,8 +193,8 @@ "name" : "bloom", "count" : 6, "VALIDITY" : [1,1,1,1,1,1], - "OFFSET" : [0,52,104,134,186,238,268], - "DATA" : ["3a3000000400000000000200080000000d0000000e000000280000002e0000003000000032000000010004002bee16778e13c989","3a3000000400000000000100040000000a0000000f000100280000002c0000002e0000003000000002000300d2946a4a57c3ade1","3a30000002000000040001000c000000180000001c000000de6bdafe5cb5","3a3000000400000000000200080000000d0000000e000000280000002e0000003000000032000000010004002bee16778e13c989","3a3000000400000000000100040000000a0000000f000100280000002c0000002e0000003000000002000300d2946a4a57c3ade1","3a30000002000000040001000c000000180000001c000000de6bdafe5cb5"] + "OFFSET" : [0,44,96,126,170,222,252], + "DATA" : ["3a3000000300000000000200080000000f000100200000002600000028000000010003002bee167757c3ade1","3a3000000400000000000100040000000a0000000f000100280000002c0000002e0000003000000002000300d2946a4a57c3ade1","3a30000002000000040001000c000000180000001c000000de6bdafe5cb5","3a3000000300000000000200080000000f000100200000002600000028000000010003002bee167757c3ade1","3a3000000400000000000100040000000a0000000f000100280000002c0000002e0000003000000002000300d2946a4a57c3ade1","3a30000002000000040001000c000000180000001c000000de6bdafe5cb5"] }] }] }] diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-_row_id.arrow.json index b6c3229670..58a1920693 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-_row_id.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-_row_id.arrow.json @@ -17,7 +17,7 @@ "name" : "_row_id", "count" : 6, "VALIDITY" : [1,1,1,1,1,1], - "DATA" : ["2","5","7","11","12","13"] + "DATA" : ["2","4","6","10","11","12"] }] }] } \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/metadata.arrow.json index a9e14e8216..e44f54b5b5 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/metadata.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/metadata.arrow.json @@ -182,7 +182,7 @@ "name" : "max", "count" : 10, "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["13","18145","0","0","0","13","18145","0","0","0"] + "DATA" : ["12","18145","0","0","0","12","18145","0","0","0"] }] },{ "name" : "timestamp-tz-micro-utc", @@ -219,8 +219,8 @@ "name" : "bloom", "count" : 10, "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,116,224,356,384,436,552,660,792,820,872], - "DATA" : ["3a30000009000000000007000400000005000100090000000a0001000b0000000c0000000e0000000f00000050000000600000006200000066000000680000006c0000006e00000070000000720000000200050007000b000c000d00e014b429d2949e70ce9021c36a4a2fe1952196e1dbd6706b","3a300000080000000000070001000100020000000500000008000100090001000a0000000c00000048000000580000005c0000005e0000006000000064000000680000006a00000000006d115a21372ee43ae14667fa29fd97c606da8f8afc2a05a2b0fd5002fe4df8559a9e","3a3000000b0000000000000002000200040001000500010006000100090000000a0000000b0000000c0001000e0001000f0000006000000062000000680000006c000000700000007400000076000000780000007a0000007e00000082000000c402595b6f650af4582a1c3b57f945fbe42583504c016dbb93972a6e309bfa4e6e9008db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000009b003601d1018a6364dfb0e7","3a30000009000000000007000400000005000100090000000a0001000b0000000c0000000e0000000f00000050000000600000006200000066000000680000006c0000006e00000070000000720000000200050007000b000c000d00e014b429d2949e70ce9021c36a4a2fe1952196e1dbd6706b","3a300000080000000000070001000100020000000500000008000100090001000a0000000c00000048000000580000005c0000005e0000006000000064000000680000006a00000000006d115a21372ee43ae14667fa29fd97c606da8f8afc2a05a2b0fd5002fe4df8559a9e","3a3000000b0000000000000002000200040001000500010006000100090000000a0000000b0000000c0001000e0001000f0000006000000062000000680000006c000000700000007400000076000000780000007a0000007e00000082000000c402595b6f650af4582a1c3b57f945fbe42583504c016dbb93972a6e309bfa4e6e9008db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000009b003601d1018a6364dfb0e7"] + "OFFSET" : [0,124,232,364,392,444,568,676,808,836,888], + "DATA" : ["3a3000000a00000000000700040000000500000008000000090000000a0000000b0000000c0001000d0000000e00000058000000680000006a0000006c0000006e000000700000007200000074000000780000007a0000000200040006000a000b000c00e014b429d2947789102921c36a4ae8128d1496e18e13c989","3a300000080000000000070001000100020000000500000008000100090001000a0000000c00000048000000580000005c0000005e0000006000000064000000680000006a00000000006d115a21372ee43ae14667fa29fd97c606da8f8afc2a05a2b0fd5002fe4df8559a9e","3a3000000b0000000000000002000200040001000500010006000100090000000a0000000b0000000c0001000e0001000f0000006000000062000000680000006c000000700000007400000076000000780000007a0000007e00000082000000c402595b6f650af4582a1c3b57f945fbe42583504c016dbb93972a6e309bfa4e6e9008db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000009b003601d1018a6364dfb0e7","3a3000000a00000000000700040000000500000008000000090000000a0000000b0000000c0001000d0000000e00000058000000680000006a0000006c0000006e000000700000007200000074000000780000007a0000000200040006000a000b000c00e014b429d2947789102921c36a4ae8128d1496e18e13c989","3a300000080000000000070001000100020000000500000008000100090001000a0000000c00000048000000580000005c0000005e0000006000000064000000680000006a00000000006d115a21372ee43ae14667fa29fd97c606da8f8afc2a05a2b0fd5002fe4df8559a9e","3a3000000b0000000000000002000200040001000500010006000100090000000a0000000b0000000c0001000e0001000f0000006000000062000000680000006c000000700000007400000076000000780000007a0000007e00000082000000c402595b6f650af4582a1c3b57f945fbe42583504c016dbb93972a6e309bfa4e6e9008db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000009b003601d1018a6364dfb0e7"] }] }] }] diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/foo/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/foo/chunks/leaf-c00.arrow.json index 89d0377d25..7709e2fb5c 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/foo/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/foo/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -174,11 +165,6 @@ "count" : 5, "VALIDITY" : [1,1,1,1,1], "DATA" : ["420fce314175df402adbeae3cfbbb856","420fce314175df402adbeae3cfbbb856","4cd9b7672d7fbee8fb51fb1e049f6903","4cd9b7672d7fbee8fb51fb1e049f6903","4cd9b7672d7fbee8fb51fb1e049f6903"] - },{ - "name" : "xt$legacy_iid", - "count" : 5, - "VALIDITY" : [1,1,1,1,1], - "DATA" : ["720575940379279360","720575940379279360","576460752303423488","576460752303423488","576460752303423488"] },{ "name" : "xt$system_from", "count" : 5, diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/hello/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/hello/chunks/leaf-c00.arrow.json index 7100bc76fa..a6f25fa228 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/hello/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/hello/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -149,11 +140,6 @@ "count" : 2, "VALIDITY" : [1,1], "DATA" : ["cb8815ee85f74c61a8032ea1c949cf8d","cb8815ee85f74c61a8032ea1c949cf8d"] - },{ - "name" : "xt$legacy_iid", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["0","0"] },{ "name" : "xt$system_from", "count" : 2, diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/world/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/world/chunks/leaf-c00.arrow.json index c503724662..ee784b61da 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/world/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/world/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -149,11 +140,6 @@ "count" : 3, "VALIDITY" : [1,1,1], "DATA" : ["424f5622c8264deda5dbe2144d665c38","424f5622c8264deda5dbe2144d665c38","424f5622c8264deda5dbe2144d665c38"] - },{ - "name" : "xt$legacy_iid", - "count" : 3, - "VALIDITY" : [1,1,1], - "DATA" : ["72057594037927936","72057594037927936","72057594037927936"] },{ "name" : "xt$system_from", "count" : 3, diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/xt$txs/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/xt$txs/chunks/leaf-c00.arrow.json index 7e78438ce9..0840aa8784 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/xt$txs/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/tables/xt$txs/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -147,11 +138,6 @@ "count" : 6, "VALIDITY" : [1,1,1,1,1,1], "DATA" : ["2d9400fe884f154caa33862ea52eac95","4f109710492db95684cceb0d4f906c4a","657188d6a2a8753f6a2ef42bee7c12d8","88a681ae37728a5dd25693aaf1a8808e","a4e167a76a05add8a8654c169b07b044","d088c719d195af87f0012cb3a60c804c"] - },{ - "name" : "xt$legacy_iid", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : ["792633534417207296","864691128455135232","504403158265495552","360287970189639680","144115188075855872","936748722493063168"] },{ "name" : "xt$system_from", "count" : 6, diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/tables/xt$txs/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/tables/xt$txs/chunks/leaf-c00.arrow.json index 011f23d49d..5c2bea7fbd 100644 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/tables/xt$txs/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/tables/xt$txs/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -147,11 +138,6 @@ "count" : 1, "VALIDITY" : [1], "DATA" : ["a4e167a76a05add8a8654c169b07b044"] - },{ - "name" : "xt$legacy_iid", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["432345564227567616"] },{ "name" : "xt$system_from", "count" : 1, diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/tables/xt_docs/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/tables/xt_docs/chunks/leaf-c00.arrow.json index d1025b2751..42e6ba99f0 100644 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/tables/xt_docs/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/tables/xt_docs/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -332,11 +323,6 @@ "count" : 6, "VALIDITY" : [1,1,1,1,1,1], "DATA" : ["55bfc78e47207dc5125af00a5f52d66c","64b0cf833d08d23b08185c18bb7a0ef2","9e3f856e68998313827ff18dd4d88e78","bfc55eb61f526d86de90b2bb2e648a89","d9c7fae2a04e047164936265ba33cf80","fbfa9e45ee9bd2f827b8dde9e41d3814"] - },{ - "name" : "xt$legacy_iid", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : ["288230376151711744","72057594037927936","144115188075855872","216172782113783808","0","360287970189639680"] },{ "name" : "xt$system_from", "count" : 6, diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/tables/table/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/tables/table/chunks/leaf-c00.arrow.json index 6c7b490bbd..dae6d7b63f 100644 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/tables/table/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/tables/table/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -186,11 +177,6 @@ "count" : 2, "VALIDITY" : [1,1], "DATA" : ["4cd9b7672d7fbee8fb51fb1e049f6903","a4e167a76a05add8a8654c169b07b044"] - },{ - "name" : "xt$legacy_iid", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["72057594037927936","0"] },{ "name" : "xt$system_from", "count" : 2, diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/tables/xt$txs/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/tables/xt$txs/chunks/leaf-c00.arrow.json index a1cf5bf7a3..5c2bea7fbd 100644 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/tables/xt$txs/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/tables/xt$txs/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -147,11 +138,6 @@ "count" : 1, "VALIDITY" : [1], "DATA" : ["a4e167a76a05add8a8654c169b07b044"] - },{ - "name" : "xt$legacy_iid", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["144115188075855872"] },{ "name" : "xt$system_from", "count" : 1, diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/tables/xt$txs/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/tables/xt$txs/chunks/leaf-c00.arrow.json index be25d5e75f..d64cccea4c 100644 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/tables/xt$txs/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/multi-block-metadata/tables/xt$txs/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -147,11 +138,6 @@ "count" : 2, "VALIDITY" : [1,1], "DATA" : ["659fa87d49ccd11c201a9454d9336063","a4e167a76a05add8a8654c169b07b044"] - },{ - "name" : "xt$legacy_iid", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["504403158265495552","288230376151711744"] },{ "name" : "xt$system_from", "count" : 2, diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/tables/xt_docs/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/tables/xt_docs/chunks/leaf-c00.arrow.json index 6256d3be2f..8032ab142c 100644 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/tables/xt_docs/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/multi-block-metadata/tables/xt_docs/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -327,11 +318,6 @@ "count" : 6, "VALIDITY" : [1,1,1,1,1,1], "DATA" : ["47a6245d9effb01c6b67db10b5d9aaa8","55bfc78e47207dc5125af00a5f52d66c","64b0cf833d08d23b08185c18bb7a0ef2","d9c7fae2a04e047164936265ba33cf80","f4f7e2b0aae952281ff649b14ba5a6dc","fbfa9e45ee9bd2f827b8dde9e41d3814"] - },{ - "name" : "xt$legacy_iid", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : ["144115188075855872","360287970189639680","216172782113783808","0","72057594037927936","432345564227567616"] },{ "name" : "xt$system_from", "count" : 6, diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-_row_id.arrow.json index a13c0f3ad6..c5aafd432e 100644 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-_row_id.arrow.json +++ b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-_row_id.arrow.json @@ -17,7 +17,7 @@ "name" : "_row_id", "count" : 3, "VALIDITY" : [1,1,1], - "DATA" : ["2","3","6"] + "DATA" : ["2","3","5"] }] }] } \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/metadata.arrow.json index 9a08ada7ea..42c9c98189 100644 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/metadata.arrow.json +++ b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/metadata.arrow.json @@ -182,7 +182,7 @@ "name" : "max", "count" : 10, "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["6","6554","0","0","0","6","6554","0","0","0"] + "DATA" : ["5","6554","0","0","0","5","6554","0","0","0"] }] },{ "name" : "timestamp-tz-micro-utc", @@ -219,8 +219,8 @@ "name" : "bloom", "count" : 10, "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,74,148,238,266,318,392,466,556,584,636], - "DATA" : ["3a300000060000000000020004000000050000000a0000000b0000000f000100380000003e00000040000000420000004400000046000000020003000600d29477896a4ae81257c3ade1","3a300000060000000000020005000000090000000a0000000c0000000f000100380000003e0000004000000042000000440000004600000000005d0d9a19fc2ac562f85511b8f23846a9","3a30000008000000000000000200000004000000090000000b0000000c0001000e0000000f000000480000004a0000004c0000004e00000050000000520000005600000058000000c4020af41c3b4c0193972a6e309bfa4e08db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000003e007c00ba008a6364dfb0e7","3a300000060000000000020004000000050000000a0000000b0000000f000100380000003e00000040000000420000004400000046000000020003000600d29477896a4ae81257c3ade1","3a300000060000000000020005000000090000000a0000000c0000000f000100380000003e0000004000000042000000440000004600000000005d0d9a19fc2ac562f85511b8f23846a9","3a30000008000000000000000200000004000000090000000b0000000c0001000e0000000f000000480000004a0000004c0000004e00000050000000520000005600000058000000c4020af41c3b4c0193972a6e309bfa4e08db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000003e007c00ba008a6364dfb0e7"] + "OFFSET" : [0,66,140,230,258,310,376,450,540,568,620], + "DATA" : ["3a3000000500000000000200040000000a0000000e0000000f0002003000000036000000380000003a0000003c000000020003000500d2946a4adbd6706b57c3ade1","3a300000060000000000020005000000090000000a0000000c0000000f000100380000003e0000004000000042000000440000004600000000005d0d9a19fc2ac562f85511b8f23846a9","3a30000008000000000000000200000004000000090000000b0000000c0001000e0000000f000000480000004a0000004c0000004e00000050000000520000005600000058000000c4020af41c3b4c0193972a6e309bfa4e08db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000003e007c00ba008a6364dfb0e7","3a3000000500000000000200040000000a0000000e0000000f0002003000000036000000380000003a0000003c000000020003000500d2946a4adbd6706b57c3ade1","3a300000060000000000020005000000090000000a0000000c0000000f000100380000003e0000004000000042000000440000004600000000005d0d9a19fc2ac562f85511b8f23846a9","3a30000008000000000000000200000004000000090000000b0000000c0001000e0000000f000000480000004a0000004c0000004e00000050000000520000005600000058000000c4020af41c3b4c0193972a6e309bfa4e08db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000003e007c00ba008a6364dfb0e7"] }] }] }] diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-_row_id.arrow.json index a2b66b760b..fd88a2db73 100644 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-_row_id.arrow.json +++ b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-_row_id.arrow.json @@ -17,7 +17,7 @@ "name" : "_row_id", "count" : 3, "VALIDITY" : [1,1,1], - "DATA" : ["0","1","5"] + "DATA" : ["0","1","4"] }] }] } \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/metadata.arrow.json index a27de3c3e7..e5df373abf 100644 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/metadata.arrow.json +++ b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/metadata.arrow.json @@ -164,7 +164,7 @@ "name" : "max", "count" : 6, "VALIDITY" : [1,0,0,1,0,0], - "DATA" : ["5","0","0","5","0","0"] + "DATA" : ["4","0","0","4","0","0"] }] },{ "name" : "utf8", @@ -194,7 +194,7 @@ "count" : 6, "VALIDITY" : [1,1,1,1,1,1], "OFFSET" : [0,74,126,178,252,304,356], - "DATA" : ["3a300000060000000000030005000000080000000a0000000e0000000f0000003800000040000000420000004400000046000000480000000000010005002beefc2a1677f855dbd6706b","3a300000040000000100010002000100040000000b000000280000002c00000030000000320000007009b09d891af031fbe74281","3a300000040000000000020001000000050000000a000000280000002e00000030000000320000003e007c00ba00956a83d4713e","3a300000060000000000030005000000080000000a0000000e0000000f0000003800000040000000420000004400000046000000480000000000010005002beefc2a1677f855dbd6706b","3a300000040000000100010002000100040000000b000000280000002c00000030000000320000007009b09d891af031fbe74281","3a300000040000000000020001000000050000000a000000280000002e00000030000000320000003e007c00ba00956a83d4713e"] + "DATA" : ["3a300000060000000000030005000000080000000a0000000d0000000e0000003800000040000000420000004400000046000000480000000000010004002beefc2a1677f8558e13c989","3a300000040000000100010002000100040000000b000000280000002c00000030000000320000007009b09d891af031fbe74281","3a300000040000000000020001000000050000000a000000280000002e00000030000000320000003e007c00ba00956a83d4713e","3a300000060000000000030005000000080000000a0000000d0000000e0000003800000040000000420000004400000046000000480000000000010004002beefc2a1677f8558e13c989","3a300000040000000100010002000100040000000b000000280000002c00000030000000320000007009b09d891af031fbe74281","3a300000040000000000020001000000050000000a000000280000002e00000030000000320000003e007c00ba00956a83d4713e"] }] }] }] diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/tables/xt$txs/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/tables/xt$txs/chunks/leaf-c00.arrow.json index 97211eb6c4..a66ee627eb 100644 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/tables/xt$txs/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/writes-log-file/tables/xt$txs/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -147,11 +138,6 @@ "count" : 3, "VALIDITY" : [1,1,1], "DATA" : ["1ef6a989c60d8dc721fc8b54fb3bc0fe","4a2474f0c7afe8df22b2da22ac73e818","a4e167a76a05add8a8654c169b07b044"] - },{ - "name" : "xt$legacy_iid", - "count" : 3, - "VALIDITY" : [1,1,1], - "DATA" : ["432345564227567616","216172782113783808","144115188075855872"] },{ "name" : "xt$system_from", "count" : 3, diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/tables/xt_docs/chunks/leaf-c00.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/tables/xt_docs/chunks/leaf-c00.arrow.json index 7c416ff028..123456ac5b 100644 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/tables/xt_docs/chunks/leaf-c00.arrow.json +++ b/src/test/resources/xtdb/indexer-test/writes-log-file/tables/xt_docs/chunks/leaf-c00.arrow.json @@ -8,15 +8,6 @@ "byteWidth" : 16 }, "children" : [ ] - },{ - "name" : "xt$legacy_iid", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] },{ "name" : "xt$system_from", "nullable" : false, @@ -154,11 +145,6 @@ "count" : 4, "VALIDITY" : [1,1,1,1], "DATA" : ["9e3f856e68998313827ff18dd4d88e78","9e3f856e68998313827ff18dd4d88e78","d9c7fae2a04e047164936265ba33cf80","d9c7fae2a04e047164936265ba33cf80"] - },{ - "name" : "xt$legacy_iid", - "count" : 4, - "VALIDITY" : [1,1,1,1], - "DATA" : ["72057594037927936","72057594037927936","0","0"] },{ "name" : "xt$system_from", "count" : 4, diff --git a/src/test/resources/xtdb/sql/plan_test_expectations/test-dynamic-parameters-103-update-app-time.edn b/src/test/resources/xtdb/sql/plan_test_expectations/test-dynamic-parameters-103-update-app-time.edn index e7e627e8e6..b323e86eca 100644 --- a/src/test/resources/xtdb/sql/plan_test_expectations/test-dynamic-parameters-103-update-app-time.edn +++ b/src/test/resources/xtdb/sql/plan_test_expectations/test-dynamic-parameters-103-update-app-time.edn @@ -1,39 +1,31 @@ [:update {:table "users"} [:rename - {x3 _iid, + {x2 xt$iid, x5 xt$system_to, - x6 _row_id, - x7 xt$id, - x8 xt$system_from, - x10 first_name, - x11 xt$valid_from, - x12 xt$valid_to} + x6 xt$system_from, + x8 first_name, + x9 xt$valid_from, + x10 xt$valid_to} [:project - [x3 + [x2 x5 x6 - x7 - x8 - {x10 ?_2} - {x11 (cast-tstz (greatest x4 ?_0))} - {x12 (cast-tstz (least x2 ?_1))}] + {x8 ?_2} + {x9 (cast-tstz (greatest x4 ?_0))} + {x10 (cast-tstz (least x3 ?_1))}] [:rename {id x1, - xt$valid_to x2, - _iid x3, + xt$iid x2, + xt$valid_to x3, xt$valid_from x4, xt$system_to x5, - _row_id x6, - xt$id x7, - xt$system_from x8} + xt$system_from x6} [:scan {:table users, :for-valid-time [:between ?_0 ?_1]} [{id (= id ?_3)} + xt$iid xt$valid_to - _iid xt$valid_from xt$system_to - _row_id - xt$id xt$system_from]]]]]] diff --git a/src/test/resources/xtdb/sql/plan_test_expectations/test-dynamic-parameters-103-update-set-value.edn b/src/test/resources/xtdb/sql/plan_test_expectations/test-dynamic-parameters-103-update-set-value.edn index 5a65da5ea4..7fd923937a 100644 --- a/src/test/resources/xtdb/sql/plan_test_expectations/test-dynamic-parameters-103-update-set-value.edn +++ b/src/test/resources/xtdb/sql/plan_test_expectations/test-dynamic-parameters-103-update-set-value.edn @@ -1,30 +1,20 @@ [:update {:table "t1"} [:rename - {x2 _iid, + {x1 xt$iid, x4 xt$system_to, - x5 _row_id, - x6 xt$id, - x7 xt$system_from, - x9 col1, - x10 xt$valid_from, - x11 xt$valid_to} + x5 xt$system_from, + x7 col1, + x8 xt$valid_from, + x9 xt$valid_to} [:project - [x2 x4 x5 x6 x7 {x9 ?_0} {x10 (cast-tstz x3)} {x11 (cast-tstz x1)}] + [x1 x4 x5 {x7 ?_0} {x8 (cast-tstz x3)} {x9 (cast-tstz x2)}] [:rename - {xt$valid_to x1, - _iid x2, + {xt$iid x1, + xt$valid_to x2, xt$valid_from x3, xt$system_to x4, - _row_id x5, - xt$id x6, - xt$system_from x7} + xt$system_from x5} [:scan {:table t1} - (xt$valid_to - _iid - xt$valid_from - xt$system_to - _row_id - xt$id - xt$system_from)]]]]] + (xt$iid xt$valid_to xt$valid_from xt$system_to xt$system_from)]]]]] diff --git a/src/test/resources/xtdb/sql/plan_test_expectations/test-for-all-valid-time-387-delete.edn b/src/test/resources/xtdb/sql/plan_test_expectations/test-for-all-valid-time-387-delete.edn index 613e9dc4b4..28a1c13951 100644 --- a/src/test/resources/xtdb/sql/plan_test_expectations/test-for-all-valid-time-387-delete.edn +++ b/src/test/resources/xtdb/sql/plan_test_expectations/test-for-all-valid-time-387-delete.edn @@ -1,29 +1,19 @@ [:delete {:table "users"} [:rename - {x2 _iid, + {x1 xt$iid, x4 xt$system_to, - x5 _row_id, - x6 xt$id, - x7 xt$system_from, - x9 xt$valid_from, - x10 xt$valid_to} + x5 xt$system_from, + x7 xt$valid_from, + x8 xt$valid_to} [:project - [x2 x4 x5 x6 x7 {x9 (cast-tstz x3)} {x10 (cast-tstz x1)}] + [x1 x4 x5 {x7 (cast-tstz x3)} {x8 (cast-tstz x2)}] [:rename - {xt$valid_to x1, - _iid x2, + {xt$iid x1, + xt$valid_to x2, xt$valid_from x3, xt$system_to x4, - _row_id x5, - xt$id x6, - xt$system_from x7} + xt$system_from x5} [:scan {:table users, :for-valid-time :all-time} - (xt$valid_to - _iid - xt$valid_from - xt$system_to - _row_id - xt$id - xt$system_from)]]]]] + (xt$iid xt$valid_to xt$valid_from xt$system_to xt$system_from)]]]]] diff --git a/src/test/resources/xtdb/sql/plan_test_expectations/test-for-all-valid-time-387-update.edn b/src/test/resources/xtdb/sql/plan_test_expectations/test-for-all-valid-time-387-update.edn index 2188ea2462..ca172d7b73 100644 --- a/src/test/resources/xtdb/sql/plan_test_expectations/test-for-all-valid-time-387-update.edn +++ b/src/test/resources/xtdb/sql/plan_test_expectations/test-for-all-valid-time-387-update.edn @@ -1,37 +1,20 @@ [:update {:table "users"} [:rename - {x2 _iid, + {x1 xt$iid, x4 xt$system_to, - x5 _row_id, - x6 xt$id, - x7 xt$system_from, - x9 first_name, - x10 xt$valid_from, - x11 xt$valid_to} + x5 xt$system_from, + x7 first_name, + x8 xt$valid_from, + x9 xt$valid_to} [:project - [x2 - x4 - x5 - x6 - x7 - {x9 "Sue"} - {x10 (cast-tstz x3)} - {x11 (cast-tstz x1)}] + [x1 x4 x5 {x7 "Sue"} {x8 (cast-tstz x3)} {x9 (cast-tstz x2)}] [:rename - {xt$valid_to x1, - _iid x2, + {xt$iid x1, + xt$valid_to x2, xt$valid_from x3, xt$system_to x4, - _row_id x5, - xt$id x6, - xt$system_from x7} + xt$system_from x5} [:scan {:table users, :for-valid-time :all-time} - (xt$valid_to - _iid - xt$valid_from - xt$system_to - _row_id - xt$id - xt$system_from)]]]]] + (xt$iid xt$valid_to xt$valid_from xt$system_to xt$system_from)]]]]] diff --git a/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-delete-plan.edn b/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-delete-plan.edn index 732371a19e..412180eec2 100644 --- a/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-delete-plan.edn +++ b/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-delete-plan.edn @@ -1,39 +1,31 @@ [:delete {:table "users"} [:rename - {x3 _iid, + {x2 xt$iid, x5 xt$system_to, - x6 _row_id, - x7 xt$id, - x8 xt$system_from, - x10 xt$valid_from, - x11 xt$valid_to} + x6 xt$system_from, + x8 xt$valid_from, + x9 xt$valid_to} [:project - [x3 + [x2 x5 x6 - x7 - x8 - {x10 (cast-tstz (greatest x4 #time/date "2020-05-01"))} - {x11 (cast-tstz (least x2 #time/date "9999-12-31"))}] + {x8 (cast-tstz (greatest x4 #time/date "2020-05-01"))} + {x9 (cast-tstz (least x3 #time/date "9999-12-31"))}] [:rename {id x1, - xt$valid_to x2, - _iid x3, + xt$iid x2, + xt$valid_to x3, xt$valid_from x4, xt$system_to x5, - _row_id x6, - xt$id x7, - xt$system_from x8} + xt$system_from x6} [:scan {:table users, :for-valid-time [:between #time/date "2020-05-01" #time/date "9999-12-31"]} [{id (= id ?_0)} + xt$iid xt$valid_to - _iid xt$valid_from xt$system_to - _row_id - xt$id xt$system_from]]]]]] diff --git a/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-update-plan-with-column-references.edn b/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-update-plan-with-column-references.edn index 0f4effdd1b..b13db42b1e 100644 --- a/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-update-plan-with-column-references.edn +++ b/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-update-plan-with-column-references.edn @@ -1,32 +1,26 @@ [:update {:table "foo"} [:rename - {x3 _iid, + {x2 xt$iid, x5 xt$system_to, - x6 _row_id, - x7 xt$id, - x8 xt$system_from, + x6 xt$system_from, x1 bar, - x10 xt$valid_from, - x11 xt$valid_to} + x8 xt$valid_from, + x9 xt$valid_to} [:project - [x3 x5 x6 x7 x8 x1 {x10 (cast-tstz x4)} {x11 (cast-tstz x2)}] + [x2 x5 x6 x1 {x8 (cast-tstz x4)} {x9 (cast-tstz x3)}] [:rename {baz x1, - xt$valid_to x2, - _iid x3, + xt$iid x2, + xt$valid_to x3, xt$valid_from x4, xt$system_to x5, - _row_id x6, - xt$id x7, - xt$system_from x8} + xt$system_from x6} [:scan {:table foo} (baz + xt$iid xt$valid_to - _iid xt$valid_from xt$system_to - _row_id - xt$id xt$system_from)]]]]] diff --git a/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-update-plan-with-period-references.edn b/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-update-plan-with-period-references.edn index 44653cd184..ef477d5a8f 100644 --- a/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-update-plan-with-period-references.edn +++ b/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-update-plan-with-period-references.edn @@ -1,37 +1,25 @@ [:update {:table "foo"} [:rename - {x5 _iid, + {x5 xt$iid, x2 xt$system_to, - x6 _row_id, - x7 xt$id, x1 xt$system_from, - x9 bar, - x10 xt$valid_from, - x11 xt$valid_to} + x7 bar, + x8 xt$valid_from, + x9 xt$valid_to} [:project [x5 x2 - x6 - x7 x1 - {x9 (and (< x1 x4) (> x2 x3))} - {x10 (cast-tstz x3)} - {x11 (cast-tstz x4)}] + {x7 (and (< x1 x4) (> x2 x3))} + {x8 (cast-tstz x3)} + {x9 (cast-tstz x4)}] [:rename {xt$system_from x1, xt$system_to x2, xt$valid_from x3, xt$valid_to x4, - _iid x5, - _row_id x6, - xt$id x7} + xt$iid x5} [:scan {:table foo} - (xt$system_from - xt$system_to - xt$valid_from - xt$valid_to - _iid - _row_id - xt$id)]]]]] + (xt$system_from xt$system_to xt$valid_from xt$valid_to xt$iid)]]]]] diff --git a/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-update-plan.edn b/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-update-plan.edn index 905c14ab5f..ab45df20ec 100644 --- a/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-update-plan.edn +++ b/src/test/resources/xtdb/sql/plan_test_expectations/test-sql-update-plan.edn @@ -1,41 +1,33 @@ [:update {:table "users"} [:rename - {x3 _iid, + {x2 xt$iid, x5 xt$system_to, - x6 _row_id, - x7 xt$id, - x8 xt$system_from, - x10 first_name, - x11 xt$valid_from, - x12 xt$valid_to} + x6 xt$system_from, + x8 first_name, + x9 xt$valid_from, + x10 xt$valid_to} [:project - [x3 + [x2 x5 x6 - x7 - x8 - {x10 "Sue"} - {x11 (cast-tstz (greatest x4 #time/date "2021-07-01"))} - {x12 (cast-tstz (least x2 #time/date "9999-12-31"))}] + {x8 "Sue"} + {x9 (cast-tstz (greatest x4 #time/date "2021-07-01"))} + {x10 (cast-tstz (least x3 #time/date "9999-12-31"))}] [:rename {id x1, - xt$valid_to x2, - _iid x3, + xt$iid x2, + xt$valid_to x3, xt$valid_from x4, xt$system_to x5, - _row_id x6, - xt$id x7, - xt$system_from x8} + xt$system_from x6} [:scan {:table users, :for-valid-time [:between #time/date "2021-07-01" #time/date "9999-12-31"]} [{id (= id ?_0)} + xt$iid xt$valid_to - _iid xt$valid_from xt$system_to - _row_id - xt$id xt$system_from]]]]]] From cc24631ababf4e9b393e8def4ce52def34140a5e Mon Sep 17 00:00:00 2001 From: James Henderson Date: Fri, 4 Aug 2023 22:12:51 +0100 Subject: [PATCH 7/8] add row-counts to the chunk-level metadata so that we don't need the per-table metadata files for scan stats, #2663 --- core/src/main/clojure/xtdb/indexer/live_index.clj | 3 ++- core/src/main/clojure/xtdb/operator/scan.clj | 13 ++++--------- .../chunk-metadata/00.transit.json | 2 +- .../chunk-metadata/00.transit.json | 2 +- .../chunk-metadata/00.transit.json | 2 +- .../chunk-metadata/00.transit.json | 2 +- .../chunk-metadata/00.transit.json | 2 +- .../writes-log-file/chunk-metadata/00.transit.json | 2 +- 8 files changed, 12 insertions(+), 16 deletions(-) diff --git a/core/src/main/clojure/xtdb/indexer/live_index.clj b/core/src/main/clojure/xtdb/indexer/live_index.clj index 76437271e5..5c9f4a2ed1 100644 --- a/core/src/main/clojure/xtdb/indexer/live_index.clj +++ b/core/src/main/clojure/xtdb/indexer/live_index.clj @@ -170,7 +170,8 @@ (let [chunk-idx-str (util/->lex-hex-string chunk-idx) !fut (trie/write-trie-bufs! obj-store (format "tables/%s/chunks" table-name) chunk-idx-str bufs) table-metadata (MapEntry/create table-name - {:col-types (live-rel->col-types live-rel-rdr)})] + {:col-types (live-rel->col-types live-rel-rdr) + :row-count (.rowCount live-rel-rdr)})] (-> !fut (util/then-apply (fn [_] table-metadata))))))) diff --git a/core/src/main/clojure/xtdb/operator/scan.clj b/core/src/main/clojure/xtdb/operator/scan.clj index 10125759a7..5f20c577d8 100644 --- a/core/src/main/clojure/xtdb/operator/scan.clj +++ b/core/src/main/clojure/xtdb/operator/scan.clj @@ -828,15 +828,10 @@ :when (not (types/temporal-column? (util/str->normal-form-str (str col-name))))] select)) - row-count (->> (meta/with-all-metadata metadata-mgr normalized-table-name - (util/->jbifn - (fn [_chunk-idx ^ITableMetadata table-metadata] - (let [id-col-idx (.rowIndex table-metadata "xt$id" -1) - ^BigIntVector count-vec (-> (.metadataRoot table-metadata) - ^ListVector (.getVector "columns") - ^StructVector (.getDataVector) - (.getChild "count"))] - (.get count-vec id-col-idx))))) + row-count (->> (for [{:keys [tables]} (vals (.chunksMetadata metadata-mgr)) + :let [{:keys [row-count]} (get tables normalized-table-name)] + :when row-count] + row-count) (reduce +))] {:col-types col-types diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-metadata/00.transit.json index e94df4560a..696b7948df 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",8165,"~:system-time",["~#time/instant","2020-01-02T00:00:00Z"]]],"~:next-chunk-idx",6,"~:tables",["^ ","device_info",["^ ","~:col-types",["^ ","xt$id","~:utf8","os_name","^:","model","^:","api_version","^:","manufacturer","^:"]],"device_readings",["^ ","^8",["^ ","cpu_avg_1min","~:f64","cpu_avg_15min","^A","device_id","^:","battery_level","^A","mem_used","^A","battery_status","^:","mem_free","^A","time",["~:timestamp-tz","~:micro","UTC"],"ssid","^:","rssi","^A","cpu_avg_5min","^A","battery_temperature","^A","^9","^:","bssid","^:"]],"xt$txs",["^ ","^8",["^ ","^9","~:i64","xt$tx_time",["^I","^J","UTC"],"xt$committed?","~:bool","xt$error",["~:union",["~#set",["~:clj-form","~:null"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",8165,"~:system-time",["~#time/instant","2020-01-02T00:00:00Z"]]],"~:next-chunk-idx",6,"~:tables",["^ ","device_info",["^ ","~:col-types",["^ ","xt$id","~:utf8","os_name","^:","model","^:","api_version","^:","manufacturer","^:"],"~:row-count",2],"device_readings",["^ ","^8",["^ ","cpu_avg_1min","~:f64","cpu_avg_15min","^B","device_id","^:","battery_level","^B","mem_used","^B","battery_status","^:","mem_free","^B","time",["~:timestamp-tz","~:micro","UTC"],"ssid","^:","rssi","^B","cpu_avg_5min","^B","battery_temperature","^B","^9","^:","bssid","^:"],"^?",2],"xt$txs",["^ ","^8",["^ ","^9","~:i64","xt$tx_time",["^J","^K","UTC"],"xt$committed?","~:bool","xt$error",["~:union",["~#set",["~:clj-form","~:null"]]]],"^?",2]]] \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-metadata/00.transit.json index 1417014675..91550f9988 100644 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",18145,"~:system-time",["~#time/instant","2020-01-06T00:00:00Z"]]],"~:next-chunk-idx",16,"~:tables",["^ ","world",["^ ","~:col-types",["^ ","b","~:i64","xt$id","~:uuid"]],"foo",["^ ","^8",["^ ","^:","^9","bar","^9","toto",["~:union",["~#set",["~:utf8","~:absent"]]]]],"hello",["^ ","^8",["^ ","a","^9","^:","^;"]],"xt$txs",["^ ","^8",["^ ","^:","^9","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["^=",["^>",["~:clj-form","~:null"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",18145,"~:system-time",["~#time/instant","2020-01-06T00:00:00Z"]]],"~:next-chunk-idx",16,"~:tables",["^ ","world",["^ ","~:col-types",["^ ","b","~:i64","xt$id","~:uuid"],"~:row-count",3],"foo",["^ ","^8",["^ ","^:","^9","bar","^9","toto",["~:union",["~#set",["~:utf8","~:absent"]]]],"^<",5],"hello",["^ ","^8",["^ ","a","^9","^:","^;"],"^<",2],"xt$txs",["^ ","^8",["^ ","^:","^9","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["^>",["^?",["~:clj-form","~:null"]]]],"^<",6]]] \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-metadata/00.transit.json index 11a71645fd..87e178fd1b 100644 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",0,"~:system-time",["~#time/instant","2020-01-01T00:00:00Z"]]],"~:next-chunk-idx",7,"~:tables",["^ ","xt_docs",["^ ","~:col-types",["^ ","struct",["~:union",["~#set",[["~:struct",["^ ","~$a",["^:",["^;",["~:bool","~:i64"]]],"~$b",["^:",["^;",["~:utf8","~:absent"]]],"~$c",["^:",["^;",["^?","^@"]]]]],"^@"]]],"xt$id",["^:",["^;",["^?","~:keyword","^>"]]],"list",["^:",["^;",[["~:list",["^:",["^;",["~:f64","^?",["~:timestamp-tz","~:micro","UTC"],"^="]]]],"^@"]]]]],"xt$txs",["^ ","^8",["^ ","^A","^>","xt$tx_time",["^F","^G","UTC"],"xt$committed?","^=","xt$error",["^:",["^;",["~:clj-form","~:null"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",0,"~:system-time",["~#time/instant","2020-01-01T00:00:00Z"]]],"~:next-chunk-idx",7,"~:tables",["^ ","xt_docs",["^ ","~:col-types",["^ ","struct",["~:union",["~#set",[["~:struct",["^ ","~$a",["^:",["^;",["~:bool","~:i64"]]],"~$b",["^:",["^;",["~:utf8","~:absent"]]],"~$c",["^:",["^;",["^?","^@"]]]]],"^@"]]],"xt$id",["^:",["^;",["^?","~:keyword","^>"]]],"list",["^:",["^;",[["~:list",["^:",["^;",["~:f64","^?",["~:timestamp-tz","~:micro","UTC"],"^="]]]],"^@"]]]],"~:row-count",6],"xt$txs",["^ ","^8",["^ ","^A","^>","xt$tx_time",["^F","^G","UTC"],"xt$committed?","^=","xt$error",["^:",["^;",["~:clj-form","~:null"]]]],"^H",1]]] \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-metadata/00.transit.json index aca0f0f4ac..ea6f255e47 100644 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",0,"~:system-time",["~#time/instant","2020-01-01T00:00:00Z"]]],"~:next-chunk-idx",3,"~:tables",["^ ","xt$txs",["^ ","~:col-types",["^ ","xt$id","~:i64","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["~:union",["~#set",["~:clj-form","~:null"]]]]],"table",["^ ","^8",["^ ","^9","^:","foo","^:","bar","~:utf8","baz",["^A",["^B",["~:f64","^:"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",0,"~:system-time",["~#time/instant","2020-01-01T00:00:00Z"]]],"~:next-chunk-idx",3,"~:tables",["^ ","xt$txs",["^ ","~:col-types",["^ ","xt$id","~:i64","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["~:union",["~#set",["~:clj-form","~:null"]]]],"~:row-count",1],"table",["^ ","^8",["^ ","^9","^:","foo","^:","bar","~:utf8","baz",["^A",["^B",["~:f64","^:"]]]],"^E",2]]] \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-metadata/00.transit.json index c98474d7da..def37e3282 100644 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",5845,"~:system-time",["~#time/instant","2020-01-02T00:00:00Z"]]],"~:next-chunk-idx",8,"~:tables",["^ ","xt_docs",["^ ","~:col-types",["^ ","struct",["~:union",["~#set",["~:absent",["~:struct",["^ ","~$a",["^:",["^;",["~:bool","~:i64"]]],"~$b",["^:",["^;",["~:utf8",["^=",["^ ","~$c","^@","~$d","^@"]]]]]]]]]],"xt$id",["^:",["^;",["^@","~:keyword","^?"]]],"list",["^:",["^;",[["~:list",["^:",["^;",["~:f64","^@",["~:timestamp-tz","~:micro","UTC"],"^>"]]]],"^<"]]]]],"xt$txs",["^ ","^8",["^ ","^A","^?","xt$tx_time",["^F","^G","UTC"],"xt$committed?","^>","xt$error",["^:",["^;",["~:clj-form","~:null"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",5845,"~:system-time",["~#time/instant","2020-01-02T00:00:00Z"]]],"~:next-chunk-idx",8,"~:tables",["^ ","xt_docs",["^ ","~:col-types",["^ ","struct",["~:union",["~#set",["~:absent",["~:struct",["^ ","~$a",["^:",["^;",["~:bool","~:i64"]]],"~$b",["^:",["^;",["~:utf8",["^=",["^ ","~$c","^@","~$d","^@"]]]]]]]]]],"xt$id",["^:",["^;",["^@","~:keyword","^?"]]],"list",["^:",["^;",[["~:list",["^:",["^;",["~:f64","^@",["~:timestamp-tz","~:micro","UTC"],"^>"]]]],"^<"]]]],"~:row-count",6],"xt$txs",["^ ","^8",["^ ","^A","^?","xt$tx_time",["^F","^G","UTC"],"xt$committed?","^>","xt$error",["^:",["^;",["~:clj-form","~:null"]]]],"^H",2]]] \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-metadata/00.transit.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-metadata/00.transit.json index 96d097d6e3..bea762ea95 100644 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-metadata/00.transit.json +++ b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-metadata/00.transit.json @@ -1 +1 @@ -["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",6554,"~:system-time",["~#time/instant","2020-01-03T00:00:00Z"]]],"~:next-chunk-idx",7,"~:tables",["^ ","xt_docs",["^ ","~:col-types",["^ ","xt$id","~:utf8","month",["~:union",["~#set",["^:","~:absent"]]]]],"xt$txs",["^ ","^8",["^ ","^9","~:i64","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["^<",["^=",["~:clj-form","~:null"]]]]]]] \ No newline at end of file +["^ ","~:latest-completed-tx",["~#xtdb/tx-key",["^ ","~:tx-id",6554,"~:system-time",["~#time/instant","2020-01-03T00:00:00Z"]]],"~:next-chunk-idx",7,"~:tables",["^ ","xt_docs",["^ ","~:col-types",["^ ","xt$id","~:utf8","month",["~:union",["~#set",["^:","~:absent"]]]],"~:row-count",4],"xt$txs",["^ ","^8",["^ ","^9","~:i64","xt$tx_time",["~:timestamp-tz","~:micro","UTC"],"xt$committed?","~:bool","xt$error",["^<",["^=",["~:clj-form","~:null"]]]],"^?",3]]] \ No newline at end of file From c044fe2d060589bf4e411d06eea75f4ccb3973cc Mon Sep 17 00:00:00 2001 From: James Henderson Date: Fri, 4 Aug 2023 22:17:10 +0100 Subject: [PATCH 8/8] remove the live-chunk files, #2663 --- core/src/main/clojure/xtdb/indexer.clj | 219 +++------ core/src/main/clojure/xtdb/live_chunk.clj | 451 ------------------ core/src/main/clojure/xtdb/metadata.clj | 3 - core/src/main/clojure/xtdb/node.clj | 1 - core/src/main/clojure/xtdb/watermark.clj | 14 +- src/main/clojure/xtdb/test_util.clj | 6 +- src/test/clojure/xtdb/datalog_test.clj | 10 +- src/test/clojure/xtdb/indexer_test.clj | 119 +---- src/test/clojure/xtdb/operator/scan_test.clj | 2 +- src/test/clojure/xtdb/operator_test.clj | 11 +- src/test/clojure/xtdb/stats_test.clj | 3 +- .../device_info/content-_row_id.arrow.json | 23 - .../content-api_version.arrow.json | 37 -- .../content-manufacturer.arrow.json | 37 -- .../device_info/content-model.arrow.json | 37 -- .../device_info/content-os_name.arrow.json | 37 -- .../device_info/content-xt$id.arrow.json | 37 -- .../chunk-00/device_info/metadata.arrow.json | 190 -------- .../content-_row_id.arrow.json | 23 - .../content-battery_level.arrow.json | 37 -- .../content-battery_status.arrow.json | 37 -- .../content-battery_temperature.arrow.json | 37 -- .../device_readings/content-bssid.arrow.json | 37 -- .../content-cpu_avg_15min.arrow.json | 37 -- .../content-cpu_avg_1min.arrow.json | 37 -- .../content-cpu_avg_5min.arrow.json | 37 -- .../content-device_id.arrow.json | 37 -- .../content-mem_free.arrow.json | 37 -- .../content-mem_used.arrow.json | 37 -- .../device_readings/content-rssi.arrow.json | 37 -- .../device_readings/content-ssid.arrow.json | 37 -- .../device_readings/content-time.arrow.json | 38 -- .../device_readings/content-xt$id.arrow.json | 37 -- .../device_readings/metadata.arrow.json | 268 ----------- .../xt$txs/content-_row_id.arrow.json | 23 - .../xt$txs/content-xt$committed?.arrow.json | 36 -- .../xt$txs/content-xt$error.arrow.json | 54 --- .../chunk-00/xt$txs/content-xt$id.arrow.json | 38 -- .../xt$txs/content-xt$tx_time.arrow.json | 38 -- .../chunk-00/xt$txs/metadata.arrow.json | 216 --------- .../chunk-00/foo/content-_row_id.arrow.json | 23 - .../chunk-00/foo/content-bar.arrow.json | 38 -- .../chunk-00/foo/content-toto.arrow.json | 54 --- .../chunk-00/foo/content-xt$id.arrow.json | 38 -- .../chunk-00/foo/metadata.arrow.json | 202 -------- .../chunk-00/hello/content-_row_id.arrow.json | 23 - .../chunk-00/hello/content-a.arrow.json | 38 -- .../chunk-00/hello/content-xt$id.arrow.json | 43 -- .../chunk-00/hello/metadata.arrow.json | 202 -------- .../chunk-00/world/content-_row_id.arrow.json | 23 - .../chunk-00/world/content-b.arrow.json | 38 -- .../chunk-00/world/content-xt$id.arrow.json | 43 -- .../chunk-00/world/metadata.arrow.json | 202 -------- .../xt$txs/content-_row_id.arrow.json | 23 - .../xt$txs/content-xt$committed?.arrow.json | 36 -- .../xt$txs/content-xt$error.arrow.json | 54 --- .../chunk-00/xt$txs/content-xt$id.arrow.json | 38 -- .../xt$txs/content-xt$tx_time.arrow.json | 38 -- .../chunk-00/xt$txs/metadata.arrow.json | 228 --------- .../xt$txs/content-_row_id.arrow.json | 23 - .../xt$txs/content-xt$committed?.arrow.json | 36 -- .../xt$txs/content-xt$error.arrow.json | 54 --- .../chunk-00/xt$txs/content-xt$id.arrow.json | 38 -- .../xt$txs/content-xt$tx_time.arrow.json | 38 -- .../chunk-00/xt$txs/metadata.arrow.json | 216 --------- .../xt_docs/content-_row_id.arrow.json | 23 - .../chunk-00/xt_docs/content-list.arrow.json | 121 ----- .../xt_docs/content-struct.arrow.json | 184 ------- .../chunk-00/xt_docs/content-xt$id.arrow.json | 71 --- .../chunk-00/xt_docs/metadata.arrow.json | 385 --------------- .../chunk-00/table/content-_row_id.arrow.json | 23 - .../chunk-00/table/content-bar.arrow.json | 37 -- .../chunk-00/table/content-baz.arrow.json | 51 -- .../chunk-00/table/content-foo.arrow.json | 38 -- .../chunk-00/table/content-xt$id.arrow.json | 38 -- .../chunk-00/table/metadata.arrow.json | 228 --------- .../xt$txs/content-_row_id.arrow.json | 23 - .../xt$txs/content-xt$committed?.arrow.json | 36 -- .../xt$txs/content-xt$error.arrow.json | 54 --- .../chunk-00/xt$txs/content-xt$id.arrow.json | 38 -- .../xt$txs/content-xt$tx_time.arrow.json | 38 -- .../chunk-00/xt$txs/metadata.arrow.json | 216 --------- .../xt$txs/content-_row_id.arrow.json | 39 -- .../xt$txs/content-xt$committed?.arrow.json | 64 --- .../xt$txs/content-xt$error.arrow.json | 90 ---- .../chunk-00/xt$txs/content-xt$id.arrow.json | 66 --- .../xt$txs/content-xt$tx_time.arrow.json | 66 --- .../chunk-00/xt$txs/metadata.arrow.json | 216 --------- .../xt_docs/content-_row_id.arrow.json | 39 -- .../chunk-00/xt_docs/content-list.arrow.json | 211 -------- .../xt_docs/content-struct.arrow.json | 338 ------------- .../chunk-00/xt_docs/content-xt$id.arrow.json | 123 ----- .../chunk-00/xt_docs/metadata.arrow.json | 439 ----------------- .../chunk-00/foo/content-_row_id.arrow.json | 23 - .../chunk-00/foo/metadata.arrow.json | 111 ----- .../xt$txs/content-_row_id.arrow.json | 23 - .../xt$txs/content-xt$committed?.arrow.json | 36 -- .../xt$txs/content-xt$error.arrow.json | 54 --- .../chunk-00/xt$txs/content-xt$id.arrow.json | 38 -- .../xt$txs/content-xt$tx_time.arrow.json | 38 -- .../chunk-00/xt$txs/metadata.arrow.json | 228 --------- .../xt_docs/content-_row_id.arrow.json | 23 - .../chunk-00/xt_docs/content-month.arrow.json | 54 --- .../chunk-00/xt_docs/content-xt$id.arrow.json | 37 -- .../chunk-00/xt_docs/metadata.arrow.json | 202 -------- 105 files changed, 114 insertions(+), 8289 deletions(-) delete mode 100644 core/src/main/clojure/xtdb/live_chunk.clj delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-api_version.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-manufacturer.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-model.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-os_name.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-battery_level.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-battery_status.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-battery_temperature.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-bssid.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-cpu_avg_15min.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-cpu_avg_1min.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-cpu_avg_5min.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-device_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-mem_free.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-mem_used.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-rssi.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-ssid.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-time.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$committed?.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$error.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$tx_time.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-bar.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-toto.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/content-a.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-b.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$committed?.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$error.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$tx_time.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$committed?.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$error.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$tx_time.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-list.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-struct.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-bar.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-baz.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-foo.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$committed?.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$error.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$tx_time.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$committed?.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$error.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$tx_time.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-list.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-struct.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/foo/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/foo/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$committed?.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$error.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$tx_time.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/metadata.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-_row_id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-month.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-xt$id.arrow.json delete mode 100644 src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/metadata.arrow.json diff --git a/core/src/main/clojure/xtdb/indexer.clj b/core/src/main/clojure/xtdb/indexer.clj index 4ad8bf483d..4928dc74bf 100644 --- a/core/src/main/clojure/xtdb/indexer.clj +++ b/core/src/main/clojure/xtdb/indexer.clj @@ -5,7 +5,6 @@ [xtdb.datalog :as d] [xtdb.error :as err] xtdb.indexer.live-index - xtdb.live-chunk [xtdb.metadata :as meta] xtdb.object-store [xtdb.operator :as op] @@ -34,8 +33,7 @@ (org.apache.arrow.vector.ipc ArrowStreamReader) xtdb.RowCounter (xtdb.api.protocols ClojureForm TransactionInstant) - (xtdb.indexer.live_index ILiveIndex ILiveIndexTx) - (xtdb.live_chunk ILiveChunk ILiveChunkTx ILiveTableTx) + (xtdb.indexer.live_index ILiveIndex ILiveIndexTx ILiveTableTx) xtdb.metadata.IMetadataManager xtdb.object_store.ObjectStore xtdb.operator.IRaQuerySource @@ -52,7 +50,6 @@ (^xtdb.api.protocols.TransactionInstant latestCompletedTx [])) (defprotocol Finish - (^void finish-block! [_]) (^void finish-chunk! [_])) (def ^:private abort-exn (err/runtime-err :abort-exn)) @@ -82,7 +79,7 @@ (.digest eid-bytes) (Arrays/copyOfRange 0 16)))))) -(defn- ->put-indexer ^xtdb.indexer.OpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk, +(defn- ->put-indexer ^xtdb.indexer.OpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^IVectorReader tx-ops-rdr, ^Instant system-time] (let [put-leg (.legReader tx-ops-rdr :put) doc-rdr (.structKeyReader put-leg "document") @@ -95,53 +92,40 @@ table-name (.getName table-rdr) table-rel-rdr (vr/rel-reader (for [sk (.structKeys table-rdr)] (.structKeyReader table-rdr sk)) - (.valueCount table-rdr))] + (.valueCount table-rdr)) + live-table (.liveTable live-idx-tx table-name)] {:table-name table-name :id-rdr (.structKeyReader table-rdr "xt$id") - ;; dual write to live-chunk and live-idx for now - :live-chunk-table (let [live-table (.liveTable live-chunk table-name)] - {:live-table live-table - :table-copier (.rowCopier (.writer live-table) table-rel-rdr)}) + :live-table live-table - :live-idx-table (let [live-table (.liveTable live-idx-tx table-name)] - {:live-table live-table - :doc-copier (-> (.docWriter live-table) - (vw/struct-writer->rel-copier table-rel-rdr))})}))))] + :doc-copier (-> (.docWriter live-table) + (vw/struct-writer->rel-copier table-rel-rdr))}))))] (reify OpIndexer (indexOp [_ tx-op-idx] - (let [row-id (.nextRowId live-chunk) - - {:keys [^IVectorReader id-rdr, live-chunk-table, live-idx-table]} + (let [{:keys [^IVectorReader id-rdr, ^ILiveTableTx live-table, ^IRowCopier doc-copier]} (nth tables (.getTypeId doc-rdr tx-op-idx)) - eid (.getObject id-rdr tx-op-idx)] - - (let [{:keys [^ILiveTableTx live-table, ^IRowCopier table-copier]} live-chunk-table] - (.writeRowId live-table row-id) - (.copyRow table-copier tx-op-idx)) - - (let [valid-from (if (= :null (.getLeg valid-from-rdr tx-op-idx)) - system-time-µs - (.getLong valid-from-rdr tx-op-idx)) - valid-to (if (= :null (.getLeg valid-to-rdr tx-op-idx)) - util/end-of-time-μs - (.getLong valid-to-rdr tx-op-idx))] - (when-not (> valid-to valid-from) - (throw (err/runtime-err :xtdb.indexer/invalid-valid-times - {:valid-from (util/micros->instant valid-from) - :valid-to (util/micros->instant valid-to)}))) + eid (.getObject id-rdr tx-op-idx) - (let [{:keys [^xtdb.indexer.live_index.ILiveTableTx live-table, ^IRowCopier doc-copier]} live-idx-table] - (.logPut live-table (->iid eid) valid-from valid-to #(.copyRow doc-copier tx-op-idx)))) + valid-from (if (= :null (.getLeg valid-from-rdr tx-op-idx)) + system-time-µs + (.getLong valid-from-rdr tx-op-idx)) + valid-to (if (= :null (.getLeg valid-to-rdr tx-op-idx)) + util/end-of-time-μs + (.getLong valid-to-rdr tx-op-idx))] + (when-not (> valid-to valid-from) + (throw (err/runtime-err :xtdb.indexer/invalid-valid-times + {:valid-from (util/micros->instant valid-from) + :valid-to (util/micros->instant valid-to)}))) + (.logPut live-table (->iid eid) valid-from valid-to #(.copyRow doc-copier tx-op-idx)) (.addRows row-counter 1)) nil)))) -(defn- ->delete-indexer ^xtdb.indexer.OpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx ^ILiveChunkTx live-chunk - ^IVectorReader tx-ops-rdr, ^Instant current-time] +(defn- ->delete-indexer ^xtdb.indexer.OpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^IVectorReader tx-ops-rdr, ^Instant current-time] (let [delete-leg (.legReader tx-ops-rdr :delete) table-rdr (.structKeyReader delete-leg "table") id-rdr (.structKeyReader delete-leg "xt$id") @@ -170,15 +154,14 @@ nil)))) -(defn- ->evict-indexer ^xtdb.indexer.OpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx ^ILiveChunkTx live-chunk ^IVectorReader tx-ops-rdr] +(defn- ->evict-indexer ^xtdb.indexer.OpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^IVectorReader tx-ops-rdr] (let [evict-leg (.legReader tx-ops-rdr :evict) table-rdr (.structKeyReader evict-leg "_table") id-rdr (.structKeyReader evict-leg "xt$id")] (reify OpIndexer (indexOp [_ tx-op-idx] - (let [row-id (.nextRowId live-chunk) - table (.getObject table-rdr tx-op-idx) + (let [table (.getObject table-rdr tx-op-idx) eid (.getObject id-rdr tx-op-idx)] (-> (.liveTable live-idx-tx table) @@ -300,7 +283,7 @@ (definterface SqlOpIndexer (^void indexOp [^xtdb.vector.RelationReader inRelation, queryOpts])) -(defn- ->sql-upsert-indexer ^xtdb.indexer.SqlOpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk, +(defn- ->sql-upsert-indexer ^xtdb.indexer.SqlOpIndexer [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx {{:keys [^Instant current-time]} :basis}] (let [current-time-µs (util/instant->micros current-time)] @@ -313,8 +296,6 @@ (.withName vec (util/str->normal-form-str (.getName vec)))))) (.rowCount in-rel)) table (util/str->normal-form-str table) - live-table (.liveTable live-chunk table) - table-copier (.rowCopier (.writer live-table) content-rel) id-col (.readerForName in-rel "xt$id") valid-from-rdr (.readerForName in-rel "xt$valid_from") valid-from-ts-rdr (some-> valid-from-rdr (.legReader :timestamp-tz-micro-utc)) @@ -326,23 +307,19 @@ (vw/struct-writer->rel-copier content-rel))] (dotimes [idx row-count] - (let [row-id (.nextRowId live-chunk)] - (.writeRowId live-table row-id) - (.copyRow table-copier idx) - - (let [eid (.getObject id-col idx) - valid-from (if (and valid-from-rdr (= :timestamp-tz-micro-utc (.getLeg valid-from-rdr idx))) - (.getLong valid-from-ts-rdr idx) - current-time-µs) - valid-to (if (and valid-to-rdr (= :timestamp-tz-micro-utc (.getLeg valid-to-rdr idx))) - (.getLong valid-to-ts-rdr idx) - util/end-of-time-μs)] - (when (> valid-from valid-to) - (throw (err/runtime-err :xtdb.indexer/invalid-valid-times - {:valid-from (util/micros->instant valid-from) - :valid-to (util/micros->instant valid-to)}))) - - (.logPut live-idx-table (->iid eid) valid-from valid-to #(.copyRow live-idx-table-copier idx))))) + (let [eid (.getObject id-col idx) + valid-from (if (and valid-from-rdr (= :timestamp-tz-micro-utc (.getLeg valid-from-rdr idx))) + (.getLong valid-from-ts-rdr idx) + current-time-µs) + valid-to (if (and valid-to-rdr (= :timestamp-tz-micro-utc (.getLeg valid-to-rdr idx))) + (.getLong valid-to-ts-rdr idx) + util/end-of-time-μs)] + (when (> valid-from valid-to) + (throw (err/runtime-err :xtdb.indexer/invalid-valid-times + {:valid-from (util/micros->instant valid-from) + :valid-to (util/micros->instant valid-to)}))) + + (.logPut live-idx-table (->iid eid) valid-from valid-to #(.copyRow live-idx-table-copier idx)))) (.addRows row-counter row-count)))))) @@ -381,13 +358,13 @@ (.addRows row-counter row-count))))) -(defn- ->sql-indexer ^xtdb.indexer.OpIndexer [^BufferAllocator allocator, ^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^ILiveChunk doc-idxer +(defn- ->sql-indexer ^xtdb.indexer.OpIndexer [^BufferAllocator allocator, ^RowCounter row-counter, ^ILiveIndexTx live-idx-tx ^IVectorReader tx-ops-rdr, ^IRaQuerySource ra-src, wm-src, ^IScanEmitter scan-emitter {:keys [default-all-valid-time? basis default-tz] :as tx-opts}] (let [sql-leg (.legReader tx-ops-rdr :sql) query-rdr (.structKeyReader sql-leg "query") params-rdr (.structKeyReader sql-leg "params") - upsert-idxer (->sql-upsert-indexer row-counter live-idx-tx doc-idxer tx-opts) + upsert-idxer (->sql-upsert-indexer row-counter live-idx-tx tx-opts) delete-idxer (->sql-delete-indexer row-counter live-idx-tx) erase-idxer (->sql-erase-indexer row-counter live-idx-tx)] (reify OpIndexer @@ -447,54 +424,32 @@ (def ^:private ^:const ^String txs-table "xt$txs") -(defn- add-tx-row! [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^ILiveChunkTx live-chunk-tx, ^TransactionInstant tx-key, ^Throwable t] +(defn- add-tx-row! [^RowCounter row-counter, ^ILiveIndexTx live-idx-tx, ^TransactionInstant tx-key, ^Throwable t] (let [tx-id (.tx-id tx-key) system-time-µs (util/instant->micros (.system-time tx-key)) - row-id (.nextRowId live-chunk-tx)] + live-table (.liveTable live-idx-tx txs-table) + doc-writer (.docWriter live-table)] - (let [live-table (.liveTable live-idx-tx txs-table) - doc-writer (.docWriter live-table)] - (.logPut live-table (->iid tx-id) system-time-µs util/end-of-time-μs - (fn write-doc! [] - (.startStruct doc-writer) - (doto (.structKeyWriter doc-writer "xt$id" :i64) - (.writeLong tx-id)) + (.logPut live-table (->iid tx-id) system-time-µs util/end-of-time-μs + (fn write-doc! [] + (.startStruct doc-writer) + (doto (.structKeyWriter doc-writer "xt$id" :i64) + (.writeLong tx-id)) - (doto (.structKeyWriter doc-writer "xt$tx_time" types/temporal-col-type) - (.writeLong system-time-µs)) + (doto (.structKeyWriter doc-writer "xt$tx_time" types/temporal-col-type) + (.writeLong system-time-µs)) - (doto (.structKeyWriter doc-writer "xt$committed?" :bool) - (.writeBoolean (nil? t))) + (doto (.structKeyWriter doc-writer "xt$committed?" :bool) + (.writeBoolean (nil? t))) - (let [e-wtr (.structKeyWriter doc-writer "xt$error" [:union #{:null :clj-form}])] - (if (or (nil? t) (= t abort-exn)) - (doto (.writerForType e-wtr :null) - (.writeNull nil)) - (doto (.writerForType e-wtr :clj-form) - (.writeObject (pr-str t))))) - (.endStruct doc-writer)))) - - (let [live-table (.liveTable live-chunk-tx txs-table) - doc-writer (.writer live-table)] - - (.writeRowId live-table row-id) - - (doto (.writerForName doc-writer "xt$id" :i64) - (.writeLong tx-id)) - - (doto (.writerForName doc-writer "xt$tx_time" types/temporal-col-type) - (.writeLong system-time-µs)) - - (doto (.writerForName doc-writer "xt$committed?" :bool) - (.writeBoolean (nil? t))) - - (let [e-wtr (.writerForName doc-writer "xt$error" [:union #{:null :clj-form}])] - (if (or (nil? t) (= t abort-exn)) - (doto (.writerForType e-wtr :null) - (.writeNull nil)) - (doto (.writerForType e-wtr :clj-form) - (.writeObject (pr-str t)))))) + (let [e-wtr (.structKeyWriter doc-writer "xt$error" [:union #{:null :clj-form}])] + (if (or (nil? t) (= t abort-exn)) + (doto (.writerForType e-wtr :null) + (.writeNull nil)) + (doto (.writerForType e-wtr :clj-form) + (.writeObject (pr-str t))))) + (.endStruct doc-writer))) (.addRows row-counter 1))) @@ -503,7 +458,6 @@ ^IMetadataManager metadata-mgr ^IScanEmitter scan-emitter ^IRaQuerySource ra-src - ^ILiveChunk live-chunk ^ILiveIndex live-idx ^:volatile-mutable ^TransactionInstant latest-completed-tx @@ -515,14 +469,13 @@ IIndexer (indexTx [this {:keys [system-time] :as tx-key} tx-root] - (util/with-open [live-chunk-tx (.startTx live-chunk) - live-idx-tx (.startTx live-idx tx-key)] + (util/with-open [live-idx-tx (.startTx live-idx tx-key)] (let [^DenseUnionVector tx-ops-vec (-> ^ListVector (.getVector tx-root "tx-ops") (.getDataVector)) wm-src (reify IWatermarkSource (openWatermark [_ _tx] - (wm/->wm nil (.openWatermark live-chunk-tx) (.openWatermark live-idx-tx)))) + (wm/->wm nil (.openWatermark live-idx-tx)))) tx-opts {:basis {:tx tx-key, :current-time system-time} :default-tz (ZoneId/of (str (-> (.getVector tx-root "default-tz") @@ -533,12 +486,11 @@ (letfn [(index-tx-ops [^DenseUnionVector tx-ops-vec] (let [tx-ops-rdr (vr/vec->reader tx-ops-vec) - !put-idxer (delay (->put-indexer row-counter live-idx-tx live-chunk-tx tx-ops-rdr system-time)) - !delete-idxer (delay (->delete-indexer row-counter live-idx-tx live-chunk-tx tx-ops-rdr system-time)) - !evict-idxer (delay (->evict-indexer row-counter live-idx-tx live-chunk-tx tx-ops-rdr)) + !put-idxer (delay (->put-indexer row-counter live-idx-tx tx-ops-rdr system-time)) + !delete-idxer (delay (->delete-indexer row-counter live-idx-tx tx-ops-rdr system-time)) + !evict-idxer (delay (->evict-indexer row-counter live-idx-tx tx-ops-rdr)) !call-idxer (delay (->call-indexer allocator ra-src wm-src scan-emitter tx-ops-rdr tx-opts)) - !sql-idxer (delay (->sql-indexer allocator row-counter live-idx-tx live-chunk-tx - tx-ops-rdr ra-src wm-src scan-emitter tx-opts))] + !sql-idxer (delay (->sql-indexer allocator row-counter live-idx-tx tx-ops-rdr ra-src wm-src scan-emitter tx-opts))] (dotimes [tx-op-idx (.valueCount tx-ops-rdr)] (when-let [more-tx-ops (case (.getTypeId tx-ops-rdr tx-op-idx) 0 (.indexOp ^OpIndexer @!sql-idxer tx-op-idx) @@ -569,16 +521,13 @@ (when (not= e abort-exn) (log/debug e "aborted tx")) - (with-open [live-chunk-tx (.startTx live-chunk) - live-idx-tx (.startTx live-idx tx-key)] - (add-tx-row! row-counter live-idx-tx live-chunk-tx tx-key e) - (.commit live-chunk-tx) + (with-open [live-idx-tx (.startTx live-idx tx-key)] + (add-tx-row! row-counter live-idx-tx tx-key e) (.commit live-idx-tx))) (do - (add-tx-row! row-counter live-idx-tx live-chunk-tx tx-key nil) + (add-tx-row! row-counter live-idx-tx tx-key nil) - (.commit live-chunk-tx) (.commit live-idx-tx))) (set! (.-latest-completed-tx this) tx-key) @@ -586,9 +535,6 @@ (finally (.unlock wm-lock wm-lock-stamp))))) - (while (.isBlockFull live-chunk) - (finish-block! this)) - (when (>= (.getChunkRowCount row-counter) rows-per-chunk) (finish-chunk! this)) @@ -614,7 +560,7 @@ (or (maybe-existing-wm) (let [^IWatermark old-wm (.shared-wm this)] (try - (let [^IWatermark shared-wm (wm/->wm latest-completed-tx (.openWatermark live-chunk) (.openWatermark live-idx))] + (let [^IWatermark shared-wm (wm/->wm latest-completed-tx (.openWatermark live-idx))] (set! (.shared-wm this) shared-wm) (doto shared-wm .retain)) (finally @@ -626,28 +572,7 @@ (latestCompletedTx [_] latest-completed-tx) Finish - (finish-block! [this] - (try - (.finishBlock live-chunk) - - (let [wm-lock-stamp (.writeLock wm-lock)] - (try - (when-let [^IWatermark shared-wm (.shared-wm this)] - (set! (.shared-wm this) nil) - (.close shared-wm)) - - (.nextBlock live-chunk) - - (finally - (.unlock wm-lock wm-lock-stamp)))) - - (catch Throwable t - (clojure.tools.logging/error t "fail") - (throw t)))) - (finish-chunk! [this] - @(.finishChunk live-chunk) - (let [chunk-idx (.getChunkIdx row-counter) table-metadata (.finishChunk live-idx chunk-idx)] @@ -664,8 +589,6 @@ (set! (.shared-wm this) nil) (.close shared-wm)) - (.nextChunk live-chunk) - (finally (.unlock wm-lock wm-lock-stamp)))) @@ -680,18 +603,16 @@ :object-store (ig/ref :xtdb/object-store) :metadata-mgr (ig/ref ::meta/metadata-manager) :scan-emitter (ig/ref :xtdb.operator.scan/scan-emitter) - :live-chunk (ig/ref :xtdb/live-chunk) :live-index (ig/ref :xtdb.indexer/live-index) :ra-src (ig/ref ::op/ra-query-source) :rows-per-chunk 102400} opts)) (defmethod ig/init-key :xtdb/indexer - [_ {:keys [allocator object-store metadata-mgr scan-emitter, ra-src - live-chunk live-index, rows-per-chunk]}] + [_ {:keys [allocator object-store metadata-mgr scan-emitter, ra-src, live-index, rows-per-chunk]}] (let [{:keys [latest-completed-tx next-chunk-idx], :or {next-chunk-idx 0}} (meta/latest-chunk-metadata metadata-mgr)] - (->Indexer allocator object-store metadata-mgr scan-emitter ra-src live-chunk live-index + (->Indexer allocator object-store metadata-mgr scan-emitter ra-src live-index latest-completed-tx (RowCounter. next-chunk-idx) diff --git a/core/src/main/clojure/xtdb/live_chunk.clj b/core/src/main/clojure/xtdb/live_chunk.clj deleted file mode 100644 index d1285578bd..0000000000 --- a/core/src/main/clojure/xtdb/live_chunk.clj +++ /dev/null @@ -1,451 +0,0 @@ -(ns xtdb.live-chunk - (:require [clojure.tools.logging :as log] - [juxt.clojars-mirrors.integrant.core :as ig] - [xtdb.blocks :as blocks] - [xtdb.bloom :as bloom] - [xtdb.metadata :as meta] - xtdb.object-store - [xtdb.types :as types] - [xtdb.util :as util] - [xtdb.vector :as vec] - [xtdb.vector.reader :as vr] - [xtdb.vector.writer :as vw]) - (:import [clojure.lang MapEntry] - java.lang.AutoCloseable - (java.util ArrayList HashMap List Map) - (java.util.concurrent CompletableFuture) - java.util.concurrent.atomic.AtomicInteger - (java.util.function Consumer) - (org.apache.arrow.memory BufferAllocator) - (org.apache.arrow.vector BigIntVector ValueVector VectorLoader VectorSchemaRoot VectorUnloader) - org.roaringbitmap.longlong.Roaring64Bitmap - org.roaringbitmap.RoaringBitmap - (xtdb ICursor SliceCursor) - (xtdb.metadata IMetadataManager IMetadataPredicate ITableMetadata ITableMetadataWriter) - xtdb.object_store.ObjectStore - (xtdb.vector IVectorReader IRelationWriter IVectorWriter))) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface IRowCounter - (^int blockIdx []) - (^long blockRowCount []) - (^long chunkRowCount []) - (^ints blockRowCounts []) - - (^void addRows [^int rowCount]) - (^int nextBlock []) - (^void nextChunk [])) - -(deftype RowCounter [^:volatile-mutable ^int block-idx - ^:volatile-mutable ^long chunk-row-count - ^List block-row-counts - ^:volatile-mutable ^long block-row-count] - IRowCounter - (blockIdx [_] block-idx) - (blockRowCount [_] block-row-count) - (chunkRowCount [_] chunk-row-count) - - (blockRowCounts [_] - (int-array (cond-> (vec block-row-counts) - (pos? block-row-count) (conj block-row-count)))) - - (addRows [this row-count] - (set! (.block-row-count this) (+ block-row-count row-count))) - - (nextBlock [this] - (set! (.block-idx this) (inc block-idx)) - (.add block-row-counts block-row-count) - (set! (.chunk-row-count this) (+ chunk-row-count block-row-count)) - (set! (.block-row-count this) 0)) - - (nextChunk [this] - (.clear block-row-counts) - (set! (.chunk-row-count this) 0) - (set! (.block-idx this) 0))) - -(defn- ->row-counter ^xtdb.live_chunk.IRowCounter [^long block-idx] - (let [block-row-counts (ArrayList. ^List (repeat block-idx 0))] - (RowCounter. block-idx 0 block-row-counts 0))) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface ILiveTableTx - (^xtdb.vector.IRelationWriter writer []) - - (^void writeRowId [^long rowId]) - (^boolean containsRowId [^long rowId]) - - (^xtdb.live_chunk.ILiveTableWatermark openWatermark [^boolean retain]) - - (^void commit []) - (^void close [])) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface ILiveTableWatermark - (^java.util.Map columnTypes []) - (^xtdb.ICursor #_ liveBlocks [^java.util.Set #_ colNames, - ^xtdb.metadata.IMetadataPredicate metadataPred]) - (^void close [])) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface ILiveTable - (^xtdb.live_chunk.ILiveTableWatermark openWatermark [^boolean retain]) - (^xtdb.live_chunk.ILiveTableTx startTx []) - (^void finishBlock []) - (^java.util.concurrent.CompletableFuture finishChunk []) - - (^void close [])) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface ILiveChunkWatermark - (^xtdb.live_chunk.ILiveTableWatermark liveTable [^String table]) - (^java.util.Map allColumnTypes []) - (^void close [])) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface ILiveChunkTx - (^xtdb.live_chunk.ILiveTableTx liveTable [^String table]) - (^xtdb.live_chunk.ILiveChunkWatermark openWatermark []) - - (^long nextRowId []) - (^void commit []) - (^void close [])) - -#_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} -(definterface ILiveChunk - (^xtdb.live_chunk.ILiveTable liveTable [^String table]) - (^xtdb.live_chunk.ILiveChunkWatermark openWatermark []) - (^xtdb.live_chunk.ILiveChunkTx startTx []) - - (^long chunkIdx []) - (^boolean isChunkFull []) - (^boolean isBlockFull []) - - (^void finishBlock []) - (^void nextBlock []) - (^java.util.concurrent.CompletableFuture finishChunk []) - (^void nextChunk []) - (^void close [])) - -(defn- ->excluded-block-idxs ^org.roaringbitmap.RoaringBitmap [^ITableMetadata table-metadata, ^IMetadataPredicate metadata-pred] - (let [exclude-block-idxs (RoaringBitmap.)] - (when-let [pred (some-> metadata-pred (.build table-metadata))] - (dotimes [block-idx (.blockCount table-metadata)] - (when-not (.test pred block-idx) - (.add exclude-block-idxs block-idx)))) - - exclude-block-idxs)) - -(defn- without-block-idxs [^ICursor inner, ^RoaringBitmap exclude-block-idxs] - (let [!block-idx (AtomicInteger. 0)] - (reify ICursor - (tryAdvance [_ c] - (let [!advanced? (volatile! false)] - (while (and (not @!advanced?) - (.tryAdvance inner - (reify Consumer - (accept [_ el] - (let [block-idx (.getAndIncrement !block-idx)] - (when-not (.contains exclude-block-idxs block-idx) - (.accept c el) - (vreset! !advanced? true)))))))) - @!advanced?)) - - (close [_] (.close inner))))) - -(defn- open-wm-rel ^xtdb.vector.RelationReader [^IRelationWriter rel, ^BigIntVector row-id-vec, retain?] - (let [out-cols (ArrayList.)] - (try - (.syncRowCount rel) - (doseq [^ValueVector v (cons row-id-vec - (->> (vals rel) - (map #(.getVector ^IVectorWriter %))))] - (.add out-cols (vr/vec->reader (cond-> v - retain? (util/slice-vec))))) - - (vr/rel-reader out-cols) - - (catch Throwable t - (when retain? (util/close out-cols)) - (throw t))))) - -(deftype LiveTableTx [^BufferAllocator allocator, ^ObjectStore object-store - ^String table-name, ^ITableMetadataWriter table-metadata-writer - ^IRelationWriter static-rel, ^BigIntVector static-row-id-vec, ^Roaring64Bitmap static-row-id-bitmap - ^IRelationWriter transient-rel, ^BigIntVector transient-row-id-vec, ^Roaring64Bitmap transient-row-id-bitmap - ^IRowCounter row-counter] - ILiveTableTx - (writer [_] transient-rel) - - (writeRowId [_ row-id] - (.addLong transient-row-id-bitmap row-id) - - (let [dest-idx (.getValueCount transient-row-id-vec)] - (.setSafe transient-row-id-vec dest-idx row-id) - (.setValueCount transient-row-id-vec (inc dest-idx)))) - - (containsRowId [_ row-id] - (or (.contains static-row-id-bitmap row-id) - (.contains transient-row-id-bitmap row-id))) - - (openWatermark [_ retain?] - (let [col-types (->> transient-rel - (into {} (map (fn [[col-name ^IVectorWriter col]] - (let [v (.getVector col)] - (MapEntry/create col-name (types/field->col-type (.getField v)))))))) - row-counts (.blockRowCounts row-counter) - static-wm-rel (open-wm-rel static-rel static-row-id-vec retain?) - transient-wm-rel (open-wm-rel transient-rel transient-row-id-vec false)] - - (reify ILiveTableWatermark - (columnTypes [_] col-types) - - (liveBlocks [_ col-names metadata-pred] - (let [excluded-block-idxs (->excluded-block-idxs (.tableMetadata table-metadata-writer) metadata-pred)] - (util/->concat-cursor (-> (vr/rel-reader (->> static-wm-rel - (filter (comp col-names #(.getName ^IVectorReader %))))) - (vr/with-absent-cols allocator col-names) - (SliceCursor. row-counts) - (without-block-idxs excluded-block-idxs)) - (-> (vr/rel-reader (->> transient-wm-rel - (filter (comp col-names #(.getName ^IVectorReader %))))) - (vr/with-absent-cols allocator col-names) - (SliceCursor. (int-array [(.rowCount transient-wm-rel)])))))) - - AutoCloseable - (close [_] (when retain? (.close static-wm-rel)))))) - - (commit [_] - (.addRows row-counter (.getValueCount transient-row-id-vec)) - (doto static-row-id-bitmap (.or transient-row-id-bitmap)) - - (doto (vw/->writer static-row-id-vec) - (vw/append-vec (vr/vec->reader transient-row-id-vec)) - (.syncValueCount)) - - (let [copier (.rowCopier static-rel (vw/rel-wtr->rdr transient-rel))] - (dotimes [idx (.getValueCount transient-row-id-vec)] - (.copyRow copier idx))) - - (.clear transient-rel)) - - AutoCloseable - (close [_] - (.close transient-rel) - (util/try-close transient-row-id-vec))) - -(deftype LiveTable [^BufferAllocator allocator, ^ObjectStore object-store, ^IMetadataManager metadata-mgr - ^String table-name, - ^IRelationWriter static-rel, ^BigIntVector row-id-vec, ^Roaring64Bitmap row-id-bitmap - ^ITableMetadataWriter table-metadata-writer - ^long chunk-idx, ^IRowCounter row-counter] - ILiveTable - (startTx [_] - (LiveTableTx. allocator object-store table-name table-metadata-writer - static-rel row-id-vec row-id-bitmap - (vw/->rel-writer allocator) (BigIntVector. (types/col-type->field "_row_id" :i64) allocator) (Roaring64Bitmap.) - row-counter)) - - (openWatermark [_ retain?] - (let [col-types (->> (vals static-rel) - (into {} (map (fn [^IVectorWriter col] - (let [v (.getVector col)] - (MapEntry/create (.getName v) (types/field->col-type (.getField v)))))))) - row-counts (.blockRowCounts row-counter) - wm-rel (open-wm-rel static-rel row-id-vec retain?)] - - (reify ILiveTableWatermark - (columnTypes [_] col-types) - - (liveBlocks [_ col-names metadata-pred] - (let [excluded-block-idxs (->excluded-block-idxs (.tableMetadata table-metadata-writer) metadata-pred)] - (-> (vr/rel-reader (->> wm-rel - (filter (comp col-names #(.getName ^IVectorReader %))))) - (vr/with-absent-cols allocator col-names) - (SliceCursor. row-counts) - (without-block-idxs excluded-block-idxs)))) - - AutoCloseable - (close [_] (when retain? (.close wm-rel)))))) - - (finishBlock [_] - (let [block-meta-wtr (.writeBlockMetadata table-metadata-writer (.blockIdx row-counter))] - (doseq [^IVectorReader live-vec (cons (vr/vec->reader row-id-vec) (seq (vw/rel-wtr->rdr static-rel)))] - (.writeMetadata block-meta-wtr (.select live-vec (.chunkRowCount row-counter) (.blockRowCount row-counter)))) - (.endBlock block-meta-wtr)) - - (.nextBlock row-counter)) - - (finishChunk [_] - (let [row-counts (.blockRowCounts row-counter) - block-meta-wtr (.writeBlockMetadata table-metadata-writer -1)] - (-> (CompletableFuture/allOf - (->> (cons row-id-vec (map #(.getVector ^IVectorWriter %) (vals static-rel))) - (map (fn [^ValueVector live-vec] - (let [live-root (VectorSchemaRoot/of (into-array [live-vec]))] - (.writeMetadata block-meta-wtr (vr/vec->reader live-vec)) - - (.putObject object-store (meta/->chunk-obj-key chunk-idx table-name (.getName live-vec)) - (with-open [write-root (VectorSchemaRoot/create (.getSchema live-root) allocator)] - (let [loader (VectorLoader. write-root)] - (with-open [^ICursor slices (blocks/->slices live-root row-counts)] - (let [buf (util/build-arrow-ipc-byte-buffer write-root :file - (fn [write-batch!] - (.forEachRemaining slices - (reify Consumer - (accept [_ sliced-root] - (with-open [arb (.getRecordBatch (VectorUnloader. sliced-root))] - (.load loader arb) - (write-batch!)))))))] - (.nextChunk row-counter) - buf)))))))) - (into-array CompletableFuture))) - - (util/then-compose - (fn [_] - (.endBlock block-meta-wtr) - (.finishChunk table-metadata-writer)))))) - - AutoCloseable - (close [_] - (util/try-close static-rel) - (util/try-close row-id-vec) - (util/try-close table-metadata-writer))) - -(deftype LiveChunkTx [^BufferAllocator allocator - ^ObjectStore object-store - ^IMetadataManager metadata-mgr - ^Map live-tables, ^Map live-table-txs - ^long chunk-idx, ^long tx-start-row, ^IRowCounter row-counter - ^:volatile-mutable ^long tx-row-count] - ILiveChunkTx - (liveTable [_ table-name] - (letfn [(->live-table [table] - (LiveTable. allocator object-store metadata-mgr table - (vw/->rel-writer allocator) (BigIntVector. (types/col-type->field "_row_id" :i64) allocator) (Roaring64Bitmap.) - (.openTableMetadataWriter metadata-mgr table chunk-idx) - chunk-idx (->row-counter (.blockIdx row-counter)))) - - (->live-table-tx [table-name] - (-> ^ILiveTable (.computeIfAbsent live-tables table-name - (util/->jfn ->live-table)) - (.startTx)))] - - (.computeIfAbsent live-table-txs table-name - (util/->jfn ->live-table-tx)))) - - (openWatermark [_] - (util/with-close-on-catch [wms (HashMap.)] - (doseq [[table-name ^ILiveTableTx live-table] live-table-txs] - (.put wms table-name (.openWatermark live-table false))) - - (doseq [[table-name ^ILiveTable live-table] live-tables] - (.computeIfAbsent wms table-name - (util/->jfn (fn [_] (.openWatermark live-table false))))) - - (reify ILiveChunkWatermark - (liveTable [_ table-name] (.get wms table-name)) - - (allColumnTypes [_] (update-vals wms #(.columnTypes ^ILiveTableWatermark %))) - - AutoCloseable - (close [_] (run! util/try-close (.values wms)))))) - - (nextRowId [this] - (let [tx-row-count (.tx-row-count this)] - (set! (.tx-row-count this) (inc tx-row-count)) - (+ tx-start-row tx-row-count))) - - (commit [_] - (doseq [^ILiveTableTx live-table (.values live-table-txs)] - (.commit live-table)) - (.addRows row-counter tx-row-count)) - - AutoCloseable - (close [_] - (run! util/try-close (.values live-table-txs)) - (.clear live-table-txs))) - -(deftype LiveChunk [^BufferAllocator allocator - ^ObjectStore object-store - ^IMetadataManager metadata-mgr - ^long rows-per-block, ^long rows-per-chunk - ^Map live-tables - ^:volatile-mutable ^long chunk-idx - ^IRowCounter row-counter] - ILiveChunk - (liveTable [_ table] (.get live-tables table)) - - (startTx [_] - (LiveChunkTx. allocator object-store metadata-mgr - live-tables (HashMap.) - chunk-idx (+ chunk-idx (.chunkRowCount row-counter) (.blockRowCount row-counter)) - row-counter 0)) - - (openWatermark [_] - (util/with-close-on-catch [wms (HashMap.)] - (doseq [[table-name ^ILiveTable live-table] live-tables] - (.put wms table-name (.openWatermark live-table true))) - - (reify ILiveChunkWatermark - (liveTable [_ table-name] (.get wms table-name)) - - (allColumnTypes [_] (update-vals wms #(.columnTypes ^ILiveTableWatermark %))) - - AutoCloseable - (close [_] (util/close wms))))) - - (chunkIdx [_] chunk-idx) - (isBlockFull [_] (>= (.blockRowCount row-counter) rows-per-block)) - (isChunkFull [_] (>= (.chunkRowCount row-counter) rows-per-chunk)) - - (finishBlock [_] - (doseq [^ILiveTable live-table (.values live-tables)] - (.finishBlock live-table))) - - (nextBlock [_] (.nextBlock row-counter)) - - (finishChunk [_] - (let [futs (for [^ILiveTable live-table (.values live-tables)] - (.finishChunk live-table))] - - (CompletableFuture/allOf (into-array CompletableFuture futs)))) - - (nextChunk [this] - (run! util/try-close (.values live-tables)) - (.clear live-tables) - - (set! (.chunk-idx this) - (+ (.chunk-idx this) (.chunkRowCount row-counter))) - - (.nextChunk row-counter)) - - AutoCloseable - (close [_] - (run! util/try-close (.values live-tables)) - (.clear live-tables))) - -(defmethod ig/prep-key :xtdb/live-chunk [_ opts] - (merge {:rows-per-block 1024 - :rows-per-chunk 102400 - :allocator (ig/ref :xtdb/allocator) - :object-store (ig/ref :xtdb/object-store) - :metadata-mgr (ig/ref ::meta/metadata-manager)} - opts)) - -(defmethod ig/init-key :xtdb/live-chunk [_ {:keys [allocator object-store metadata-mgr ^long rows-per-block ^long rows-per-chunk]}] - (let [^long chunk-idx (:next-chunk-idx (meta/latest-chunk-metadata metadata-mgr) 0) - bloom-false-positive-probability (bloom/bloom-false-positive-probability? rows-per-chunk)] - - (when (> bloom-false-positive-probability 0.05) - (log/warn "Bloom should be sized for large chunks:" rows-per-chunk - "false positive probability:" bloom-false-positive-probability - "bits:" bloom/bloom-bits - "can be set via system property xtdb.bloom.bits")) - - (LiveChunk. allocator object-store metadata-mgr - rows-per-block rows-per-chunk - (HashMap.) - chunk-idx (->row-counter 0)))) - -(defmethod ig/halt-key! :xtdb/live-chunk [_ live-chunk] - (util/try-close live-chunk)) diff --git a/core/src/main/clojure/xtdb/metadata.clj b/core/src/main/clojure/xtdb/metadata.clj index 00879544e0..0b037092ed 100644 --- a/core/src/main/clojure/xtdb/metadata.clj +++ b/core/src/main/clojure/xtdb/metadata.clj @@ -108,9 +108,6 @@ (defn- ->table-metadata-obj-key [chunk-idx table-name] (format "chunk-%s/%s/metadata.arrow" (util/->lex-hex-string chunk-idx) table-name)) -(defn ->chunk-obj-key [chunk-idx table-name column-name] - (format "chunk-%s/%s/content-%s.arrow" (util/->lex-hex-string chunk-idx) table-name column-name)) - (defn- obj-key->chunk-idx [obj-key] (some-> (second (re-matches #"chunk-metadata/(\p{XDigit}+).transit.json" obj-key)) (util/<-lex-hex-string))) diff --git a/core/src/main/clojure/xtdb/node.clj b/core/src/main/clojure/xtdb/node.clj index d7b8fcfd0e..ad8bdebbcd 100644 --- a/core/src/main/clojure/xtdb/node.clj +++ b/core/src/main/clojure/xtdb/node.clj @@ -124,7 +124,6 @@ :xtdb/allocator {} :xtdb/default-tz nil :xtdb/indexer {} - :xtdb/live-chunk {} :xtdb.indexer/live-index {} :xtdb/ingester {} :xtdb.metadata/metadata-manager {} diff --git a/core/src/main/clojure/xtdb/watermark.clj b/core/src/main/clojure/xtdb/watermark.clj index 165f470ed3..150ef58be3 100644 --- a/core/src/main/clojure/xtdb/watermark.clj +++ b/core/src/main/clojure/xtdb/watermark.clj @@ -2,18 +2,15 @@ (:require [clojure.tools.logging :as log] xtdb.api.protocols xtdb.indexer.live-index - xtdb.live-chunk [xtdb.util :as util]) (:import java.lang.AutoCloseable java.util.concurrent.atomic.AtomicInteger xtdb.api.protocols.TransactionInstant - xtdb.indexer.live_index.ILiveIndexWatermark - xtdb.live_chunk.ILiveChunkWatermark)) + xtdb.indexer.live_index.ILiveIndexWatermark)) #_{:clj-kondo/ignore [:unused-binding :clojure-lsp/unused-public-var]} (definterface IWatermark (^xtdb.api.protocols.TransactionInstant txBasis []) - (^xtdb.live_chunk.ILiveChunkWatermark liveChunk []) (^xtdb.indexer.live_index.ILiveIndexWatermark liveIndex []) (^void retain []) @@ -25,11 +22,9 @@ (definterface IWatermarkSource (^xtdb.watermark.IWatermark openWatermark [^xtdb.api.protocols.TransactionInstant txKey])) -(deftype Watermark [^TransactionInstant tx-key, ^ILiveChunkWatermark live-chunk, ^ILiveIndexWatermark live-idx-wm - ^AtomicInteger ref-cnt] +(deftype Watermark [^TransactionInstant tx-key, ^ILiveIndexWatermark live-idx-wm, ^AtomicInteger ref-cnt] IWatermark (txBasis [_] tx-key) - (liveChunk [_] live-chunk) (liveIndex [_] live-idx-wm) (retain [this] @@ -43,8 +38,7 @@ (when (zero? (.decrementAndGet ref-cnt)) (log/trace "close wm" (hash this)) - (util/try-close live-chunk) (util/try-close live-idx-wm)))) -(defn ->wm ^xtdb.watermark.IWatermark [tx-key live-chunk live-idx-wm] - (Watermark. tx-key live-chunk live-idx-wm (AtomicInteger. 1))) +(defn ->wm ^xtdb.watermark.IWatermark [tx-key live-idx-wm] + (Watermark. tx-key live-idx-wm (AtomicInteger. 1))) diff --git a/src/main/clojure/xtdb/test_util.clj b/src/main/clojure/xtdb/test_util.clj index d45cc5dc56..bb9ca2621a 100644 --- a/src/main/clojure/xtdb/test_util.clj +++ b/src/main/clojure/xtdb/test_util.clj @@ -125,7 +125,6 @@ (with-opts {:xtdb.log/memory-log {:instant-src (->mock-clock)}} f)) (defn finish-chunk! [node] - (idx/finish-block! (component node :xtdb/indexer)) (idx/finish-chunk! (component node :xtdb/indexer))) (defn open-vec @@ -246,10 +245,7 @@ :xtdb.buffer-pool/buffer-pool {:cache-path (.resolve node-dir buffers-dir)} :xtdb.object-store/file-system-object-store {:root-path (.resolve node-dir "objects")} :xtdb/indexer (->> {:rows-per-chunk rows-per-chunk} - (into {} (filter val))) - :xtdb/live-chunk (->> {:rows-per-block rows-per-block - :rows-per-chunk rows-per-chunk} - (into {} (filter val)))}))) + (into {} (filter val)))}))) (defn ->local-submit-node ^java.lang.AutoCloseable [{:keys [^Path node-dir]}] (node/start-submit-node {:xtdb.tx-producer/tx-producer {:clock (->mock-clock)} diff --git a/src/test/clojure/xtdb/datalog_test.clj b/src/test/clojure/xtdb/datalog_test.clj index 67a518b8cb..76844085a7 100644 --- a/src/test/clojure/xtdb/datalog_test.clj +++ b/src/test/clojure/xtdb/datalog_test.clj @@ -1072,7 +1072,7 @@ :spectre])))) (t/deftest bug-non-string-table-names-599 - (with-open [node (node/start-node {:xtdb/live-chunk {:rows-per-block 10, :rows-per-chunk 1000}})] + (with-open [node (node/start-node {:xtdb/indexer {:rows-per-chunk 1000}})] (letfn [(submit-ops! [ids] (last (for [tx-ops (->> (for [id ids] [:put :t1 {:xt/id id, @@ -1095,7 +1095,7 @@ (t/is (= 160 (count-table tx))))))) (t/deftest bug-dont-throw-on-non-existing-column-597 - (with-open [node (node/start-node {:xtdb/live-chunk {:rows-per-block 10, :rows-per-chunk 1000}})] + (with-open [node (node/start-node {:xtdb/indexer {:rows-per-chunk 1000}})] (letfn [(submit-ops! [ids] (last (for [tx-ops (->> (for [id ids] [:put :t1 {:xt/id id, @@ -1117,7 +1117,7 @@ :where [(match :xt_docs [xt/id some-attr])]})))))) (t/deftest add-better-metadata-support-for-keywords - (with-open [node (node/start-node {:xtdb/live-chunk {:rows-per-block 10, :rows-per-chunk 1000}})] + (with-open [node (node/start-node {:xtdb/indexer {:rows-per-chunk 1000}})] (letfn [(submit-ops! [ids] (last (for [tx-ops (->> (for [id ids] [:put :t1 {:xt/id id, @@ -2190,7 +2190,7 @@ :where [(match :foo {:xt/id id})]})))) (t/deftest test-metadata-filtering-for-time-data-607 - (with-open [node (node/start-node {:xtdb/live-chunk {:rows-per-block 1, :rows-per-chunk 1}})] + (with-open [node (node/start-node {:xtdb/indexer {:rows-per-chunk 1}})] (xt/submit-tx node [[:put :xt_docs {:xt/id 1 :start-date #time/date "2000-01-01"}] [:put :xt_docs {:xt/id 2 :start-date #time/date "3000-01-01"}]]) (t/is (= [{:id 1}] @@ -2319,7 +2319,7 @@ [(= #time/time "08:12:13.366" #time/time "08:12:13.366") d]]})))) (t/deftest bug-temporal-queries-wrong-at-boundary-2531 - (with-open [node (node/start-node {:xtdb/live-chunk {:rows-per-block 10, :rows-per-chunk 10} + (with-open [node (node/start-node {:xtdb/indexer {:rows-per-chunk 10} :xtdb.tx-producer/tx-producer {:instant-src (tu/->mock-clock)} :xtdb.log/memory-log {:instant-src (tu/->mock-clock)}})] (doseq [i (range 10)] diff --git a/src/test/clojure/xtdb/indexer_test.clj b/src/test/clojure/xtdb/indexer_test.clj index 565b592ce1..0663d13588 100644 --- a/src/test/clojure/xtdb/indexer_test.clj +++ b/src/test/clojure/xtdb/indexer_test.clj @@ -99,21 +99,6 @@ (t/is (= last-tx-key (tu/then-await-tx last-tx-key node (Duration/ofSeconds 2)))) - (t/testing "watermark" - (with-open [^IWatermark watermark (.openWatermark wm-src last-tx-key)] - (let [live-blocks (-> (.liveChunk watermark) - (.liveTable "device_info") - (.liveBlocks #{"xt$id" "model"} nil)) - !res (volatile! [])] - (.forEachRemaining live-blocks - (reify Consumer - (accept [_ content-cols] - (vswap! !res conj (vr/rel->rows content-cols))))) - - (t/is (= [[{:xt$id "device-info-demo000000", :model "pinto"} - {:xt$id "device-info-demo000001", :model "mustang"}]] - @!res))))) - (tu/finish-chunk! node) (t/is (= {:latest-completed-tx last-tx-key @@ -121,13 +106,10 @@ (-> (meta/latest-chunk-metadata mm) (select-keys [:latest-completed-tx :next-chunk-idx])))) - (let [objects-list (->> (.listObjects os "chunk-00/device_info") (filter #(str/ends-with? % "/metadata.arrow")))] - (t/is (= 1 (count objects-list))) - (t/is (= ["chunk-00/device_info/metadata.arrow"] objects-list))) - (tj/check-json (.toPath (io/as-file (io/resource "xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format"))) (.resolve node-dir "objects")) + #_ ; TODO port to buffer pool test that doesn't depend on the structure of the indexer (t/testing "buffer pool" (let [buffer-name "chunk-00/device_info/metadata.arrow" ^ArrowBuf buffer @(.getBuffer bp buffer-name) @@ -419,10 +401,12 @@ (select-keys [:latest-completed-tx :next-chunk-idx])))) (let [objs (.listObjects os)] - (t/is (= 1 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_info/metadata\.arrow" %) objs)))) - (t/is (= 4 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_readings/metadata\.arrow" %) objs)))) - (t/is (= 1 (count (filter #(re-matches #"chunk-.*/device_info/content-api_version\.arrow" %) objs)))) - (t/is (= 4 (count (filter #(re-matches #"chunk-.*/device_readings/content-battery_level\.arrow" %) objs)))))))))) + (t/is (= 4 (count (filter #(re-matches #"chunk-metadata/\p{XDigit}+\.transit.json" %) objs)))) + (t/is (= 2 (count (filter #(re-matches #"tables/device_info/chunks/.+\.arrow" %) objs)))) + (t/is (= 4 (count (filter #(re-matches #"tables/device_readings/chunks/leaf-\p{XDigit}+\.arrow" %) objs)))) + (t/is (= 4 (count (filter #(re-matches #"tables/device_readings/chunks/trie-\p{XDigit}+\.arrow" %) objs)))) + (t/is (= 4 (count (filter #(re-matches #"tables/xt\$txs/chunks/leaf-\p{XDigit}+\.arrow" %) objs)))) + (t/is (= 4 (count (filter #(re-matches #"tables/xt\$txs/chunks/trie-\p{XDigit}+\.arrow" %) objs)))))))))) (t/deftest can-ingest-ts-devices-mini-into-multiple-nodes (let [node-dir (util/->path "target/can-ingest-ts-devices-mini-into-multiple-nodes") @@ -455,9 +439,12 @@ (t/is (= last-tx-key (tu/latest-completed-tx node))) (let [objs (.listObjects os)] - (t/is (= 13 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_(?:info|readings)/metadata.arrow" %) objs)))) - (t/is (= 2 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_info/content-api_version\.arrow" %) objs)))) - (t/is (= 11 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_readings/content-battery_level\.arrow" %) objs))))))))))) + (t/is (= 11 (count (filter #(re-matches #"chunk-metadata/\p{XDigit}+\.transit.json" %) objs)))) + (t/is (= 4 (count (filter #(re-matches #"tables/device_info/chunks/.+\.arrow" %) objs)))) + (t/is (= 11 (count (filter #(re-matches #"tables/device_readings/chunks/leaf-\p{XDigit}+\.arrow" %) objs)))) + (t/is (= 11 (count (filter #(re-matches #"tables/device_readings/chunks/trie-\p{XDigit}+\.arrow" %) objs)))) + (t/is (= 11 (count (filter #(re-matches #"tables/xt\$txs/chunks/leaf-\p{XDigit}+\.arrow" %) objs)))) + (t/is (= 11 (count (filter #(re-matches #"tables/xt\$txs/chunks/trie-\p{XDigit}+\.arrow" %) objs))))))))))) (t/deftest can-ingest-ts-devices-mini-with-stop-start-and-reach-same-state (let [node-dir (util/->path "target/can-ingest-ts-devices-mini-with-stop-start-and-reach-same-state") @@ -499,10 +486,12 @@ (t/is (< next-chunk-idx (count first-half-tx-ops))) (let [objs (.listObjects os)] - (t/is (= 2 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_info/metadata\.arrow" %) objs)))) - (t/is (= 5 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_readings/metadata\.arrow" %) objs)))) - (t/is (= 2 (count (filter #(re-matches #"chunk-.*/device_info/content-api_version\.arrow" %) objs)))) - (t/is (= 5 (count (filter #(re-matches #"chunk-.*/device_readings/content-battery_level\.arrow" %) objs)))))) + (t/is (= 5 (count (filter #(re-matches #"chunk-metadata/\p{XDigit}+\.transit.json" %) objs)))) + (t/is (= 4 (count (filter #(re-matches #"tables/device_info/chunks/.+\.arrow" %) objs)))) + (t/is (= 5 (count (filter #(re-matches #"tables/device_readings/chunks/leaf-\p{XDigit}+\.arrow" %) objs)))) + (t/is (= 5 (count (filter #(re-matches #"tables/device_readings/chunks/trie-\p{XDigit}+\.arrow" %) objs)))) + (t/is (= 5 (count (filter #(re-matches #"tables/xt\$txs/chunks/leaf-\p{XDigit}+\.arrow" %) objs)))) + (t/is (= 5 (count (filter #(re-matches #"tables/xt\$txs/chunks/trie-\p{XDigit}+\.arrow" %) objs)))))) (t/is (= :utf8 (.columnType mm "device_readings" "xt$id"))) @@ -538,10 +527,12 @@ ^IMetadataManager mm (tu/component node ::meta/metadata-manager)]] (let [objs (.listObjects os)] - (t/is (= 2 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_info/metadata\.arrow" %) objs)))) - (t/is (= 11 (count (filter #(re-matches #"chunk-\p{XDigit}+/device_readings/metadata\.arrow" %) objs)))) - (t/is (= 2 (count (filter #(re-matches #"chunk-.*/device_info/content-api_version\.arrow" %) objs)))) - (t/is (= 11 (count (filter #(re-matches #"chunk-.*/device_readings/content-battery_level\.arrow" %) objs))))) + (t/is (= 11 (count (filter #(re-matches #"chunk-metadata/\p{XDigit}+\.transit.json" %) objs)))) + (t/is (= 4 (count (filter #(re-matches #"tables/device_info/chunks/.+\.arrow" %) objs)))) + (t/is (= 11 (count (filter #(re-matches #"tables/device_readings/chunks/leaf-\p{XDigit}+\.arrow" %) objs)))) + (t/is (= 11 (count (filter #(re-matches #"tables/device_readings/chunks/trie-\p{XDigit}+\.arrow" %) objs)))) + (t/is (= 11 (count (filter #(re-matches #"tables/xt\$txs/chunks/leaf-\p{XDigit}+\.arrow" %) objs)))) + (t/is (= 11 (count (filter #(re-matches #"tables/xt\$txs/chunks/trie-\p{XDigit}+\.arrow" %) objs))))) (t/is (= :utf8 (.columnType mm "device_info" "xt$id"))))))))))))) @@ -625,63 +616,3 @@ (tj/check-json (.toPath (io/as-file (io/resource "xtdb/indexer-test/can-index-sql-insert"))) (.resolve node-dir "objects")))))) - -(deftest test-skips-irrelevant-live-blocks-632 - (with-open [node (node/start-node {:xtdb/live-chunk {:rows-per-block 2, :rows-per-chunk 10}})] - (-> (xt/submit-tx node [[:put :xt_docs {:name "Håkan", :xt/id :hak}]]) - (tu/then-await-tx node)) - - (tu/finish-chunk! node) - - (xt/submit-tx node [[:put :xt_docs {:name "Dan", :xt/id :dan}] - [:put :xt_docs {:name "Ivan", :xt/id :iva}]]) - - (-> (xt/submit-tx node [[:put :xt_docs {:name "James", :xt/id :jms}] - [:put :xt_docs {:name "Jon", :xt/id :jon}]]) - (tu/then-await-tx node)) - - (let [^IMetadataManager metadata-mgr (tu/component node ::meta/metadata-manager) - ^IWatermarkSource wm-src (tu/component node :xtdb/indexer)] - (with-open [params (tu/open-params {'?name "Ivan"})] - (let [gt-literal-selector (expr.meta/->metadata-selector '(> name "Ivan") '{name :utf8} {}) - gt-param-selector (expr.meta/->metadata-selector '(> name ?name) '{name :utf8} params)] - - (t/is (= #{0} (set (keys (.chunksMetadata metadata-mgr))))) - - (letfn [(test-live-blocks [^IWatermark wm, metadata-pred] - (with-open [live-blocks (-> (.liveChunk wm) - (.liveTable "xt_docs") - (.liveBlocks #{"_row_id" "name"} metadata-pred))] - (let [!res (atom [])] - (.forEachRemaining live-blocks - (reify Consumer - (accept [_ in-rel] - (swap! !res conj (vr/rel->rows in-rel))))) - @!res)))] - - (with-open [wm1 (.openWatermark wm-src nil)] - (t/is (= [[{:_row_id 2, :name "Dan"} {:_row_id 3, :name "Ivan"}] - [{:_row_id 5, :name "James"} {:_row_id 6, :name "Jon"}]] - (test-live-blocks wm1 nil)) - "no selector") - - (t/is (= [[{:_row_id 5, :name "James"} {:_row_id 6, :name "Jon"}]] - (test-live-blocks wm1 gt-literal-selector)) - "only second block, literal selector") - - (t/is (= [[{:_row_id 5, :name "James"} {:_row_id 6, :name "Jon"}]] - (test-live-blocks wm1 gt-param-selector)) - "only second block, param selector") - - (let [next-tx (-> (xt/submit-tx node [[:put :xt_docs {:name "Jeremy", :xt/id :jdt}]]) - (tu/then-await-tx node))] - - (with-open [wm2 (.openWatermark wm-src next-tx)] - (t/is (= [[{:_row_id 5, :name "James"} {:_row_id 6, :name "Jon"}]] - (test-live-blocks wm1 gt-literal-selector)) - "replay with wm1") - - (t/is (= [[{:_row_id 5, :name "James"} {:_row_id 6, :name "Jon"}] - [{:_row_id 8, :name "Jeremy"}]] - (test-live-blocks wm2 gt-literal-selector)) - "now on wm2")))))))))) diff --git a/src/test/clojure/xtdb/operator/scan_test.clj b/src/test/clojure/xtdb/operator/scan_test.clj index 5c942725bd..4a7c71ad5a 100644 --- a/src/test/clojure/xtdb/operator/scan_test.clj +++ b/src/test/clojure/xtdb/operator/scan_test.clj @@ -40,7 +40,7 @@ {:node node}))))) (t/deftest test-chunk-boundary - (with-open [node (node/start-node {:xtdb/live-chunk {:rows-per-block 5, :rows-per-chunk 20}})] + (with-open [node (node/start-node {:xtdb/indexer {:rows-per-chunk 20}})] (->> (for [i (range 110)] [:put :xt_docs {:xt/id i}]) (partition-all 10) diff --git a/src/test/clojure/xtdb/operator_test.clj b/src/test/clojure/xtdb/operator_test.clj index ceef6d4687..29ab406154 100644 --- a/src/test/clojure/xtdb/operator_test.clj +++ b/src/test/clojure/xtdb/operator_test.clj @@ -12,7 +12,7 @@ (t/use-fixtures :once tu/with-allocator) (t/deftest test-find-gt-ivan - (with-open [node (node/start-node {:xtdb/live-chunk {:rows-per-block 2, :rows-per-chunk 10}})] + (with-open [node (node/start-node {:xtdb/indexer {:rows-per-chunk 10}})] (-> (xt/submit-tx node [[:put :xt_docs {:name "Håkan", :xt/id :hak}]]) (tu/then-await-tx node)) @@ -39,8 +39,9 @@ (t/is (= #{0 2} (set (keys (.chunksMetadata metadata-mgr))))) + #_ ; TODO reinstate metadata (let [expected-match [(meta/map->ChunkMatch - {:chunk-idx 2, :block-idxs (doto (RoaringBitmap.) (.add 1)), :col-names #{"_row_id" "xt$id" "name"}})]] + {:chunk-idx 2, :block-idxs (doto (RoaringBitmap.) (.add 1)), :col-names #{"xt$id" "name"}})]] (t/is (= expected-match (meta/matching-chunks metadata-mgr "xt_docs" (expr.meta/->metadata-selector '(> name "Ivan") '{name :utf8} {}))) @@ -63,7 +64,7 @@ tx2))))))) (t/deftest test-find-eq-ivan - (with-open [node (node/start-node {:xtdb/live-chunk {:rows-per-block 3, :rows-per-chunk 10}})] + (with-open [node (node/start-node {:xtdb/indexer {:rows-per-chunk 10}})] (-> (xt/submit-tx node [[:put :xt_docs {:name "Håkan", :xt/id :hak}] [:put :xt_docs {:name "James", :xt/id :jms}] [:put :xt_docs {:name "Ivan", :xt/id :iva}]]) @@ -78,8 +79,10 @@ (tu/finish-chunk! node) (let [^IMetadataManager metadata-mgr (tu/component node ::meta/metadata-manager)] (t/is (= #{0 4} (set (keys (.chunksMetadata metadata-mgr))))) + + #_ ; TODO reinstate metadata (let [expected-match [(meta/map->ChunkMatch - {:chunk-idx 0, :block-idxs (doto (RoaringBitmap.) (.add 0)), :col-names #{"_row_id" "xt$id" "name"}})]] + {:chunk-idx 0, :block-idxs (doto (RoaringBitmap.) (.add 0)), :col-names #{"xt$id" "name"}})]] (t/is (= expected-match (meta/matching-chunks metadata-mgr "xt_docs" (expr.meta/->metadata-selector '(= name "Ivan") '{name :utf8} {}))) diff --git a/src/test/clojure/xtdb/stats_test.clj b/src/test/clojure/xtdb/stats_test.clj index 40a417869d..a4f8c18513 100644 --- a/src/test/clojure/xtdb/stats_test.clj +++ b/src/test/clojure/xtdb/stats_test.clj @@ -9,8 +9,7 @@ (t/use-fixtures :each tu/with-allocator) (deftest test-scan - (with-open [node (node/start-node {:xtdb/indexer {:rows-per-chunk 2} - :xtdb/live-chunk {:rows-per-block 2 , :rows-per-chunk 2}})] + (with-open [node (node/start-node {:xtdb/indexer {:rows-per-chunk 2}})] (let [scan-emitter (util/component node :xtdb.operator.scan/scan-emitter)] (xt/submit-tx node [[:put :foo {:xt/id "foo1"}] [:put :bar {:xt/id "bar1"}]]) diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-_row_id.arrow.json deleted file mode 100644 index fc4a264960..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "_row_id", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["0","3"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-api_version.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-api_version.arrow.json deleted file mode 100644 index 48e510a75f..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-api_version.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "api_version", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "api_version", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,2,4], - "DATA" : ["23","23"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-manufacturer.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-manufacturer.arrow.json deleted file mode 100644 index eadc934b76..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-manufacturer.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "manufacturer", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "manufacturer", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,6,12], - "DATA" : ["iobeam","iobeam"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-model.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-model.arrow.json deleted file mode 100644 index 4536a0bf48..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-model.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "model", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "model", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,5,12], - "DATA" : ["pinto","mustang"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-os_name.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-os_name.arrow.json deleted file mode 100644 index f346a1245c..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-os_name.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "os_name", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "os_name", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,5,10], - "DATA" : ["6.0.1","6.0.1"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-xt$id.arrow.json deleted file mode 100644 index b429cda470..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/content-xt$id.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "xt$id", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,22,44], - "DATA" : ["device-info-demo000000","device-info-demo000001"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/metadata.arrow.json deleted file mode 100644 index da6265646e..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_info/metadata.arrow.json +++ /dev/null @@ -1,190 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "utf8", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,6,12], - "children" : [{ - "name" : "$data$", - "count" : 12, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 12, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,7,12,19,24,35,47,54,59,66,71,82,94], - "DATA" : ["_row_id","xt$id","os_name","model","api_version","manufacturer","_row_id","xt$id","os_name","model","api_version","manufacturer"] - },{ - "name" : "root-col?", - "count" : 12, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1,1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 12, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["2","2","2","2","2","2","2","2","2","2","2","2"] - },{ - "name" : "types", - "count" : 12, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 12, - "VALIDITY" : [1,0,0,0,0,0,1,0,0,0,0,0], - "children" : [{ - "name" : "min", - "count" : 12, - "VALIDITY" : [1,0,0,0,0,0,1,0,0,0,0,0], - "DATA" : ["0","0","0","0","0","0","0","0","0","0","0","0"] - },{ - "name" : "max", - "count" : 12, - "VALIDITY" : [1,0,0,0,0,0,1,0,0,0,0,0], - "DATA" : ["3","0","0","0","0","0","3","0","0","0","0","0"] - }] - },{ - "name" : "utf8", - "count" : 12, - "VALIDITY" : [0,1,1,1,1,1,0,1,1,1,1,1], - "children" : [{ - "name" : "min", - "count" : 12, - "VALIDITY" : [0,1,1,1,1,1,0,1,1,1,1,1], - "OFFSET" : [0,0,22,27,34,36,42,42,64,69,76,78,84], - "DATA" : ["","device-info-demo000000","6.0.1","mustang","23","iobeam","","device-info-demo000000","6.0.1","mustang","23","iobeam"] - },{ - "name" : "max", - "count" : 12, - "VALIDITY" : [0,1,1,1,1,1,0,1,1,1,1,1], - "OFFSET" : [0,0,22,27,32,34,40,40,62,67,72,74,80], - "DATA" : ["","device-info-demo000001","6.0.1","pinto","23","iobeam","","device-info-demo000001","6.0.1","pinto","23","iobeam"] - }] - }] - },{ - "name" : "bloom", - "count" : 12, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,52,96,134,194,224,262,314,358,396,456,486,524], - "DATA" : ["3a3000000400000000000100050000000a0000000f000100280000002c0000002e0000003000000000000300fc2af85557c3ade1","3a3000000300000001000000080003000c00000020000000220000002a000000e92f09380a38f066d695f9b3","3a30000003000000060000000700000008000000200000002200000024000000ffad90682123","3a3000000500000004000100050000000b0000000d0000000e000000300000003400000036000000380000003a00000068c95ff09f3d1f57ff16b13b","3a300000020000000000010008000000180000001c0000006d07676cea39","3a3000000300000005000000090000000f000000200000002200000024000000f3d627730da5","3a3000000400000000000100050000000a0000000f000100280000002c0000002e0000003000000000000300fc2af85557c3ade1","3a3000000300000001000000080003000c00000020000000220000002a000000e92f09380a38f066d695f9b3","3a30000003000000060000000700000008000000200000002200000024000000ffad90682123","3a3000000500000004000100050000000b0000000d0000000e000000300000003400000036000000380000003a00000068c95ff09f3d1f57ff16b13b","3a300000020000000000010008000000180000001c0000006d07676cea39","3a3000000300000005000000090000000f000000200000002200000024000000f3d627730da5"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-_row_id.arrow.json deleted file mode 100644 index c985d14f76..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "_row_id", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["1","4"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-battery_level.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-battery_level.arrow.json deleted file mode 100644 index 42fdeaed34..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-battery_level.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "battery_level", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "f64", - "nullable" : false, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "battery_level", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "f64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : [59.0,86.0] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-battery_status.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-battery_status.arrow.json deleted file mode 100644 index 87c1f2602c..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-battery_status.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "battery_status", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "battery_status", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,11,22], - "DATA" : ["discharging","discharging"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-battery_temperature.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-battery_temperature.arrow.json deleted file mode 100644 index d589751192..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-battery_temperature.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "battery_temperature", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "f64", - "nullable" : false, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "battery_temperature", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "f64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : [89.5,93.7] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-bssid.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-bssid.arrow.json deleted file mode 100644 index 65554db8d2..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-bssid.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "bssid", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "bssid", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,17,34], - "DATA" : ["01:02:03:04:05:06","A0:B1:C5:D2:E0:F3"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-cpu_avg_15min.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-cpu_avg_15min.arrow.json deleted file mode 100644 index dc6540e1cb..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-cpu_avg_15min.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "cpu_avg_15min", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "f64", - "nullable" : false, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "cpu_avg_15min", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "f64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : [8.654,8.822] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-cpu_avg_1min.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-cpu_avg_1min.arrow.json deleted file mode 100644 index 335d82187a..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-cpu_avg_1min.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "cpu_avg_1min", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "f64", - "nullable" : false, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "cpu_avg_1min", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "f64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : [24.81,4.93] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-cpu_avg_5min.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-cpu_avg_5min.arrow.json deleted file mode 100644 index 2c9ab15997..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-cpu_avg_5min.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "cpu_avg_5min", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "f64", - "nullable" : false, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "cpu_avg_5min", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "f64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : [10.802,8.106] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-device_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-device_id.arrow.json deleted file mode 100644 index 45660a4399..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-device_id.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "device_id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "device_id", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,22,44], - "DATA" : ["device-info-demo000000","device-info-demo000001"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-mem_free.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-mem_free.arrow.json deleted file mode 100644 index a7a1f7a630..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-mem_free.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "mem_free", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "f64", - "nullable" : false, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "mem_free", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "f64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : [4.10011078E8,7.20742332E8] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-mem_used.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-mem_used.arrow.json deleted file mode 100644 index 1f587f7fe6..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-mem_used.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "mem_used", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "f64", - "nullable" : false, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "mem_used", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "f64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : [5.89988922E8,2.79257668E8] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-rssi.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-rssi.arrow.json deleted file mode 100644 index ef6e181b0c..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-rssi.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "rssi", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "f64", - "nullable" : false, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "rssi", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "f64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : [-50.0,-61.0] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-ssid.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-ssid.arrow.json deleted file mode 100644 index 952446a997..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-ssid.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "ssid", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "ssid", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,8,19], - "DATA" : ["demo-net","stealth-net"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-time.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-time.arrow.json deleted file mode 100644 index 4f3b85ed49..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-time.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "time", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "nullable" : false, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "time", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : [1479211200000000,1479211200000000] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-xt$id.arrow.json deleted file mode 100644 index 91d3823d63..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/content-xt$id.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "xt$id", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,18,36], - "DATA" : ["reading-demo000000","reading-demo000001"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/metadata.arrow.json deleted file mode 100644 index e10955095f..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/device_readings/metadata.arrow.json +++ /dev/null @@ -1,268 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "f64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - }] - },{ - "name" : "utf8", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,15,30], - "children" : [{ - "name" : "$data$", - "count" : 30, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 30, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,7,15,19,32,41,46,59,63,82,96,101,109,113,125,137,144,152,156,169,178,183,196,200,219,233,238,246,250,262,274], - "DATA" : ["_row_id","mem_used","rssi","battery_level","device_id","bssid","cpu_avg_15min","ssid","battery_temperature","battery_status","xt$id","mem_free","time","cpu_avg_1min","cpu_avg_5min","_row_id","mem_used","rssi","battery_level","device_id","bssid","cpu_avg_15min","ssid","battery_temperature","battery_status","xt$id","mem_free","time","cpu_avg_1min","cpu_avg_5min"] - },{ - "name" : "root-col?", - "count" : 30, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 30, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2","2"] - },{ - "name" : "types", - "count" : 30, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 30, - "VALIDITY" : [1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0], - "children" : [{ - "name" : "min", - "count" : 30, - "VALIDITY" : [1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0], - "DATA" : ["1","0","0","0","0","0","0","0","0","0","0","0","0","0","0","1","0","0","0","0","0","0","0","0","0","0","0","0","0","0"] - },{ - "name" : "max", - "count" : 30, - "VALIDITY" : [1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0], - "DATA" : ["4","0","0","0","0","0","0","0","0","0","0","0","0","0","0","4","0","0","0","0","0","0","0","0","0","0","0","0","0","0"] - }] - },{ - "name" : "f64", - "count" : 30, - "VALIDITY" : [0,1,1,1,0,0,1,0,1,0,0,1,0,1,1,0,1,1,1,0,0,1,0,1,0,0,1,0,1,1], - "children" : [{ - "name" : "min", - "count" : 30, - "VALIDITY" : [0,1,1,1,0,0,1,0,1,0,0,1,0,1,1,0,1,1,1,0,0,1,0,1,0,0,1,0,1,1], - "DATA" : [0.0,2.79257668E8,-61.0,59.0,0.0,0.0,8.654,0.0,89.5,0.0,0.0,4.10011078E8,0.0,4.93,8.106,0.0,2.79257668E8,-61.0,59.0,0.0,0.0,8.654,0.0,89.5,0.0,0.0,4.10011078E8,0.0,4.93,8.106] - },{ - "name" : "max", - "count" : 30, - "VALIDITY" : [0,1,1,1,0,0,1,0,1,0,0,1,0,1,1,0,1,1,1,0,0,1,0,1,0,0,1,0,1,1], - "DATA" : [0.0,5.89988922E8,-50.0,86.0,0.0,0.0,8.822,0.0,93.7,0.0,0.0,7.20742332E8,0.0,24.81,10.802,0.0,5.89988922E8,-50.0,86.0,0.0,0.0,8.822,0.0,93.7,0.0,0.0,7.20742332E8,0.0,24.81,10.802] - }] - },{ - "name" : "utf8", - "count" : 30, - "VALIDITY" : [0,0,0,0,1,1,0,1,0,1,1,0,0,0,0,0,0,0,0,1,1,0,1,0,1,1,0,0,0,0], - "children" : [{ - "name" : "min", - "count" : 30, - "VALIDITY" : [0,0,0,0,1,1,0,1,0,1,1,0,0,0,0,0,0,0,0,1,1,0,1,0,1,1,0,0,0,0], - "OFFSET" : [0,0,0,0,0,22,39,39,47,47,58,76,76,76,76,76,76,76,76,76,98,115,115,123,123,134,152,152,152,152,152], - "DATA" : ["","","","","device-info-demo000000","01:02:03:04:05:06","","demo-net","","discharging","reading-demo000000","","","","","","","","","device-info-demo000000","01:02:03:04:05:06","","demo-net","","discharging","reading-demo000000","","","",""] - },{ - "name" : "max", - "count" : 30, - "VALIDITY" : [0,0,0,0,1,1,0,1,0,1,1,0,0,0,0,0,0,0,0,1,1,0,1,0,1,1,0,0,0,0], - "OFFSET" : [0,0,0,0,0,22,39,39,50,50,61,79,79,79,79,79,79,79,79,79,101,118,118,129,129,140,158,158,158,158,158], - "DATA" : ["","","","","device-info-demo000001","A0:B1:C5:D2:E0:F3","","stealth-net","","discharging","reading-demo000001","","","","","","","","","device-info-demo000001","A0:B1:C5:D2:E0:F3","","stealth-net","","discharging","reading-demo000001","","","",""] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 30, - "VALIDITY" : [0,0,0,0,0,0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0,0], - "children" : [{ - "name" : "min", - "count" : 30, - "VALIDITY" : [0,0,0,0,0,0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0,0], - "DATA" : [0,0,0,0,0,0,0,0,0,0,0,0,1479211200000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1479211200000000,0,0] - },{ - "name" : "max", - "count" : 30, - "VALIDITY" : [0,0,0,0,0,0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0,0], - "DATA" : [0,0,0,0,0,0,0,0,0,0,0,0,1479211200000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1479211200000000,0,0] - }] - }] - },{ - "name" : "bloom", - "count" : 30, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,52,120,180,248,292,360,420,472,540,578,630,698,736,796,864,916,984,1044,1112,1156,1224,1284,1336,1404,1442,1494,1562,1600,1660,1728], - "DATA" : ["3a3000000400000000000200080000000d0000000e000000280000002e0000003000000032000000010004002bee16778e13c989","3a3000000600000000000000010000000300000004000000050000000a000000380000003a0000003c0000003e000000400000004200000022a54295d8356ed64056b1fd","3a3000000500000000000000090001000a0000000e0000000f000000300000003200000036000000380000003a000000f456000006c90c9200807a6b","3a30000006000000010000000500000006000000070000000a0000000d000000380000003a0000003c0000003e00000040000000420000004cf00080986046c28c040080","3a3000000300000001000000080003000c00000020000000220000002a000000e92f09380a38f066d695f9b3","3a30000006000000000000000100000002000000080000000b0000000e000000380000003a0000003c0000003e000000400000004200000058377ea7a127e8b65a9832c7","3a30000005000000010000000300000006000100080000000b000000300000003200000034000000380000003a000000516f96eadb659594fce7633b","3a300000040000000300010004000100080000000e000000280000002c00000030000000320000000e5174fa3f69daa3490b443a","3a3000000600000006000000090000000b0000000c0000000d0000000e000000380000003a0000003c0000003e000000400000004200000000604a5e01a0945c4a3e93dc","3a3000000300000003000000070000000d0000002000000022000000240000001cbc2a702396","3a300000040000000000010001000100060000000c000000280000002c0000003000000032000000a3b2a4b23328c39d747f444c","3a300000060000000500000008000000090000000a0000000c0000000d000000380000003a0000003c0000003e0000004000000042000000d37a4570f959ff3ab9051f39","3a30000003000000010000000a0000000d000000200000002200000024000000490c55f1cffe","3a300000050000000000000006000100090000000d0000000f000000300000003200000036000000380000003a000000eba9e9a6dfdb7b2bd30d32e9","3a300000060000000300000004000000080000000a0000000e0000000f000000380000003a0000003c0000003e00000040000000420000002b52506c82bbc6f6d9248bb1","3a3000000400000000000200080000000d0000000e000000280000002e0000003000000032000000010004002bee16778e13c989","3a3000000600000000000000010000000300000004000000050000000a000000380000003a0000003c0000003e000000400000004200000022a54295d8356ed64056b1fd","3a3000000500000000000000090001000a0000000e0000000f000000300000003200000036000000380000003a000000f456000006c90c9200807a6b","3a30000006000000010000000500000006000000070000000a0000000d000000380000003a0000003c0000003e00000040000000420000004cf00080986046c28c040080","3a3000000300000001000000080003000c00000020000000220000002a000000e92f09380a38f066d695f9b3","3a30000006000000000000000100000002000000080000000b0000000e000000380000003a0000003c0000003e000000400000004200000058377ea7a127e8b65a9832c7","3a30000005000000010000000300000006000100080000000b000000300000003200000034000000380000003a000000516f96eadb659594fce7633b","3a300000040000000300010004000100080000000e000000280000002c00000030000000320000000e5174fa3f69daa3490b443a","3a3000000600000006000000090000000b0000000c0000000d0000000e000000380000003a0000003c0000003e000000400000004200000000604a5e01a0945c4a3e93dc","3a3000000300000003000000070000000d0000002000000022000000240000001cbc2a702396","3a300000040000000000010001000100060000000c000000280000002c0000003000000032000000a3b2a4b23328c39d747f444c","3a300000060000000500000008000000090000000a0000000c0000000d000000380000003a0000003c0000003e0000004000000042000000d37a4570f959ff3ab9051f39","3a30000003000000010000000a0000000d000000200000002200000024000000490c55f1cffe","3a300000050000000000000006000100090000000d0000000f000000300000003200000036000000380000003a000000eba9e9a6dfdb7b2bd30d32e9","3a300000060000000300000004000000080000000a0000000e0000000f000000380000003a0000003c0000003e00000040000000420000002b52506c82bbc6f6d9248bb1"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-_row_id.arrow.json deleted file mode 100644 index 1ec4f7c50a..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "_row_id", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["2","5"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$committed?.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$committed?.arrow.json deleted file mode 100644 index 10950ea99b..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$committed?.arrow.json +++ /dev/null @@ -1,36 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$committed?", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "bool", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "xt$committed?", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "bool", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : [1,1] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$error.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$error.arrow.json deleted file mode 100644 index beca1e675a..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$error.arrow.json +++ /dev/null @@ -1,54 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$error", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "clj-form", - "nullable" : false, - "type" : { - "name" : "ClojureFormType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "xt/clj-form", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "null", - "nullable" : true, - "type" : { - "name" : "null" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "xt$error", - "count" : 2, - "TYPE_ID" : [1,1], - "OFFSET" : [0,1], - "children" : [{ - "name" : "clj-form", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "null", - "count" : 2 - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$id.arrow.json deleted file mode 100644 index 13d10f7f2a..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$id.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "xt$id", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "i64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["0","8165"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$tx_time.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$tx_time.arrow.json deleted file mode 100644 index 0beb8fd6ee..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/content-xt$tx_time.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$tx_time", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "nullable" : false, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "xt$tx_time", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : [1577836800000000,1577923200000000] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/metadata.arrow.json deleted file mode 100644 index 131a63ca42..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-chunk-as-arrow-ipc-file-format/chunk-00/xt$txs/metadata.arrow.json +++ /dev/null @@ -1,216 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - },{ - "name" : "bool", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "null", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,5,10], - "children" : [{ - "name" : "$data$", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,7,12,22,35,43,50,55,65,78,86], - "DATA" : ["_row_id","xt$id","xt$tx_time","xt$committed?","xt$error","_row_id","xt$id","xt$tx_time","xt$committed?","xt$error"] - },{ - "name" : "root-col?", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "DATA" : ["2","2","2","2","2","2","2","2","2","2"] - },{ - "name" : "types", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["2","0","0","0","0","2","0","0","0","0"] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["5","8165","0","0","0","5","8165","0","0","0"] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "DATA" : [0,0,1577836800000000,0,0,0,0,1577836800000000,0,0] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "DATA" : [0,0,1577923200000000,0,0,0,0,1577923200000000,0,0] - }] - },{ - "name" : "bool", - "count" : 10, - "VALIDITY" : [0,0,0,1,0,0,0,0,1,0], - "DATA" : [0,0,0,1,0,0,0,0,1,0] - },{ - "name" : "null", - "count" : 10, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,1], - "DATA" : [0,0,0,0,1,0,0,0,0,1] - }] - },{ - "name" : "bloom", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,60,120,188,210,232,292,352,420,442,464], - "DATA" : ["3a3000000500000000000100040000000a0000000e0000000f000000300000003400000036000000380000003a00000002000500d2946a4adbd6706b","3a300000050000000000010005000000070000000a0000000b000000300000003400000036000000380000003a0000000000e51ffc2a3fbaf85512ed","3a300000060000000200000004000000090000000b0000000c0000000f000000380000003a0000003c0000003e00000040000000420000000af41c3b4c0193972a6e08db","3a300000010000000000020010000000130026003900","3a3000000100000000000200100000003e007c00ba00","3a3000000500000000000100040000000a0000000e0000000f000000300000003400000036000000380000003a00000002000500d2946a4adbd6706b","3a300000050000000000010005000000070000000a0000000b000000300000003400000036000000380000003a0000000000e51ffc2a3fbaf85512ed","3a300000060000000200000004000000090000000b0000000c0000000f000000380000003a0000003c0000003e00000040000000420000000af41c3b4c0193972a6e08db","3a300000010000000000020010000000130026003900","3a3000000100000000000200100000003e007c00ba00"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-_row_id.arrow.json deleted file mode 100644 index 1060f12edf..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "_row_id", - "count" : 3, - "VALIDITY" : [1,1,1], - "DATA" : ["7","8","9"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-bar.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-bar.arrow.json deleted file mode 100644 index 598d577064..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-bar.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "bar", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "bar", - "count" : 3, - "TYPE_ID" : [0,0,0], - "OFFSET" : [0,1,2], - "children" : [{ - "name" : "i64", - "count" : 3, - "VALIDITY" : [1,1,1], - "DATA" : ["1","2","2"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-toto.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-toto.arrow.json deleted file mode 100644 index d6503b0967..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-toto.arrow.json +++ /dev/null @@ -1,54 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "toto", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "absent", - "nullable" : false, - "type" : { - "name" : "AbsentType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "absent", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - }] - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "toto", - "count" : 3, - "TYPE_ID" : [0,0,1], - "OFFSET" : [0,1,0], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,4,8], - "DATA" : ["toto","toto"] - },{ - "name" : "absent", - "count" : 1 - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-xt$id.arrow.json deleted file mode 100644 index afbe018ef8..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/content-xt$id.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "xt$id", - "count" : 3, - "TYPE_ID" : [0,0,0], - "OFFSET" : [0,1,2], - "children" : [{ - "name" : "i64", - "count" : 3, - "VALIDITY" : [1,1,1], - "DATA" : ["1","1","2"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/metadata.arrow.json deleted file mode 100644 index 97b8a66dfb..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/foo/metadata.arrow.json +++ /dev/null @@ -1,202 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "utf8", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - },{ - "name" : "absent", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,4,8], - "children" : [{ - "name" : "$data$", - "count" : 8, - "VALIDITY" : [1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 8, - "VALIDITY" : [1,1,1,1,1,1,1,1], - "OFFSET" : [0,7,12,15,19,26,31,34,38], - "DATA" : ["_row_id","xt$id","bar","toto","_row_id","xt$id","bar","toto"] - },{ - "name" : "root-col?", - "count" : 8, - "VALIDITY" : [1,1,1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 8, - "VALIDITY" : [1,1,1,1,1,1,1,1], - "DATA" : ["3","3","3","3","3","3","3","3"] - },{ - "name" : "types", - "count" : 8, - "VALIDITY" : [1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 8, - "VALIDITY" : [1,1,1,0,1,1,1,0], - "children" : [{ - "name" : "min", - "count" : 8, - "VALIDITY" : [1,1,1,0,1,1,1,0], - "DATA" : ["7","1","1","0","7","1","1","0"] - },{ - "name" : "max", - "count" : 8, - "VALIDITY" : [1,1,1,0,1,1,1,0], - "DATA" : ["9","2","2","0","9","2","2","0"] - }] - },{ - "name" : "utf8", - "count" : 8, - "VALIDITY" : [0,0,0,1,0,0,0,1], - "children" : [{ - "name" : "min", - "count" : 8, - "VALIDITY" : [0,0,0,1,0,0,0,1], - "OFFSET" : [0,0,0,0,4,4,4,4,8], - "DATA" : ["","","","toto","","","","toto"] - },{ - "name" : "max", - "count" : 8, - "VALIDITY" : [0,0,0,1,0,0,0,1], - "OFFSET" : [0,0,0,0,4,4,4,4,8], - "DATA" : ["","","","toto","","","","toto"] - }] - },{ - "name" : "absent", - "count" : 8, - "VALIDITY" : [0,0,0,1,0,0,0,1], - "DATA" : [0,0,0,1,0,0,0,1] - }] - },{ - "name" : "bloom", - "count" : 8, - "VALIDITY" : [1,1,1,1,1,1,1,1], - "OFFSET" : [0,66,118,170,222,288,340,392,444], - "DATA" : ["3a300000050000000000020005000100070000000a0000000b00010030000000360000003a0000003c0000003e000000070008000900bc2bce90b4c96f579521dee4","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a3000000400000000000200040000000a0000000f000000280000002e00000030000000320000001f003e005d00746f44e95cac","3a300000050000000000020005000100070000000a0000000b00010030000000360000003a0000003c0000003e000000070008000900bc2bce90b4c96f579521dee4","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a3000000400000000000200040000000a0000000f000000280000002e00000030000000320000001f003e005d00746f44e95cac"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/content-_row_id.arrow.json deleted file mode 100644 index 72b1979e51..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "_row_id", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["0"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/content-a.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/content-a.arrow.json deleted file mode 100644 index 9c527af2a1..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/content-a.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "a", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "a", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "i64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["1"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/content-xt$id.arrow.json deleted file mode 100644 index 8bd35b50d7..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/content-xt$id.arrow.json +++ /dev/null @@ -1,43 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "uuid", - "nullable" : false, - "type" : { - "name" : "UuidType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "uuid", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$id", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "uuid", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["cb8815ee85f74c61a8032ea1c949cf8d"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/metadata.arrow.json deleted file mode 100644 index dabbb0fb00..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/hello/metadata.arrow.json +++ /dev/null @@ -1,202 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "uuid", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "UuidType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "uuid", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "UuidType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "uuid", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - }] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,3,6], - "children" : [{ - "name" : "$data$", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "OFFSET" : [0,7,8,13,20,21,26], - "DATA" : ["_row_id","a","xt$id","_row_id","a","xt$id"] - },{ - "name" : "root-col?", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : ["1","1","1","1","1","1"] - },{ - "name" : "types", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 6, - "VALIDITY" : [1,1,0,1,1,0], - "children" : [{ - "name" : "min", - "count" : 6, - "VALIDITY" : [1,1,0,1,1,0], - "DATA" : ["0","1","0","0","1","0"] - },{ - "name" : "max", - "count" : 6, - "VALIDITY" : [1,1,0,1,1,0], - "DATA" : ["0","1","0","0","1","0"] - }] - },{ - "name" : "uuid", - "count" : 6, - "VALIDITY" : [0,0,1,0,0,1], - "children" : [{ - "name" : "min", - "count" : 6, - "VALIDITY" : [0,0,1,0,0,1], - "DATA" : ["00000000000000000000000000000000","00000000000000000000000000000000","cb8815ee85f74c61a8032ea1c949cf8d","00000000000000000000000000000000","00000000000000000000000000000000","cb8815ee85f74c61a8032ea1c949cf8d"] - },{ - "name" : "max", - "count" : 6, - "VALIDITY" : [0,0,1,0,0,1], - "DATA" : ["00000000000000000000000000000000","00000000000000000000000000000000","cb8815ee85f74c61a8032ea1c949cf8d","00000000000000000000000000000000","00000000000000000000000000000000","cb8815ee85f74c61a8032ea1c949cf8d"] - }] - }] - },{ - "name" : "bloom", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "OFFSET" : [0,38,68,106,144,174,212], - "DATA" : ["3a3000000300000000000000050000000a0000002000000022000000240000000000fc2af855","3a300000020000000000010008000000180000001c00000001002bee1677","3a30000003000000000000000100000009000000200000002200000024000000a7b0897b1816","3a3000000300000000000000050000000a0000002000000022000000240000000000fc2af855","3a300000020000000000010008000000180000001c00000001002bee1677","3a30000003000000000000000100000009000000200000002200000024000000a7b0897b1816"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-_row_id.arrow.json deleted file mode 100644 index 2dca7cbc33..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "_row_id", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["1","3"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-b.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-b.arrow.json deleted file mode 100644 index 0211235cff..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-b.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "b", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "b", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "i64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["2","3"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-xt$id.arrow.json deleted file mode 100644 index d348267cc2..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/content-xt$id.arrow.json +++ /dev/null @@ -1,43 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "uuid", - "nullable" : false, - "type" : { - "name" : "UuidType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "uuid", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "xt$id", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "uuid", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["424f5622c8264deda5dbe2144d665c38","424f5622c8264deda5dbe2144d665c38"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/metadata.arrow.json deleted file mode 100644 index 19f28b45fc..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/world/metadata.arrow.json +++ /dev/null @@ -1,202 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "uuid", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "UuidType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "uuid", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "UuidType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "uuid", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - }] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,3,6], - "children" : [{ - "name" : "$data$", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "OFFSET" : [0,7,8,13,20,21,26], - "DATA" : ["_row_id","b","xt$id","_row_id","b","xt$id"] - },{ - "name" : "root-col?", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : ["2","2","2","2","2","2"] - },{ - "name" : "types", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 6, - "VALIDITY" : [1,1,0,1,1,0], - "children" : [{ - "name" : "min", - "count" : 6, - "VALIDITY" : [1,1,0,1,1,0], - "DATA" : ["1","2","0","1","2","0"] - },{ - "name" : "max", - "count" : 6, - "VALIDITY" : [1,1,0,1,1,0], - "DATA" : ["3","3","0","3","3","0"] - }] - },{ - "name" : "uuid", - "count" : 6, - "VALIDITY" : [0,0,1,0,0,1], - "children" : [{ - "name" : "min", - "count" : 6, - "VALIDITY" : [0,0,1,0,0,1], - "DATA" : ["00000000000000000000000000000000","00000000000000000000000000000000","424f5622c8264deda5dbe2144d665c38","00000000000000000000000000000000","00000000000000000000000000000000","424f5622c8264deda5dbe2144d665c38"] - },{ - "name" : "max", - "count" : 6, - "VALIDITY" : [0,0,1,0,0,1], - "DATA" : ["00000000000000000000000000000000","00000000000000000000000000000000","424f5622c8264deda5dbe2144d665c38","00000000000000000000000000000000","00000000000000000000000000000000","424f5622c8264deda5dbe2144d665c38"] - }] - }] - },{ - "name" : "bloom", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "OFFSET" : [0,44,96,126,170,222,252], - "DATA" : ["3a3000000300000000000200080000000f000100200000002600000028000000010003002bee167757c3ade1","3a3000000400000000000100040000000a0000000f000100280000002c0000002e0000003000000002000300d2946a4a57c3ade1","3a30000002000000040001000c000000180000001c000000de6bdafe5cb5","3a3000000300000000000200080000000f000100200000002600000028000000010003002bee167757c3ade1","3a3000000400000000000100040000000a0000000f000100280000002c0000002e0000003000000002000300d2946a4a57c3ade1","3a30000002000000040001000c000000180000001c000000de6bdafe5cb5"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-_row_id.arrow.json deleted file mode 100644 index 58a1920693..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 6, - "columns" : [{ - "name" : "_row_id", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : ["2","4","6","10","11","12"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$committed?.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$committed?.arrow.json deleted file mode 100644 index b5390efee1..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$committed?.arrow.json +++ /dev/null @@ -1,36 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$committed?", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "bool", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 6, - "columns" : [{ - "name" : "xt$committed?", - "count" : 6, - "TYPE_ID" : [0,0,0,0,0,0], - "OFFSET" : [0,1,2,3,4,5], - "children" : [{ - "name" : "bool", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,0] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$error.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$error.arrow.json deleted file mode 100644 index 66b874aacd..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$error.arrow.json +++ /dev/null @@ -1,54 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$error", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "clj-form", - "nullable" : false, - "type" : { - "name" : "ClojureFormType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "xt/clj-form", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "null", - "nullable" : true, - "type" : { - "name" : "null" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 6, - "columns" : [{ - "name" : "xt$error", - "count" : 6, - "TYPE_ID" : [1,1,1,1,1,0], - "OFFSET" : [0,1,2,3,4,0], - "children" : [{ - "name" : "clj-form", - "count" : 1, - "VALIDITY" : [1], - "OFFSET" : [0,284], - "DATA" : ["#xt/runtime-err {:xtdb.error/error-type :runtime-error, :xtdb.error/error-key :xtdb.indexer/invalid-valid-times, :xtdb.error/message \"Runtime error: ':xtdb.indexer/invalid-valid-times'\", :valid-from #time/instant \"2020-01-01T00:00:00Z\", :valid-to #time/instant \"2019-01-01T00:00:00Z\"}"] - },{ - "name" : "null", - "count" : 5 - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$id.arrow.json deleted file mode 100644 index 280e054de4..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$id.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 6, - "columns" : [{ - "name" : "xt$id", - "count" : 6, - "TYPE_ID" : [0,0,0,0,0,0], - "OFFSET" : [0,1,2,3,4,5], - "children" : [{ - "name" : "i64", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : ["0","4461","8538","11831","15076","18145"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$tx_time.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$tx_time.arrow.json deleted file mode 100644 index ae7c86b6fa..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/content-xt$tx_time.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$tx_time", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "nullable" : false, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 6, - "columns" : [{ - "name" : "xt$tx_time", - "count" : 6, - "TYPE_ID" : [0,0,0,0,0,0], - "OFFSET" : [0,1,2,3,4,5], - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : [1577836800000000,1577923200000000,1578009600000000,1578096000000000,1578182400000000,1578268800000000] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/metadata.arrow.json deleted file mode 100644 index e44f54b5b5..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-build-live-index/chunk-00/xt$txs/metadata.arrow.json +++ /dev/null @@ -1,228 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - },{ - "name" : "bool", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "clj-form", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "null", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,5,10], - "children" : [{ - "name" : "$data$", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,7,12,22,35,43,50,55,65,78,86], - "DATA" : ["_row_id","xt$id","xt$tx_time","xt$committed?","xt$error","_row_id","xt$id","xt$tx_time","xt$committed?","xt$error"] - },{ - "name" : "root-col?", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "DATA" : ["6","6","6","6","6","6","6","6","6","6"] - },{ - "name" : "types", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["2","0","0","0","0","2","0","0","0","0"] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["12","18145","0","0","0","12","18145","0","0","0"] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "DATA" : [0,0,1577836800000000,0,0,0,0,1577836800000000,0,0] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "DATA" : [0,0,1578268800000000,0,0,0,0,1578268800000000,0,0] - }] - },{ - "name" : "bool", - "count" : 10, - "VALIDITY" : [0,0,0,1,0,0,0,0,1,0], - "DATA" : [0,0,0,1,0,0,0,0,1,0] - },{ - "name" : "clj-form", - "count" : 10, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,1], - "DATA" : [0,0,0,0,1,0,0,0,0,1] - },{ - "name" : "null", - "count" : 10, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,1], - "DATA" : [0,0,0,0,1,0,0,0,0,1] - }] - },{ - "name" : "bloom", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,124,232,364,392,444,568,676,808,836,888], - "DATA" : ["3a3000000a00000000000700040000000500000008000000090000000a0000000b0000000c0001000d0000000e00000058000000680000006a0000006c0000006e000000700000007200000074000000780000007a0000000200040006000a000b000c00e014b429d2947789102921c36a4ae8128d1496e18e13c989","3a300000080000000000070001000100020000000500000008000100090001000a0000000c00000048000000580000005c0000005e0000006000000064000000680000006a00000000006d115a21372ee43ae14667fa29fd97c606da8f8afc2a05a2b0fd5002fe4df8559a9e","3a3000000b0000000000000002000200040001000500010006000100090000000a0000000b0000000c0001000e0001000f0000006000000062000000680000006c000000700000007400000076000000780000007a0000007e00000082000000c402595b6f650af4582a1c3b57f945fbe42583504c016dbb93972a6e309bfa4e6e9008db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000009b003601d1018a6364dfb0e7","3a3000000a00000000000700040000000500000008000000090000000a0000000b0000000c0001000d0000000e00000058000000680000006a0000006c0000006e000000700000007200000074000000780000007a0000000200040006000a000b000c00e014b429d2947789102921c36a4ae8128d1496e18e13c989","3a300000080000000000070001000100020000000500000008000100090001000a0000000c00000048000000580000005c0000005e0000006000000064000000680000006a00000000006d115a21372ee43ae14667fa29fd97c606da8f8afc2a05a2b0fd5002fe4df8559a9e","3a3000000b0000000000000002000200040001000500010006000100090000000a0000000b0000000c0001000e0001000f0000006000000062000000680000006c000000700000007400000076000000780000007a0000007e00000082000000c402595b6f650af4582a1c3b57f945fbe42583504c016dbb93972a6e309bfa4e6e9008db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000009b003601d1018a6364dfb0e7"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-_row_id.arrow.json deleted file mode 100644 index 4455de8e38..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "_row_id", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["6"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$committed?.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$committed?.arrow.json deleted file mode 100644 index 7c4d19982c..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$committed?.arrow.json +++ /dev/null @@ -1,36 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$committed?", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "bool", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$committed?", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "bool", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [1] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$error.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$error.arrow.json deleted file mode 100644 index 668b9d40d8..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$error.arrow.json +++ /dev/null @@ -1,54 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$error", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "clj-form", - "nullable" : false, - "type" : { - "name" : "ClojureFormType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "xt/clj-form", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "null", - "nullable" : true, - "type" : { - "name" : "null" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$error", - "count" : 1, - "TYPE_ID" : [1], - "OFFSET" : [0], - "children" : [{ - "name" : "clj-form", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "null", - "count" : 1 - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$id.arrow.json deleted file mode 100644 index bd21b780af..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$id.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$id", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "i64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["0"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$tx_time.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$tx_time.arrow.json deleted file mode 100644 index 67b1578fec..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/content-xt$tx_time.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$tx_time", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "nullable" : false, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$tx_time", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [1577836800000000] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/metadata.arrow.json deleted file mode 100644 index 18ec5ed4c8..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt$txs/metadata.arrow.json +++ /dev/null @@ -1,216 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - },{ - "name" : "bool", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "null", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,5,10], - "children" : [{ - "name" : "$data$", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,7,12,22,35,43,50,55,65,78,86], - "DATA" : ["_row_id","xt$id","xt$tx_time","xt$committed?","xt$error","_row_id","xt$id","xt$tx_time","xt$committed?","xt$error"] - },{ - "name" : "root-col?", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "DATA" : ["1","1","1","1","1","1","1","1","1","1"] - },{ - "name" : "types", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["6","0","0","0","0","6","0","0","0","0"] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["6","0","0","0","0","6","0","0","0","0"] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "DATA" : [0,0,1577836800000000,0,0,0,0,1577836800000000,0,0] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "DATA" : [0,0,1577836800000000,0,0,0,0,1577836800000000,0,0] - }] - },{ - "name" : "bool", - "count" : 10, - "VALIDITY" : [0,0,0,1,0,0,0,0,1,0], - "DATA" : [0,0,0,1,0,0,0,0,1,0] - },{ - "name" : "null", - "count" : 10, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,1], - "DATA" : [0,0,0,0,1,0,0,0,0,1] - }] - },{ - "name" : "bloom", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,38,76,114,136,158,196,234,272,294,316], - "DATA" : ["3a3000000300000000000000050000000b00000020000000220000002400000006007789e812","3a3000000300000000000000050000000a0000002000000022000000240000000000fc2af855","3a30000003000000090000000c0000000f0000002000000022000000240000004c012a6e08db","3a300000010000000000020010000000130026003900","3a3000000100000000000200100000001f003e005d00","3a3000000300000000000000050000000b00000020000000220000002400000006007789e812","3a3000000300000000000000050000000a0000002000000022000000240000000000fc2af855","3a30000003000000090000000c0000000f0000002000000022000000240000004c012a6e08db","3a300000010000000000020010000000130026003900","3a3000000100000000000200100000001f003e005d00"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-_row_id.arrow.json deleted file mode 100644 index cdc579dbc4..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 6, - "columns" : [{ - "name" : "_row_id", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : ["0","1","2","3","4","5"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-list.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-list.arrow.json deleted file mode 100644 index 46e2046e8a..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-list.arrow.json +++ /dev/null @@ -1,121 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "list", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "list", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "f64", - "nullable" : false, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - },{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "timestamp-tz-micro-utc", - "nullable" : false, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - },{ - "name" : "bool", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - }] - },{ - "name" : "absent", - "nullable" : false, - "type" : { - "name" : "AbsentType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "absent", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - }] - }] - }, - "batches" : [{ - "count" : 6, - "columns" : [{ - "name" : "list", - "count" : 6, - "TYPE_ID" : [0,1,0,1,1,1], - "OFFSET" : [0,0,1,1,2,3], - "children" : [{ - "name" : "list", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,2,4], - "children" : [{ - "name" : "$data$", - "count" : 4, - "TYPE_ID" : [0,1,2,3], - "OFFSET" : [0,0,0,0], - "children" : [{ - "name" : "f64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [12.0] - },{ - "name" : "utf8", - "count" : 1, - "VALIDITY" : [1], - "OFFSET" : [0,3], - "DATA" : ["foo"] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [1577836800000000] - },{ - "name" : "bool", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [0] - }] - }] - },{ - "name" : "absent", - "count" : 4 - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-struct.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-struct.arrow.json deleted file mode 100644 index 83662a607f..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-struct.arrow.json +++ /dev/null @@ -1,184 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "struct", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "absent", - "nullable" : false, - "type" : { - "name" : "AbsentType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "absent", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "struct", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "a", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "bool", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - },{ - "name" : "b", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "absent", - "nullable" : false, - "type" : { - "name" : "AbsentType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "absent", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - }] - },{ - "name" : "c", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "absent", - "nullable" : false, - "type" : { - "name" : "AbsentType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "absent", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }] - }] - }, - "batches" : [{ - "count" : 6, - "columns" : [{ - "name" : "struct", - "count" : 6, - "TYPE_ID" : [0,0,0,1,0,1], - "OFFSET" : [0,1,2,0,3,1], - "children" : [{ - "name" : "absent", - "count" : 4 - },{ - "name" : "struct", - "count" : 2, - "VALIDITY" : [1,1], - "children" : [{ - "name" : "a", - "count" : 2, - "TYPE_ID" : [0,1], - "OFFSET" : [0,0], - "children" : [{ - "name" : "i64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["1"] - },{ - "name" : "bool", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [1] - }] - },{ - "name" : "b", - "count" : 2, - "TYPE_ID" : [0,1], - "OFFSET" : [0,0], - "children" : [{ - "name" : "utf8", - "count" : 1, - "VALIDITY" : [1], - "OFFSET" : [0,1], - "DATA" : ["b"] - },{ - "name" : "absent", - "count" : 1 - }] - },{ - "name" : "c", - "count" : 2, - "TYPE_ID" : [0,1], - "OFFSET" : [0,0], - "children" : [{ - "name" : "absent", - "count" : 1 - },{ - "name" : "utf8", - "count" : 1, - "VALIDITY" : [1], - "OFFSET" : [0,1], - "DATA" : ["c"] - }] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-xt$id.arrow.json deleted file mode 100644 index 80ab5a7cbb..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/content-xt$id.arrow.json +++ /dev/null @@ -1,71 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "keyword", - "nullable" : false, - "type" : { - "name" : "KeywordType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "xt/clj-keyword", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - }] - }] - }, - "batches" : [{ - "count" : 6, - "columns" : [{ - "name" : "xt$id", - "count" : 6, - "TYPE_ID" : [0,1,0,2,1,2], - "OFFSET" : [0,0,1,0,1,1], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,3,6], - "DATA" : ["foo","bar"] - },{ - "name" : "i64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["24","52"] - },{ - "name" : "keyword", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,3,7], - "DATA" : ["baz","quux"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/metadata.arrow.json deleted file mode 100644 index c7b1abe443..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-handle-dynamic-cols-in-same-block/chunk-00/xt_docs/metadata.arrow.json +++ /dev/null @@ -1,385 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "absent", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "struct-2", - "nullable" : true, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "bool", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "utf8", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - },{ - "name" : "keyword", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "KeywordType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "xt/clj-keyword", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "KeywordType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "xt/clj-keyword", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - }] - },{ - "name" : "list", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "f64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,8,16], - "children" : [{ - "name" : "$data$", - "count" : 16, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 16, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,7,8,9,10,16,21,27,31,38,39,40,41,47,52,58,62], - "DATA" : ["_row_id","a","b","c","struct","xt$id","$data$","list","_row_id","a","b","c","struct","xt$id","$data$","list"] - },{ - "name" : "root-col?", - "count" : 16, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : [1,0,0,0,1,1,0,1,1,0,0,0,1,1,0,1] - },{ - "name" : "count", - "count" : 16, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["6","2","2","2","6","6","4","6","6","2","2","2","6","6","4","6"] - },{ - "name" : "types", - "count" : 16, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 16, - "VALIDITY" : [1,1,0,0,0,1,0,0,1,1,0,0,0,1,0,0], - "children" : [{ - "name" : "min", - "count" : 16, - "VALIDITY" : [1,1,0,0,0,1,0,0,1,1,0,0,0,1,0,0], - "DATA" : ["0","1","0","0","0","24","0","0","0","1","0","0","0","24","0","0"] - },{ - "name" : "max", - "count" : 16, - "VALIDITY" : [1,1,0,0,0,1,0,0,1,1,0,0,0,1,0,0], - "DATA" : ["5","1","0","0","0","52","0","0","5","1","0","0","0","52","0","0"] - }] - },{ - "name" : "absent", - "count" : 16, - "VALIDITY" : [0,0,1,1,1,0,0,1,0,0,1,1,1,0,0,1], - "DATA" : [0,0,1,1,1,0,0,1,0,0,1,1,1,0,0,1] - },{ - "name" : "struct-2", - "count" : 16, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,0,0,0,1,0,0,0], - "OFFSET" : [0,0,0,0,0,3,3,3,3,3,3,3,3,6,6,6,6], - "children" : [{ - "name" : "$data$", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : [1,2,3,9,10,11] - }] - },{ - "name" : "bool", - "count" : 16, - "VALIDITY" : [0,1,0,0,0,0,1,0,0,1,0,0,0,0,1,0], - "DATA" : [0,1,0,0,0,0,1,0,0,1,0,0,0,0,1,0] - },{ - "name" : "utf8", - "count" : 16, - "VALIDITY" : [0,0,1,1,0,1,1,0,0,0,1,1,0,1,1,0], - "children" : [{ - "name" : "min", - "count" : 16, - "VALIDITY" : [0,0,1,1,0,1,1,0,0,0,1,1,0,1,1,0], - "OFFSET" : [0,0,0,1,2,2,5,8,8,8,8,9,10,10,13,16,16], - "DATA" : ["","","b","c","","bar","foo","","","","b","c","","bar","foo",""] - },{ - "name" : "max", - "count" : 16, - "VALIDITY" : [0,0,1,1,0,1,1,0,0,0,1,1,0,1,1,0], - "OFFSET" : [0,0,0,1,2,2,5,8,8,8,8,9,10,10,13,16,16], - "DATA" : ["","","b","c","","foo","foo","","","","b","c","","foo","foo",""] - }] - },{ - "name" : "keyword", - "count" : 16, - "VALIDITY" : [0,0,0,0,0,1,0,0,0,0,0,0,0,1,0,0], - "children" : [{ - "name" : "min", - "count" : 16, - "VALIDITY" : [0,0,0,0,0,1,0,0,0,0,0,0,0,1,0,0], - "OFFSET" : [0,0,0,0,0,0,3,3,3,3,3,3,3,3,6,6,6], - "DATA" : ["","","","","","baz","","","","","","","","baz","",""] - },{ - "name" : "max", - "count" : 16, - "VALIDITY" : [0,0,0,0,0,1,0,0,0,0,0,0,0,1,0,0], - "OFFSET" : [0,0,0,0,0,0,4,4,4,4,4,4,4,4,8,8,8], - "DATA" : ["","","","","","quux","","","","","","","","quux","",""] - }] - },{ - "name" : "list", - "count" : 16, - "VALIDITY" : [0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,1], - "DATA" : [0,0,0,0,0,0,0,6,0,0,0,0,0,0,0,14] - },{ - "name" : "f64", - "count" : 16, - "VALIDITY" : [0,0,0,0,0,0,1,0,0,0,0,0,0,0,1,0], - "children" : [{ - "name" : "min", - "count" : 16, - "VALIDITY" : [0,0,0,0,0,0,1,0,0,0,0,0,0,0,1,0], - "DATA" : [0.0,0.0,0.0,0.0,0.0,0.0,12.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,12.0,0.0] - },{ - "name" : "max", - "count" : 16, - "VALIDITY" : [0,0,0,0,0,0,1,0,0,0,0,0,0,0,1,0], - "DATA" : [0.0,0.0,0.0,0.0,0.0,0.0,12.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,12.0,0.0] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 16, - "VALIDITY" : [0,0,0,0,0,0,1,0,0,0,0,0,0,0,1,0], - "children" : [{ - "name" : "min", - "count" : 16, - "VALIDITY" : [0,0,0,0,0,0,1,0,0,0,0,0,0,0,1,0], - "DATA" : [0,0,0,0,0,0,1577836800000000,0,0,0,0,0,0,0,1577836800000000,0] - },{ - "name" : "max", - "count" : 16, - "VALIDITY" : [0,0,0,0,0,0,1,0,0,0,0,0,0,0,1,0], - "DATA" : [0,0,0,0,0,0,1577836800000000,0,0,0,0,0,0,0,1577836800000000,0] - }] - }] - },{ - "name" : "bloom", - "count" : 16, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,108,144,180,224,290,414,510,568,676,712,748,792,858,982,1078,1136], - "DATA" : ["3a30000008000000000006000400000005000000080000000a0001000d0000000e0001000f0002004800000056000000580000005a0000005c0000006000000062000000660000000000010002000300040005002beed294fc2a16776a4af8558e13c989dbd6706b57c3ade1","3a300000020000000000040008000000180000002200000001001300260039002bee1677","3a30000002000000000004000100000018000000220000001f003e005d006200bea01a41","3a3000000300000000000300070000000f00000020000000280000002a0000001f003e005d0063001dabd755","3a30000005000000000004000200000005000000090000000a000000300000003a0000003c0000003e000000400000007c00f8007401be0f774bb6c1b363ba6895d7","3a3000000a0000000000020001000300020003000300000004000000050000000b0000000c0000000d0000000e000000580000005e000000660000006e00000070000000720000007400000076000000780000007a000000180034000fc570096f86068ab09d891a911af03199cba17cfbe7717542818efc6d97617e","3a3000000800000000000200010001000200010008000000090000000c0000000d0000000f000000480000004e0000005200000056000000580000005a0000005c0000005e0000001100220033007009b09df031ea4200004c012a6e752108db","3a3000000400000000000200090001000a0000000b000200280000002e00000032000000340000007c00f8007401649baae6f0310986a1bd39f5","3a30000008000000000006000400000005000000080000000a0001000d0000000e0001000f0002004800000056000000580000005a0000005c0000006000000062000000660000000000010002000300040005002beed294fc2a16776a4af8558e13c989dbd6706b57c3ade1","3a300000020000000000040008000000180000002200000001001300260039002bee1677","3a30000002000000000004000100000018000000220000001f003e005d006200bea01a41","3a3000000300000000000300070000000f00000020000000280000002a0000001f003e005d0063001dabd755","3a30000005000000000004000200000005000000090000000a000000300000003a0000003c0000003e000000400000007c00f8007401be0f774bb6c1b363ba6895d7","3a3000000a0000000000020001000300020003000300000004000000050000000b0000000c0000000d0000000e000000580000005e000000660000006e00000070000000720000007400000076000000780000007a000000180034000fc570096f86068ab09d891a911af03199cba17cfbe7717542818efc6d97617e","3a3000000800000000000200010001000200010008000000090000000c0000000d0000000f000000480000004e0000005200000056000000580000005a0000005c0000005e0000001100220033007009b09df031ea4200004c012a6e752108db","3a3000000400000000000200090001000a0000000b000200280000002e00000032000000340000007c00f8007401649baae6f0310986a1bd39f5"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-_row_id.arrow.json deleted file mode 100644 index 63f4a8b551..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "_row_id", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["0","1"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-bar.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-bar.arrow.json deleted file mode 100644 index be0ceea939..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-bar.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "bar", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "bar", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,5,10], - "DATA" : ["hello","world"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-baz.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-baz.arrow.json deleted file mode 100644 index b8c0900897..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-baz.arrow.json +++ /dev/null @@ -1,51 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "baz", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "f64", - "nullable" : false, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - },{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "baz", - "count" : 2, - "TYPE_ID" : [1,0], - "OFFSET" : [0,0], - "children" : [{ - "name" : "f64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [3.3] - },{ - "name" : "i64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["12"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-foo.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-foo.arrow.json deleted file mode 100644 index 9063578a64..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-foo.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "foo", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "foo", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "i64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["2","1"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-xt$id.arrow.json deleted file mode 100644 index c502000b5b..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/content-xt$id.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "xt$id", - "count" : 2, - "TYPE_ID" : [0,0], - "OFFSET" : [0,1], - "children" : [{ - "name" : "i64", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["0","1"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/metadata.arrow.json deleted file mode 100644 index d5902a1c3a..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/table/metadata.arrow.json +++ /dev/null @@ -1,228 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "utf8", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - },{ - "name" : "f64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - }] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,5,10], - "children" : [{ - "name" : "$data$", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,7,12,15,18,21,28,33,36,39,42], - "DATA" : ["_row_id","xt$id","foo","bar","baz","_row_id","xt$id","foo","bar","baz"] - },{ - "name" : "root-col?", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "DATA" : ["2","2","2","2","2","2","2","2","2","2"] - },{ - "name" : "types", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 10, - "VALIDITY" : [1,1,1,0,1,1,1,1,0,1], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [1,1,1,0,1,1,1,1,0,1], - "DATA" : ["0","0","1","0","12","0","0","1","0","12"] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [1,1,1,0,1,1,1,1,0,1], - "DATA" : ["1","1","2","0","12","1","1","2","0","12"] - }] - },{ - "name" : "utf8", - "count" : 10, - "VALIDITY" : [0,0,0,1,0,0,0,0,1,0], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [0,0,0,1,0,0,0,0,1,0], - "OFFSET" : [0,0,0,0,5,5,5,5,5,10,10], - "DATA" : ["","","","hello","","","","","hello",""] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [0,0,0,1,0,0,0,0,1,0], - "OFFSET" : [0,0,0,0,5,5,5,5,5,10,10], - "DATA" : ["","","","world","","","","","world",""] - }] - },{ - "name" : "f64", - "count" : 10, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,1], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,1], - "DATA" : [0.0,0.0,0.0,0.0,3.3,0.0,0.0,0.0,0.0,3.3] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,1], - "DATA" : [0.0,0.0,0.0,0.0,3.3,0.0,0.0,0.0,0.0,3.3] - }] - }] - },{ - "name" : "bloom", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,52,104,156,208,260,312,364,416,468,520], - "DATA" : ["3a300000040000000000020005000000080000000a000000280000002e0000003000000032000000000001002beefc2a1677f855","3a300000040000000000020005000000080000000a000000280000002e0000003000000032000000000001002beefc2a1677f855","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a3000000400000002000000090000000a0002000b000000280000002a0000002c00000032000000ddbddbf0971ca94c77a8235f","3a300000040000000000020002000000090000000c000000280000002e00000030000000320000000c00e014b429d7e3aec70000","3a300000040000000000020005000000080000000a000000280000002e0000003000000032000000000001002beefc2a1677f855","3a300000040000000000020005000000080000000a000000280000002e0000003000000032000000000001002beefc2a1677f855","3a300000040000000000020004000000080000000a000000280000002e0000003000000032000000010002002beed29416776a4a","3a3000000400000002000000090000000a0002000b000000280000002a0000002c00000032000000ddbddbf0971ca94c77a8235f","3a300000040000000000020002000000090000000c000000280000002e00000030000000320000000c00e014b429d7e3aec70000"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-_row_id.arrow.json deleted file mode 100644 index 086485d271..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "_row_id", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["2"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$committed?.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$committed?.arrow.json deleted file mode 100644 index 7c4d19982c..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$committed?.arrow.json +++ /dev/null @@ -1,36 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$committed?", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "bool", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$committed?", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "bool", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [1] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$error.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$error.arrow.json deleted file mode 100644 index 668b9d40d8..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$error.arrow.json +++ /dev/null @@ -1,54 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$error", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "clj-form", - "nullable" : false, - "type" : { - "name" : "ClojureFormType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "xt/clj-form", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "null", - "nullable" : true, - "type" : { - "name" : "null" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$error", - "count" : 1, - "TYPE_ID" : [1], - "OFFSET" : [0], - "children" : [{ - "name" : "clj-form", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "null", - "count" : 1 - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$id.arrow.json deleted file mode 100644 index bd21b780af..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$id.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$id", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "i64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["0"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$tx_time.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$tx_time.arrow.json deleted file mode 100644 index 67b1578fec..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/content-xt$tx_time.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$tx_time", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "nullable" : false, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$tx_time", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [1577836800000000] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/metadata.arrow.json deleted file mode 100644 index f115534c51..0000000000 --- a/src/test/resources/xtdb/indexer-test/can-index-sql-insert/chunk-00/xt$txs/metadata.arrow.json +++ /dev/null @@ -1,216 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - },{ - "name" : "bool", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "null", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,5,10], - "children" : [{ - "name" : "$data$", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,7,12,22,35,43,50,55,65,78,86], - "DATA" : ["_row_id","xt$id","xt$tx_time","xt$committed?","xt$error","_row_id","xt$id","xt$tx_time","xt$committed?","xt$error"] - },{ - "name" : "root-col?", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "DATA" : ["1","1","1","1","1","1","1","1","1","1"] - },{ - "name" : "types", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["2","0","0","0","0","2","0","0","0","0"] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["2","0","0","0","0","2","0","0","0","0"] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "DATA" : [0,0,1577836800000000,0,0,0,0,1577836800000000,0,0] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "DATA" : [0,0,1577836800000000,0,0,0,0,1577836800000000,0,0] - }] - },{ - "name" : "bool", - "count" : 10, - "VALIDITY" : [0,0,0,1,0,0,0,0,1,0], - "DATA" : [0,0,0,1,0,0,0,0,1,0] - },{ - "name" : "null", - "count" : 10, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,1], - "DATA" : [0,0,0,0,1,0,0,0,0,1] - }] - },{ - "name" : "bloom", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,38,76,114,136,158,196,234,272,294,316], - "DATA" : ["3a3000000300000000000000040000000a0000002000000022000000240000000200d2946a4a","3a3000000300000000000000050000000a0000002000000022000000240000000000fc2af855","3a30000003000000090000000c0000000f0000002000000022000000240000004c012a6e08db","3a300000010000000000020010000000130026003900","3a3000000100000000000200100000001f003e005d00","3a3000000300000000000000040000000a0000002000000022000000240000000200d2946a4a","3a3000000300000000000000050000000a0000002000000022000000240000000000fc2af855","3a30000003000000090000000c0000000f0000002000000022000000240000004c012a6e08db","3a300000010000000000020010000000130026003900","3a3000000100000000000200100000001f003e005d00"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-_row_id.arrow.json deleted file mode 100644 index 284994b97f..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-_row_id.arrow.json +++ /dev/null @@ -1,39 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "_row_id", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["4"] - }] - },{ - "count" : 1, - "columns" : [{ - "name" : "_row_id", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["7"] - }] - },{ - "count" : 0, - "columns" : [{ - "name" : "_row_id", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$committed?.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$committed?.arrow.json deleted file mode 100644 index 542bd7a70f..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$committed?.arrow.json +++ /dev/null @@ -1,64 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$committed?", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "bool", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$committed?", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "bool", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [1] - }] - }] - },{ - "count" : 1, - "columns" : [{ - "name" : "xt$committed?", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "bool", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [1] - }] - }] - },{ - "count" : 0, - "columns" : [{ - "name" : "xt$committed?", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "bool", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$error.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$error.arrow.json deleted file mode 100644 index 1bf3339421..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$error.arrow.json +++ /dev/null @@ -1,90 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$error", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "clj-form", - "nullable" : false, - "type" : { - "name" : "ClojureFormType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "xt/clj-form", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "null", - "nullable" : true, - "type" : { - "name" : "null" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$error", - "count" : 1, - "TYPE_ID" : [1], - "OFFSET" : [0], - "children" : [{ - "name" : "clj-form", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "null", - "count" : 1 - }] - }] - },{ - "count" : 1, - "columns" : [{ - "name" : "xt$error", - "count" : 1, - "TYPE_ID" : [1], - "OFFSET" : [0], - "children" : [{ - "name" : "clj-form", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "null", - "count" : 1 - }] - }] - },{ - "count" : 0, - "columns" : [{ - "name" : "xt$error", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "clj-form", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "null", - "count" : 0 - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$id.arrow.json deleted file mode 100644 index bfbd866186..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$id.arrow.json +++ /dev/null @@ -1,66 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$id", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "i64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["0"] - }] - }] - },{ - "count" : 1, - "columns" : [{ - "name" : "xt$id", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "i64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["5845"] - }] - }] - },{ - "count" : 0, - "columns" : [{ - "name" : "xt$id", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "i64", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$tx_time.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$tx_time.arrow.json deleted file mode 100644 index 5751d5458f..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/content-xt$tx_time.arrow.json +++ /dev/null @@ -1,66 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$tx_time", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "nullable" : false, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 1, - "columns" : [{ - "name" : "xt$tx_time", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [1577836800000000] - }] - }] - },{ - "count" : 1, - "columns" : [{ - "name" : "xt$tx_time", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [1577923200000000] - }] - }] - },{ - "count" : 0, - "columns" : [{ - "name" : "xt$tx_time", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/metadata.arrow.json deleted file mode 100644 index 98ce19ab25..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt$txs/metadata.arrow.json +++ /dev/null @@ -1,216 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - },{ - "name" : "bool", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "null", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 4, - "columns" : [{ - "name" : "block-idx", - "count" : 4, - "VALIDITY" : [1,1,1,0], - "DATA" : [0,1,2,0] - },{ - "name" : "columns", - "count" : 4, - "VALIDITY" : [1,1,1,1], - "OFFSET" : [0,5,10,10,15], - "children" : [{ - "name" : "$data$", - "count" : 15, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 15, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,7,12,22,35,43,50,55,65,78,86,93,98,108,121,129], - "DATA" : ["_row_id","xt$id","xt$tx_time","xt$committed?","xt$error","_row_id","xt$id","xt$tx_time","xt$committed?","xt$error","_row_id","xt$id","xt$tx_time","xt$committed?","xt$error"] - },{ - "name" : "root-col?", - "count" : 15, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 15, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["1","1","1","1","1","1","1","1","1","1","2","2","2","2","2"] - },{ - "name" : "types", - "count" : 15, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 15, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0,1,1,0,0,0], - "children" : [{ - "name" : "min", - "count" : 15, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0,1,1,0,0,0], - "DATA" : ["4","0","0","0","0","7","5845","0","0","0","4","0","0","0","0"] - },{ - "name" : "max", - "count" : 15, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0,1,1,0,0,0], - "DATA" : ["4","0","0","0","0","7","5845","0","0","0","7","5845","0","0","0"] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 15, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0,0,0,1,0,0], - "children" : [{ - "name" : "min", - "count" : 15, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0,0,0,1,0,0], - "DATA" : [0,0,1577836800000000,0,0,0,0,1577923200000000,0,0,0,0,1577836800000000,0,0] - },{ - "name" : "max", - "count" : 15, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0,0,0,1,0,0], - "DATA" : [0,0,1577836800000000,0,0,0,0,1577923200000000,0,0,0,0,1577923200000000,0,0] - }] - },{ - "name" : "bool", - "count" : 15, - "VALIDITY" : [0,0,0,1,0,0,0,0,1,0,0,0,0,1,0], - "DATA" : [0,0,0,1,0,0,0,0,1,0,0,0,0,1,0] - },{ - "name" : "null", - "count" : 15, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,1,0,0,0,0,1], - "DATA" : [0,0,0,0,1,0,0,0,0,1,0,0,0,0,1] - }] - },{ - "name" : "bloom", - "count" : 15, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,38,76,114,136,158,196,234,272,294,316,376,436,504,526,548], - "DATA" : ["3a30000003000000000000000d0000000e00000020000000220000002400000004008e13c989","3a3000000300000000000000050000000a0000002000000022000000240000000000fc2af855","3a30000003000000090000000c0000000f0000002000000022000000240000004c012a6e08db","3a300000010000000000020010000000130026003900","3a3000000100000000000200100000001f003e005d00","3a3000000300000000000000050000000b0000002000000022000000240000000700ce909521","3a30000003000000000000000100000008000000200000002200000024000000d516eb82e0cc","3a3000000300000002000000040000000b0000002000000022000000240000000af41c3b9397","3a300000010000000000020010000000130026003900","3a3000000100000000000200100000003e007c00ba00","3a3000000500000000000100050000000b0000000d0000000e000000300000003400000036000000380000003a00000004000700ce9095218e13c989","3a30000005000000000001000100000005000000080000000a000000300000003400000036000000380000003a0000000000d516eb82fc2ae0ccf855","3a300000060000000200000004000000090000000b0000000c0000000f000000380000003a0000003c0000003e00000040000000420000000af41c3b4c0193972a6e08db","3a300000010000000000020010000000130026003900","3a3000000100000000000200100000003e007c00ba00"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-_row_id.arrow.json deleted file mode 100644 index 3b48c4814c..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-_row_id.arrow.json +++ /dev/null @@ -1,39 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 4, - "columns" : [{ - "name" : "_row_id", - "count" : 4, - "VALIDITY" : [1,1,1,1], - "DATA" : ["0","1","2","3"] - }] - },{ - "count" : 2, - "columns" : [{ - "name" : "_row_id", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : ["5","6"] - }] - },{ - "count" : 0, - "columns" : [{ - "name" : "_row_id", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-list.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-list.arrow.json deleted file mode 100644 index e5bc70198e..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-list.arrow.json +++ /dev/null @@ -1,211 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "list", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "list", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "f64", - "nullable" : false, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - },{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "timestamp-tz-micro-utc", - "nullable" : false, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - },{ - "name" : "bool", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - }] - },{ - "name" : "absent", - "nullable" : false, - "type" : { - "name" : "AbsentType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "absent", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - }] - }] - }, - "batches" : [{ - "count" : 4, - "columns" : [{ - "name" : "list", - "count" : 4, - "TYPE_ID" : [0,1,0,1], - "OFFSET" : [0,0,1,1], - "children" : [{ - "name" : "list", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,2,4], - "children" : [{ - "name" : "$data$", - "count" : 4, - "TYPE_ID" : [0,1,2,3], - "OFFSET" : [0,0,0,0], - "children" : [{ - "name" : "f64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [12.0] - },{ - "name" : "utf8", - "count" : 1, - "VALIDITY" : [1], - "OFFSET" : [0,3], - "DATA" : ["foo"] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [1577836800000000] - },{ - "name" : "bool", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [0] - }] - }] - },{ - "name" : "absent", - "count" : 2 - }] - }] - },{ - "count" : 2, - "columns" : [{ - "name" : "list", - "count" : 2, - "TYPE_ID" : [1,1], - "OFFSET" : [0,1], - "children" : [{ - "name" : "list", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "children" : [{ - "name" : "$data$", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "f64", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - },{ - "name" : "utf8", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - },{ - "name" : "bool", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - }] - }] - },{ - "name" : "absent", - "count" : 2 - }] - }] - },{ - "count" : 0, - "columns" : [{ - "name" : "list", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "list", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "children" : [{ - "name" : "$data$", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "f64", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - },{ - "name" : "utf8", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - },{ - "name" : "bool", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - }] - }] - },{ - "name" : "absent", - "count" : 0 - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-struct.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-struct.arrow.json deleted file mode 100644 index a51d98be2c..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-struct.arrow.json +++ /dev/null @@ -1,338 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "struct", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "absent", - "nullable" : false, - "type" : { - "name" : "AbsentType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "absent", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "struct", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "a", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "bool", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - },{ - "name" : "b", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "struct", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "c", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - },{ - "name" : "d", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }] - }] - }] - }] - }, - "batches" : [{ - "count" : 4, - "columns" : [{ - "name" : "struct", - "count" : 4, - "TYPE_ID" : [0,1,0,0], - "OFFSET" : [0,0,1,2], - "children" : [{ - "name" : "absent", - "count" : 3 - },{ - "name" : "struct", - "count" : 1, - "VALIDITY" : [1], - "children" : [{ - "name" : "a", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "i64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["1"] - },{ - "name" : "bool", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - }] - },{ - "name" : "b", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "utf8", - "count" : 1, - "VALIDITY" : [1], - "OFFSET" : [0,1], - "DATA" : ["b"] - },{ - "name" : "struct", - "count" : 0, - "VALIDITY" : [ ], - "children" : [{ - "name" : "c", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "utf8", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - }] - },{ - "name" : "d", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "utf8", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - }] - }] - }] - }] - }] - }] - },{ - "count" : 2, - "columns" : [{ - "name" : "struct", - "count" : 2, - "TYPE_ID" : [0,1], - "OFFSET" : [0,0], - "children" : [{ - "name" : "absent", - "count" : 1 - },{ - "name" : "struct", - "count" : 1, - "VALIDITY" : [1], - "children" : [{ - "name" : "a", - "count" : 1, - "TYPE_ID" : [1], - "OFFSET" : [0], - "children" : [{ - "name" : "i64", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - },{ - "name" : "bool", - "count" : 1, - "VALIDITY" : [1], - "DATA" : [1] - }] - },{ - "name" : "b", - "count" : 1, - "TYPE_ID" : [1], - "OFFSET" : [0], - "children" : [{ - "name" : "utf8", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "struct", - "count" : 1, - "VALIDITY" : [1], - "children" : [{ - "name" : "c", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "utf8", - "count" : 1, - "VALIDITY" : [1], - "OFFSET" : [0,1], - "DATA" : ["c"] - }] - },{ - "name" : "d", - "count" : 1, - "TYPE_ID" : [0], - "OFFSET" : [0], - "children" : [{ - "name" : "utf8", - "count" : 1, - "VALIDITY" : [1], - "OFFSET" : [0,1], - "DATA" : ["d"] - }] - }] - }] - }] - }] - }] - },{ - "count" : 0, - "columns" : [{ - "name" : "struct", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "absent", - "count" : 0 - },{ - "name" : "struct", - "count" : 0, - "VALIDITY" : [ ], - "children" : [{ - "name" : "a", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "i64", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - },{ - "name" : "bool", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - }] - },{ - "name" : "b", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "utf8", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "struct", - "count" : 0, - "VALIDITY" : [ ], - "children" : [{ - "name" : "c", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "utf8", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - }] - },{ - "name" : "d", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "utf8", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - }] - }] - }] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-xt$id.arrow.json deleted file mode 100644 index 3110740df0..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/content-xt$id.arrow.json +++ /dev/null @@ -1,123 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "keyword", - "nullable" : false, - "type" : { - "name" : "KeywordType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "xt/clj-keyword", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 4, - "columns" : [{ - "name" : "xt$id", - "count" : 4, - "TYPE_ID" : [0,1,0,2], - "OFFSET" : [0,0,1,0], - "children" : [{ - "name" : "utf8", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,3,6], - "DATA" : ["foo","baz"] - },{ - "name" : "keyword", - "count" : 1, - "VALIDITY" : [1], - "OFFSET" : [0,3], - "DATA" : ["bar"] - },{ - "name" : "i64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["24"] - }] - }] - },{ - "count" : 2, - "columns" : [{ - "name" : "xt$id", - "count" : 2, - "TYPE_ID" : [2,1], - "OFFSET" : [0,0], - "children" : [{ - "name" : "utf8", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "keyword", - "count" : 1, - "VALIDITY" : [1], - "OFFSET" : [0,4], - "DATA" : ["quux"] - },{ - "name" : "i64", - "count" : 1, - "VALIDITY" : [1], - "DATA" : ["52"] - }] - }] - },{ - "count" : 0, - "columns" : [{ - "name" : "xt$id", - "count" : 0, - "TYPE_ID" : [ ], - "OFFSET" : [ ], - "children" : [{ - "name" : "utf8", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "keyword", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "i64", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/metadata.arrow.json deleted file mode 100644 index cbe95559cc..0000000000 --- a/src/test/resources/xtdb/indexer-test/multi-block-metadata/chunk-00/xt_docs/metadata.arrow.json +++ /dev/null @@ -1,439 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "absent", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "struct-2", - "nullable" : true, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "utf8", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - },{ - "name" : "keyword", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "KeywordType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "xt/clj-keyword", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "KeywordType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "xt/clj-keyword", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - }] - },{ - "name" : "list", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "f64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "floatingpoint", - "precision" : "DOUBLE" - }, - "children" : [ ] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - },{ - "name" : "bool", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "struct-9", - "nullable" : true, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "struct-10", - "nullable" : true, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - }] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 4, - "columns" : [{ - "name" : "block-idx", - "count" : 4, - "VALIDITY" : [1,1,1,0], - "DATA" : [0,1,2,0] - },{ - "name" : "columns", - "count" : 4, - "VALIDITY" : [1,1,1,1], - "OFFSET" : [0,7,15,15,24], - "children" : [{ - "name" : "$data$", - "count" : 24, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 24, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,7,8,9,15,20,26,30,37,38,39,40,41,47,52,56,63,64,65,66,67,73,78,84,88], - "DATA" : ["_row_id","a","b","struct","xt$id","$data$","list","_row_id","a","c","d","b","struct","xt$id","list","_row_id","a","c","d","b","struct","xt$id","$data$","list"] - },{ - "name" : "root-col?", - "count" : 24, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : [1,0,0,1,1,0,1,1,0,0,0,0,1,1,1,1,0,0,0,0,1,1,0,1] - },{ - "name" : "count", - "count" : 24, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "DATA" : ["4","1","1","4","4","4","4","2","1","1","1","1","2","2","2","6","2","1","1","2","6","6","4","6"] - },{ - "name" : "types", - "count" : 24, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 24, - "VALIDITY" : [1,1,0,0,1,0,0,1,0,0,0,0,0,1,0,1,1,0,0,0,0,1,0,0], - "children" : [{ - "name" : "min", - "count" : 24, - "VALIDITY" : [1,1,0,0,1,0,0,1,0,0,0,0,0,1,0,1,1,0,0,0,0,1,0,0], - "DATA" : ["0","1","0","0","24","0","0","5","0","0","0","0","0","52","0","0","1","0","0","0","0","24","0","0"] - },{ - "name" : "max", - "count" : 24, - "VALIDITY" : [1,1,0,0,1,0,0,1,0,0,0,0,0,1,0,1,1,0,0,0,0,1,0,0], - "DATA" : ["3","1","0","0","24","0","0","6","0","0","0","0","0","52","0","6","1","0","0","0","0","52","0","0"] - }] - },{ - "name" : "absent", - "count" : 24, - "VALIDITY" : [0,0,0,1,0,0,1,0,0,0,0,0,1,0,1,0,0,0,0,0,1,0,0,1], - "DATA" : [0,0,0,1,0,0,1,0,0,0,0,0,1,0,1,0,0,0,0,0,1,0,0,1] - },{ - "name" : "struct-2", - "count" : 24, - "VALIDITY" : [0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0], - "OFFSET" : [0,0,0,0,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2], - "children" : [{ - "name" : "$data$", - "count" : 2, - "VALIDITY" : [1,1], - "DATA" : [1,2] - }] - },{ - "name" : "utf8", - "count" : 24, - "VALIDITY" : [0,0,1,0,1,1,0,0,0,1,1,0,0,0,0,0,0,1,1,1,0,1,1,0], - "children" : [{ - "name" : "min", - "count" : 24, - "VALIDITY" : [0,0,1,0,1,1,0,0,0,1,1,0,0,0,0,0,0,1,1,1,0,1,1,0], - "OFFSET" : [0,0,0,1,1,4,7,7,7,7,8,9,9,9,9,9,9,9,10,11,12,12,15,18,18], - "DATA" : ["","","b","","baz","foo","","","","c","d","","","","","","","c","d","b","","baz","foo",""] - },{ - "name" : "max", - "count" : 24, - "VALIDITY" : [0,0,1,0,1,1,0,0,0,1,1,0,0,0,0,0,0,1,1,1,0,1,1,0], - "OFFSET" : [0,0,0,1,1,4,7,7,7,7,8,9,9,9,9,9,9,9,10,11,12,12,15,18,18], - "DATA" : ["","","b","","foo","foo","","","","c","d","","","","","","","c","d","b","","foo","foo",""] - }] - },{ - "name" : "keyword", - "count" : 24, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,1,0,0], - "children" : [{ - "name" : "min", - "count" : 24, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,1,0,0], - "OFFSET" : [0,0,0,0,0,3,3,3,3,3,3,3,3,3,7,7,7,7,7,7,7,7,10,10,10], - "DATA" : ["","","","","bar","","","","","","","","","quux","","","","","","","","bar","",""] - },{ - "name" : "max", - "count" : 24, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,1,0,0], - "OFFSET" : [0,0,0,0,0,3,3,3,3,3,3,3,3,3,7,7,7,7,7,7,7,7,11,11,11], - "DATA" : ["","","","","bar","","","","","","","","","quux","","","","","","","","quux","",""] - }] - },{ - "name" : "list", - "count" : 24, - "VALIDITY" : [0,0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1], - "DATA" : [0,0,0,0,0,0,5,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,22] - },{ - "name" : "f64", - "count" : 24, - "VALIDITY" : [0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0], - "children" : [{ - "name" : "min", - "count" : 24, - "VALIDITY" : [0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0], - "DATA" : [0.0,0.0,0.0,0.0,0.0,12.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,12.0,0.0] - },{ - "name" : "max", - "count" : 24, - "VALIDITY" : [0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0], - "DATA" : [0.0,0.0,0.0,0.0,0.0,12.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,0.0,12.0,0.0] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 24, - "VALIDITY" : [0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0], - "children" : [{ - "name" : "min", - "count" : 24, - "VALIDITY" : [0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0], - "DATA" : [0,0,0,0,0,1577836800000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1577836800000000,0] - },{ - "name" : "max", - "count" : 24, - "VALIDITY" : [0,0,0,0,0,1,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1,0], - "DATA" : [0,0,0,0,0,1577836800000000,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,1577836800000000,0] - }] - },{ - "name" : "bool", - "count" : 24, - "VALIDITY" : [0,0,0,0,0,1,0,0,1,0,0,0,0,0,0,0,1,0,0,0,0,0,1,0], - "DATA" : [0,0,0,0,0,1,0,0,1,0,0,0,0,0,0,0,1,0,0,0,0,0,1,0] - },{ - "name" : "struct-9", - "count" : 24, - "VALIDITY" : [0,0,0,0,0,0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,1,0,0,0], - "OFFSET" : [0,0,0,0,0,0,0,0,0,0,0,0,0,2,2,2,2,2,2,2,2,4,4,4,4], - "children" : [{ - "name" : "$data$", - "count" : 4, - "VALIDITY" : [1,1,1,1], - "DATA" : [8,11,16,19] - }] - },{ - "name" : "struct-10", - "count" : 24, - "VALIDITY" : [0,0,0,0,0,0,0,0,0,0,0,1,0,0,0,0,0,0,0,1,0,0,0,0], - "OFFSET" : [0,0,0,0,0,0,0,0,0,0,0,0,2,2,2,2,2,2,2,2,4,4,4,4,4], - "children" : [{ - "name" : "$data$", - "count" : 4, - "VALIDITY" : [1,1,1,1], - "DATA" : [9,10,17,18] - }] - }] - },{ - "name" : "bloom", - "count" : 24, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,80,110,140,184,264,360,418,478,500,538,576,614,658,726,748,856,892,930,968,1020,1078,1202,1298,1356], - "DATA" : ["3a30000006000000000004000400000005000000080000000a0001000f00010038000000420000004400000046000000480000004c00000000000100020003002beed294fc2a16776a4af85557c3ade1","3a300000020000000000010008000000180000001c00000001002bee1677","3a300000020000000000010001000000180000001c0000006200bea01a41","3a30000003000000000003000e0000000f00000020000000280000002a0000005d008100ba001701379bdc4d","3a3000000600000000000100010002000200030003000000040000000b000000380000003c000000420000004a0000004c0000004e00000018000fc57009068ab09d891a911af03199cba17cfbe74281","3a3000000800000000000200010001000200010008000000090000000c0000000d0000000f000000480000004e0000005200000056000000580000005a0000005c0000005e0000001100220033007009b09df031ea4200004c012a6e752108db","3a3000000400000000000200090001000a0000000b000200280000002e00000032000000340000003e007c00ba00649baae6f0310986a1bd39f5","3a3000000500000000000100050000000b0000000e0000000f000000300000003400000036000000380000003a000000050006007789e812dbd6706b","3a300000010000000000020010000000130026003900","3a3000000300000000000000070000000f00000020000000220000002400000063001dabd755","3a3000000300000000000000080000000c00000020000000220000002400000064009cde806f","3a30000003000000000000000300000007000000200000002200000024000000610c41982124","3a3000000300000000000300070000000f00000020000000280000002a0000007c00f8007401ae0e7bb5485c","3a300000060000000000000001000000050000000c0000000d0000000e000000380000003a0000003c0000003e000000400000004200000034006f8671758efc6d97617e","3a3000000100000000000200100000007c00f8007401","3a30000008000000000006000400000005000100080000000a0001000b0000000e0000000f0002004800000056000000580000005c0000005e0000006200000064000000660000000000010002000300050006002beed294fc2a778916776a4af855e812dbd6706b57c3ade1","3a300000020000000000040008000000180000002200000001001300260039002bee1677","3a3000000300000000000000070000000f00000020000000220000002400000063001dabd755","3a3000000300000000000000080000000c00000020000000220000002400000064009cde806f","3a3000000400000000000200010000000300000007000000280000002e00000030000000320000006200610cbea01a4141982124","3a3000000400000000000400070000000e0000000f000100280000003200000034000000360000007c008100f8007401ae0e7bb5379bdc4d485c","3a3000000a0000000000020001000300020003000300000004000000050000000b0000000c0000000d0000000e000000580000005e000000660000006e00000070000000720000007400000076000000780000007a000000180034000fc570096f86068ab09d891a911af03199cba17cfbe7717542818efc6d97617e","3a3000000800000000000200010001000200010008000000090000000c0000000d0000000f000000480000004e0000005200000056000000580000005a0000005c0000005e0000001100220033007009b09df031ea4200004c012a6e752108db","3a3000000400000000000200090001000a0000000b000200280000002e00000032000000340000007c00f8007401649baae6f0310986a1bd39f5"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/foo/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/foo/content-_row_id.arrow.json deleted file mode 100644 index eeb11804fa..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/foo/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 0, - "columns" : [{ - "name" : "_row_id", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/foo/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/foo/metadata.arrow.json deleted file mode 100644 index 217932b327..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/foo/metadata.arrow.json +++ /dev/null @@ -1,111 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [ ] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,0,0], - "children" : [{ - "name" : "$data$", - "count" : 0, - "VALIDITY" : [ ], - "children" : [{ - "name" : "col-name", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - },{ - "name" : "root-col?", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - },{ - "name" : "count", - "count" : 0, - "VALIDITY" : [ ], - "DATA" : [ ] - },{ - "name" : "types", - "count" : 0, - "VALIDITY" : [ ] - },{ - "name" : "bloom", - "count" : 0, - "VALIDITY" : [ ], - "OFFSET" : [0], - "DATA" : [ ] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-_row_id.arrow.json deleted file mode 100644 index c5aafd432e..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "_row_id", - "count" : 3, - "VALIDITY" : [1,1,1], - "DATA" : ["2","3","5"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$committed?.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$committed?.arrow.json deleted file mode 100644 index 9ce263e4b4..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$committed?.arrow.json +++ /dev/null @@ -1,36 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$committed?", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "bool", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "xt$committed?", - "count" : 3, - "TYPE_ID" : [0,0,0], - "OFFSET" : [0,1,2], - "children" : [{ - "name" : "bool", - "count" : 3, - "VALIDITY" : [1,1,1], - "DATA" : [1,0,1] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$error.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$error.arrow.json deleted file mode 100644 index 12df5569c5..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$error.arrow.json +++ /dev/null @@ -1,54 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$error", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "clj-form", - "nullable" : false, - "type" : { - "name" : "ClojureFormType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "xt/clj-form", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "null", - "nullable" : true, - "type" : { - "name" : "null" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "xt$error", - "count" : 3, - "TYPE_ID" : [1,0,1], - "OFFSET" : [0,0,1], - "children" : [{ - "name" : "clj-form", - "count" : 1, - "VALIDITY" : [1], - "OFFSET" : [0,284], - "DATA" : ["#xt/runtime-err {:xtdb.error/error-type :runtime-error, :xtdb.error/error-key :xtdb.indexer/invalid-valid-times, :xtdb.error/message \"Runtime error: ':xtdb.indexer/invalid-valid-times'\", :valid-from #time/instant \"2020-01-01T00:00:00Z\", :valid-to #time/instant \"2019-01-01T00:00:00Z\"}"] - },{ - "name" : "null", - "count" : 2 - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$id.arrow.json deleted file mode 100644 index db47f104ec..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$id.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "i64", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "xt$id", - "count" : 3, - "TYPE_ID" : [0,0,0], - "OFFSET" : [0,1,2], - "children" : [{ - "name" : "i64", - "count" : 3, - "VALIDITY" : [1,1,1], - "DATA" : ["0","3421","6554"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$tx_time.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$tx_time.arrow.json deleted file mode 100644 index cbdfb80f9f..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/content-xt$tx_time.arrow.json +++ /dev/null @@ -1,38 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$tx_time", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "nullable" : false, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "xt$tx_time", - "count" : 3, - "TYPE_ID" : [0,0,0], - "OFFSET" : [0,1,2], - "children" : [{ - "name" : "timestamp-tz-micro-utc", - "count" : 3, - "VALIDITY" : [1,1,1], - "DATA" : [1577836800000000,1577923200000000,1578009600000000] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/metadata.arrow.json deleted file mode 100644 index 42c9c98189..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt$txs/metadata.arrow.json +++ /dev/null @@ -1,228 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "timestamp", - "unit" : "MICROSECOND", - "timezone" : "UTC" - }, - "children" : [ ] - }] - },{ - "name" : "bool", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "clj-form", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "null", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,5,10], - "children" : [{ - "name" : "$data$", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,7,12,22,35,43,50,55,65,78,86], - "DATA" : ["_row_id","xt$id","xt$tx_time","xt$committed?","xt$error","_row_id","xt$id","xt$tx_time","xt$committed?","xt$error"] - },{ - "name" : "root-col?", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "DATA" : ["3","3","3","3","3","3","3","3","3","3"] - },{ - "name" : "types", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["2","0","0","0","0","2","0","0","0","0"] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [1,1,0,0,0,1,1,0,0,0], - "DATA" : ["5","6554","0","0","0","5","6554","0","0","0"] - }] - },{ - "name" : "timestamp-tz-micro-utc", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "children" : [{ - "name" : "min", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "DATA" : [0,0,1577836800000000,0,0,0,0,1577836800000000,0,0] - },{ - "name" : "max", - "count" : 10, - "VALIDITY" : [0,0,1,0,0,0,0,1,0,0], - "DATA" : [0,0,1578009600000000,0,0,0,0,1578009600000000,0,0] - }] - },{ - "name" : "bool", - "count" : 10, - "VALIDITY" : [0,0,0,1,0,0,0,0,1,0], - "DATA" : [0,0,0,1,0,0,0,0,1,0] - },{ - "name" : "clj-form", - "count" : 10, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,1], - "DATA" : [0,0,0,0,1,0,0,0,0,1] - },{ - "name" : "null", - "count" : 10, - "VALIDITY" : [0,0,0,0,1,0,0,0,0,1], - "DATA" : [0,0,0,0,1,0,0,0,0,1] - }] - },{ - "name" : "bloom", - "count" : 10, - "VALIDITY" : [1,1,1,1,1,1,1,1,1,1], - "OFFSET" : [0,66,140,230,258,310,376,450,540,568,620], - "DATA" : ["3a3000000500000000000200040000000a0000000e0000000f0002003000000036000000380000003a0000003c000000020003000500d2946a4adbd6706b57c3ade1","3a300000060000000000020005000000090000000a0000000c0000000f000100380000003e0000004000000042000000440000004600000000005d0d9a19fc2ac562f85511b8f23846a9","3a30000008000000000000000200000004000000090000000b0000000c0001000e0000000f000000480000004a0000004c0000004e00000050000000520000005600000058000000c4020af41c3b4c0193972a6e309bfa4e08db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000003e007c00ba008a6364dfb0e7","3a3000000500000000000200040000000a0000000e0000000f0002003000000036000000380000003a0000003c000000020003000500d2946a4adbd6706b57c3ade1","3a300000060000000000020005000000090000000a0000000c0000000f000100380000003e0000004000000042000000440000004600000000005d0d9a19fc2ac562f85511b8f23846a9","3a30000008000000000000000200000004000000090000000b0000000c0001000e0000000f000000480000004a0000004c0000004e00000050000000520000005600000058000000c4020af41c3b4c0193972a6e309bfa4e08db","3a300000010000000000050010000000110013002200260033003900","3a300000040000000000020002000000040000000f000000280000002e00000030000000320000003e007c00ba008a6364dfb0e7"] - }] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-_row_id.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-_row_id.arrow.json deleted file mode 100644 index fd88a2db73..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-_row_id.arrow.json +++ /dev/null @@ -1,23 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "_row_id", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "_row_id", - "count" : 3, - "VALIDITY" : [1,1,1], - "DATA" : ["0","1","4"] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-month.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-month.arrow.json deleted file mode 100644 index f81001ecb7..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-month.arrow.json +++ /dev/null @@ -1,54 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "month", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "absent", - "nullable" : false, - "type" : { - "name" : "AbsentType" - }, - "children" : [ ], - "metadata" : [{ - "value" : "absent", - "key" : "ARROW:extension:name" - },{ - "value" : "", - "key" : "ARROW:extension:metadata" - }] - },{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "month", - "count" : 3, - "TYPE_ID" : [0,0,1], - "OFFSET" : [0,1,0], - "children" : [{ - "name" : "absent", - "count" : 2 - },{ - "name" : "utf8", - "count" : 1, - "VALIDITY" : [1], - "OFFSET" : [0,5], - "DATA" : ["april"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-xt$id.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-xt$id.arrow.json deleted file mode 100644 index 16d9b71e9d..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/content-xt$id.arrow.json +++ /dev/null @@ -1,37 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "xt$id", - "nullable" : false, - "type" : { - "name" : "union", - "mode" : "Dense", - "typeIds" : [ ] - }, - "children" : [{ - "name" : "utf8", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - }] - }, - "batches" : [{ - "count" : 3, - "columns" : [{ - "name" : "xt$id", - "count" : 3, - "TYPE_ID" : [0,0,0], - "OFFSET" : [0,1,2], - "children" : [{ - "name" : "utf8", - "count" : 3, - "VALIDITY" : [1,1,1], - "OFFSET" : [0,3,6,9], - "DATA" : ["foo","bar","bar"] - }] - }] - }] -} \ No newline at end of file diff --git a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/metadata.arrow.json b/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/metadata.arrow.json deleted file mode 100644 index e5df373abf..0000000000 --- a/src/test/resources/xtdb/indexer-test/writes-log-file/chunk-00/xt_docs/metadata.arrow.json +++ /dev/null @@ -1,202 +0,0 @@ -{ - "schema" : { - "fields" : [{ - "name" : "block-idx", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 32, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "columns", - "nullable" : false, - "type" : { - "name" : "list" - }, - "children" : [{ - "name" : "$data$", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "col-name", - "nullable" : false, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "root-col?", - "nullable" : false, - "type" : { - "name" : "bool" - }, - "children" : [ ] - },{ - "name" : "count", - "nullable" : false, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "types", - "nullable" : false, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "i64", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "int", - "bitWidth" : 64, - "isSigned" : true - }, - "children" : [ ] - }] - },{ - "name" : "utf8", - "nullable" : true, - "type" : { - "name" : "struct" - }, - "children" : [{ - "name" : "min", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - },{ - "name" : "max", - "nullable" : true, - "type" : { - "name" : "utf8" - }, - "children" : [ ] - }] - },{ - "name" : "absent", - "nullable" : true, - "type" : { - "name" : "bool" - }, - "children" : [ ] - }] - },{ - "name" : "bloom", - "nullable" : true, - "type" : { - "name" : "binary" - }, - "children" : [ ] - }] - }] - }] - }, - "batches" : [{ - "count" : 2, - "columns" : [{ - "name" : "block-idx", - "count" : 2, - "VALIDITY" : [1,0], - "DATA" : [0,0] - },{ - "name" : "columns", - "count" : 2, - "VALIDITY" : [1,1], - "OFFSET" : [0,3,6], - "children" : [{ - "name" : "$data$", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "children" : [{ - "name" : "col-name", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "OFFSET" : [0,7,12,17,24,29,34], - "DATA" : ["_row_id","xt$id","month","_row_id","xt$id","month"] - },{ - "name" : "root-col?", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : [1,1,1,1,1,1] - },{ - "name" : "count", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "DATA" : ["3","3","3","3","3","3"] - },{ - "name" : "types", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "children" : [{ - "name" : "i64", - "count" : 6, - "VALIDITY" : [1,0,0,1,0,0], - "children" : [{ - "name" : "min", - "count" : 6, - "VALIDITY" : [1,0,0,1,0,0], - "DATA" : ["0","0","0","0","0","0"] - },{ - "name" : "max", - "count" : 6, - "VALIDITY" : [1,0,0,1,0,0], - "DATA" : ["4","0","0","4","0","0"] - }] - },{ - "name" : "utf8", - "count" : 6, - "VALIDITY" : [0,1,1,0,1,1], - "children" : [{ - "name" : "min", - "count" : 6, - "VALIDITY" : [0,1,1,0,1,1], - "OFFSET" : [0,0,3,8,8,11,16], - "DATA" : ["","bar","april","","bar","april"] - },{ - "name" : "max", - "count" : 6, - "VALIDITY" : [0,1,1,0,1,1], - "OFFSET" : [0,0,3,8,8,11,16], - "DATA" : ["","foo","april","","foo","april"] - }] - },{ - "name" : "absent", - "count" : 6, - "VALIDITY" : [0,0,1,0,0,1], - "DATA" : [0,0,1,0,0,1] - }] - },{ - "name" : "bloom", - "count" : 6, - "VALIDITY" : [1,1,1,1,1,1], - "OFFSET" : [0,74,126,178,252,304,356], - "DATA" : ["3a300000060000000000030005000000080000000a0000000d0000000e0000003800000040000000420000004400000046000000480000000000010004002beefc2a1677f8558e13c989","3a300000040000000100010002000100040000000b000000280000002c00000030000000320000007009b09d891af031fbe74281","3a300000040000000000020001000000050000000a000000280000002e00000030000000320000003e007c00ba00956a83d4713e","3a300000060000000000030005000000080000000a0000000d0000000e0000003800000040000000420000004400000046000000480000000000010004002beefc2a1677f8558e13c989","3a300000040000000100010002000100040000000b000000280000002c00000030000000320000007009b09d891af031fbe74281","3a300000040000000000020001000000050000000a000000280000002e00000030000000320000003e007c00ba00956a83d4713e"] - }] - }] - }] - }] -} \ No newline at end of file