duckdb 0.5.2-dev737.0 → 0.5.2-dev746.0

This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
package/src/duckdb.cpp CHANGED
@@ -17587,9 +17587,15 @@ public:
17587
17587
  idx_t result_idx);
17588
17588
  static unique_ptr<CompressedSegmentState> StringInitSegment(ColumnSegment &segment, block_id_t block_id);
17589
17589
 
17590
- static idx_t StringAppend(ColumnSegment &segment, SegmentStatistics &stats, UnifiedVectorFormat &data, idx_t offset,
17591
- idx_t count) {
17592
- return StringAppendBase(segment, stats, data, offset, count);
17590
+ static unique_ptr<CompressionAppendState> StringInitAppend(ColumnSegment &segment) {
17591
+ auto &buffer_manager = BufferManager::GetBufferManager(segment.db);
17592
+ auto handle = buffer_manager.Pin(segment.block);
17593
+ return make_unique<CompressionAppendState>(move(handle));
17594
+ }
17595
+
17596
+ static idx_t StringAppend(CompressionAppendState &append_state, ColumnSegment &segment, SegmentStatistics &stats,
17597
+ UnifiedVectorFormat &data, idx_t offset, idx_t count) {
17598
+ return StringAppendBase(append_state.handle, segment, stats, data, offset, count);
17593
17599
  }
17594
17600
 
17595
17601
  template <bool DUPLICATE_ELIMINATE = false>
@@ -17598,7 +17604,12 @@ public:
17598
17604
  std::unordered_map<string, int32_t> *seen_strings = nullptr) {
17599
17605
  auto &buffer_manager = BufferManager::GetBufferManager(segment.db);
17600
17606
  auto handle = buffer_manager.Pin(segment.block);
17601
-
17607
+ return StringAppendBase(handle, segment, stats, data, offset, count);
17608
+ }
17609
+ template <bool DUPLICATE_ELIMINATE = false>
17610
+ static idx_t StringAppendBase(BufferHandle &handle, ColumnSegment &segment, SegmentStatistics &stats,
17611
+ UnifiedVectorFormat &data, idx_t offset, idx_t count,
17612
+ std::unordered_map<string, int32_t> *seen_strings = nullptr) {
17602
17613
  D_ASSERT(segment.GetBlockOffset() == 0);
17603
17614
  auto source_data = (string_t *)data.data;
17604
17615
  auto result_data = (int32_t *)(handle.Ptr() + DICTIONARY_HEADER_SIZE);
@@ -78452,10 +78463,12 @@ public:
78452
78463
  public:
78453
78464
  // Sink interface
78454
78465
  unique_ptr<GlobalSinkState> GetGlobalSinkState(ClientContext &context) const override;
78455
- void Combine(ExecutionContext &context, GlobalSinkState &gstate, LocalSinkState &lstate) const override;
78456
78466
  unique_ptr<LocalSinkState> GetLocalSinkState(ExecutionContext &context) const override;
78457
78467
  SinkResultType Sink(ExecutionContext &context, GlobalSinkState &state, LocalSinkState &lstate,
78458
78468
  DataChunk &input) const override;
78469
+ void Combine(ExecutionContext &context, GlobalSinkState &gstate, LocalSinkState &lstate) const override;
78470
+ SinkFinalizeType Finalize(Pipeline &pipeline, Event &event, ClientContext &context,
78471
+ GlobalSinkState &gstate) const override;
78459
78472
 
78460
78473
  bool IsSink() const override {
78461
78474
  return true;
@@ -78478,17 +78491,27 @@ public:
78478
78491
 
78479
78492
  namespace duckdb {
78480
78493
 
78494
+ PhysicalInsert::PhysicalInsert(vector<LogicalType> types, TableCatalogEntry *table, vector<idx_t> column_index_map,
78495
+ vector<unique_ptr<Expression>> bound_defaults, idx_t estimated_cardinality,
78496
+ bool return_chunk)
78497
+ : PhysicalOperator(PhysicalOperatorType::INSERT, move(types), estimated_cardinality),
78498
+ column_index_map(std::move(column_index_map)), table(table), bound_defaults(move(bound_defaults)),
78499
+ return_chunk(return_chunk) {
78500
+ }
78501
+
78481
78502
  //===--------------------------------------------------------------------===//
78482
78503
  // Sink
78483
78504
  //===--------------------------------------------------------------------===//
78484
78505
  class InsertGlobalState : public GlobalSinkState {
78485
78506
  public:
78486
78507
  explicit InsertGlobalState(ClientContext &context, const vector<LogicalType> &return_types)
78487
- : insert_count(0), return_collection(context, return_types) {
78508
+ : insert_count(0), initialized(false), return_collection(context, return_types) {
78488
78509
  }
78489
78510
 
78490
78511
  mutex lock;
78491
78512
  idx_t insert_count;
78513
+ LocalAppendState append_state;
78514
+ bool initialized;
78492
78515
  ColumnDataCollection return_collection;
78493
78516
  };
78494
78517
 
@@ -78504,12 +78527,12 @@ public:
78504
78527
  ExpressionExecutor default_executor;
78505
78528
  };
78506
78529
 
78507
- PhysicalInsert::PhysicalInsert(vector<LogicalType> types, TableCatalogEntry *table, vector<idx_t> column_index_map,
78508
- vector<unique_ptr<Expression>> bound_defaults, idx_t estimated_cardinality,
78509
- bool return_chunk)
78510
- : PhysicalOperator(PhysicalOperatorType::INSERT, move(types), estimated_cardinality),
78511
- column_index_map(std::move(column_index_map)), table(table), bound_defaults(move(bound_defaults)),
78512
- return_chunk(return_chunk) {
78530
+ unique_ptr<GlobalSinkState> PhysicalInsert::GetGlobalSinkState(ClientContext &context) const {
78531
+ return make_unique<InsertGlobalState>(context, GetTypes());
78532
+ }
78533
+
78534
+ unique_ptr<LocalSinkState> PhysicalInsert::GetLocalSinkState(ExecutionContext &context) const {
78535
+ return make_unique<InsertLocalState>(Allocator::Get(context.client), table->GetTypes(), bound_defaults);
78513
78536
  }
78514
78537
 
78515
78538
  SinkResultType PhysicalInsert::Sink(ExecutionContext &context, GlobalSinkState &state, LocalSinkState &lstate,
@@ -78550,7 +78573,11 @@ SinkResultType PhysicalInsert::Sink(ExecutionContext &context, GlobalSinkState &
78550
78573
  }
78551
78574
 
78552
78575
  lock_guard<mutex> glock(gstate.lock);
78553
- table->storage->Append(*table, context.client, istate.insert_chunk);
78576
+ if (!gstate.initialized) {
78577
+ table->storage->InitializeLocalAppend(gstate.append_state, context.client);
78578
+ gstate.initialized = true;
78579
+ }
78580
+ table->storage->LocalAppend(gstate.append_state, *table, context.client, istate.insert_chunk);
78554
78581
 
78555
78582
  if (return_chunk) {
78556
78583
  gstate.return_collection.Append(istate.insert_chunk);
@@ -78560,14 +78587,6 @@ SinkResultType PhysicalInsert::Sink(ExecutionContext &context, GlobalSinkState &
78560
78587
  return SinkResultType::NEED_MORE_INPUT;
78561
78588
  }
78562
78589
 
78563
- unique_ptr<GlobalSinkState> PhysicalInsert::GetGlobalSinkState(ClientContext &context) const {
78564
- return make_unique<InsertGlobalState>(context, GetTypes());
78565
- }
78566
-
78567
- unique_ptr<LocalSinkState> PhysicalInsert::GetLocalSinkState(ExecutionContext &context) const {
78568
- return make_unique<InsertLocalState>(Allocator::Get(context.client), table->GetTypes(), bound_defaults);
78569
- }
78570
-
78571
78590
  void PhysicalInsert::Combine(ExecutionContext &context, GlobalSinkState &gstate, LocalSinkState &lstate) const {
78572
78591
  auto &state = (InsertLocalState &)lstate;
78573
78592
  auto &client_profiler = QueryProfiler::Get(context.client);
@@ -78575,6 +78594,15 @@ void PhysicalInsert::Combine(ExecutionContext &context, GlobalSinkState &gstate,
78575
78594
  client_profiler.Flush(context.thread.profiler);
78576
78595
  }
78577
78596
 
78597
+ SinkFinalizeType PhysicalInsert::Finalize(Pipeline &pipeline, Event &event, ClientContext &context,
78598
+ GlobalSinkState &state) const {
78599
+ auto &gstate = (InsertGlobalState &)state;
78600
+ if (gstate.initialized) {
78601
+ table->storage->FinalizeLocalAppend(gstate.append_state);
78602
+ }
78603
+ return SinkFinalizeType::READY;
78604
+ }
78605
+
78578
78606
  //===--------------------------------------------------------------------===//
78579
78607
  // Source
78580
78608
  //===--------------------------------------------------------------------===//
@@ -78775,7 +78803,7 @@ SinkResultType PhysicalUpdate::Sink(ExecutionContext &context, GlobalSinkState &
78775
78803
  for (idx_t i = 0; i < columns.size(); i++) {
78776
78804
  mock_chunk.data[columns[i]].Reference(update_chunk.data[i]);
78777
78805
  }
78778
- table.Append(tableref, context.client, mock_chunk);
78806
+ table.LocalAppend(tableref, context.client, mock_chunk);
78779
78807
  } else {
78780
78808
  if (return_chunk) {
78781
78809
  mock_chunk.SetCardinality(update_chunk);
@@ -80430,9 +80458,10 @@ public:
80430
80458
 
80431
80459
  public:
80432
80460
  unique_ptr<GlobalSinkState> GetGlobalSinkState(ClientContext &context) const override;
80433
-
80434
80461
  SinkResultType Sink(ExecutionContext &context, GlobalSinkState &state, LocalSinkState &lstate,
80435
80462
  DataChunk &input) const override;
80463
+ SinkFinalizeType Finalize(Pipeline &pipeline, Event &event, ClientContext &context,
80464
+ GlobalSinkState &gstate) const override;
80436
80465
 
80437
80466
  bool IsSink() const override {
80438
80467
  return true;
@@ -80461,13 +80490,14 @@ PhysicalCreateTableAs::PhysicalCreateTableAs(LogicalOperator &op, SchemaCatalogE
80461
80490
  //===--------------------------------------------------------------------===//
80462
80491
  class CreateTableAsGlobalState : public GlobalSinkState {
80463
80492
  public:
80464
- CreateTableAsGlobalState() {
80465
- inserted_count = 0;
80493
+ CreateTableAsGlobalState() : inserted_count(0), initialized(false) {
80466
80494
  }
80467
80495
 
80468
80496
  mutex append_lock;
80469
80497
  TableCatalogEntry *table;
80470
80498
  int64_t inserted_count;
80499
+ LocalAppendState append_state;
80500
+ bool initialized;
80471
80501
  };
80472
80502
 
80473
80503
  unique_ptr<GlobalSinkState> PhysicalCreateTableAs::GetGlobalSinkState(ClientContext &context) const {
@@ -80482,11 +80512,24 @@ SinkResultType PhysicalCreateTableAs::Sink(ExecutionContext &context, GlobalSink
80482
80512
  auto &sink = (CreateTableAsGlobalState &)state;
80483
80513
  D_ASSERT(sink.table);
80484
80514
  lock_guard<mutex> client_guard(sink.append_lock);
80485
- sink.table->storage->Append(*sink.table, context.client, input);
80515
+ if (!sink.initialized) {
80516
+ sink.table->storage->InitializeLocalAppend(sink.append_state, context.client);
80517
+ sink.initialized = true;
80518
+ }
80519
+ sink.table->storage->LocalAppend(sink.append_state, *sink.table, context.client, input);
80486
80520
  sink.inserted_count += input.size();
80487
80521
  return SinkResultType::NEED_MORE_INPUT;
80488
80522
  }
80489
80523
 
80524
+ SinkFinalizeType PhysicalCreateTableAs::Finalize(Pipeline &pipeline, Event &event, ClientContext &context,
80525
+ GlobalSinkState &state) const {
80526
+ auto &gstate = (CreateTableAsGlobalState &)state;
80527
+ if (gstate.initialized) {
80528
+ gstate.table->storage->FinalizeLocalAppend(gstate.append_state);
80529
+ }
80530
+ return SinkFinalizeType::READY;
80531
+ }
80532
+
80490
80533
  //===--------------------------------------------------------------------===//
80491
80534
  // Source
80492
80535
  //===--------------------------------------------------------------------===//
@@ -125062,9 +125105,7 @@ void Appender::FlushInternal(ColumnDataCollection &collection) {
125062
125105
  }
125063
125106
 
125064
125107
  void InternalAppender::FlushInternal(ColumnDataCollection &collection) {
125065
- for (auto &chunk : collection.Chunks()) {
125066
- table.storage->Append(table, context, chunk);
125067
- }
125108
+ table.storage->LocalAppend(table, context, collection);
125068
125109
  }
125069
125110
 
125070
125111
  void BaseAppender::Close() {
@@ -129455,9 +129496,7 @@ void ClientContext::Append(TableDescription &description, ColumnDataCollection &
129455
129496
  throw Exception("Failed to append: table entry has different number of columns!");
129456
129497
  }
129457
129498
  }
129458
- for (auto &chunk : collection.Chunks()) {
129459
- table_entry->storage->Append(*table_entry, *this, chunk);
129460
- }
129499
+ table_entry->storage->LocalAppend(*table_entry, *this, collection);
129461
129500
  });
129462
129501
  }
129463
129502
 
@@ -130935,7 +130974,7 @@ DBConfig::DBConfig(std::unordered_map<string, string> &config_dict, bool read_on
130935
130974
  DBConfig::~DBConfig() {
130936
130975
  }
130937
130976
 
130938
- DatabaseInstance::DatabaseInstance() {
130977
+ DatabaseInstance::DatabaseInstance() : is_invalidated(false) {
130939
130978
  }
130940
130979
 
130941
130980
  DatabaseInstance::~DatabaseInstance() {
@@ -198390,6 +198429,18 @@ idx_t FixedSizeFinalAnalyze(AnalyzeState &state_p) {
198390
198429
  //===--------------------------------------------------------------------===//
198391
198430
  // Compress
198392
198431
  //===--------------------------------------------------------------------===//
198432
+ struct UncompressedCompressState : public CompressionState {
198433
+ explicit UncompressedCompressState(ColumnDataCheckpointer &checkpointer);
198434
+
198435
+ ColumnDataCheckpointer &checkpointer;
198436
+ unique_ptr<ColumnSegment> current_segment;
198437
+ ColumnAppendState append_state;
198438
+
198439
+ virtual void CreateEmptySegment(idx_t row_start);
198440
+ void FlushSegment(idx_t segment_size);
198441
+ void Finalize(idx_t segment_size);
198442
+ };
198443
+
198393
198444
  UncompressedCompressState::UncompressedCompressState(ColumnDataCheckpointer &checkpointer)
198394
198445
  : checkpointer(checkpointer) {
198395
198446
  CreateEmptySegment(checkpointer.GetRowGroup().start);
@@ -198404,6 +198455,7 @@ void UncompressedCompressState::CreateEmptySegment(idx_t row_start) {
198404
198455
  state.overflow_writer = make_unique<WriteOverflowStringsToDisk>(checkpointer.GetColumnData().block_manager);
198405
198456
  }
198406
198457
  current_segment = move(compressed_segment);
198458
+ current_segment->InitializeAppend(append_state);
198407
198459
  }
198408
198460
 
198409
198461
  void UncompressedCompressState::FlushSegment(idx_t segment_size) {
@@ -198426,17 +198478,16 @@ void UncompressedFunctions::Compress(CompressionState &state_p, Vector &data, id
198426
198478
  UnifiedVectorFormat vdata;
198427
198479
  data.ToUnifiedFormat(count, vdata);
198428
198480
 
198429
- ColumnAppendState append_state;
198430
198481
  idx_t offset = 0;
198431
198482
  while (count > 0) {
198432
- idx_t appended = state.current_segment->Append(append_state, vdata, offset, count);
198483
+ idx_t appended = state.current_segment->Append(state.append_state, vdata, offset, count);
198433
198484
  if (appended == count) {
198434
198485
  // appended everything: finished
198435
198486
  return;
198436
198487
  }
198437
198488
  auto next_start = state.current_segment->start + state.current_segment->count;
198438
198489
  // the segment is full: flush it to disk
198439
- state.FlushSegment(state.current_segment->FinalizeAppend());
198490
+ state.FlushSegment(state.current_segment->FinalizeAppend(state.append_state));
198440
198491
 
198441
198492
  // now create a new segment and continue appending
198442
198493
  state.CreateEmptySegment(next_start);
@@ -198447,7 +198498,7 @@ void UncompressedFunctions::Compress(CompressionState &state_p, Vector &data, id
198447
198498
 
198448
198499
  void UncompressedFunctions::FinalizeCompress(CompressionState &state_p) {
198449
198500
  auto &state = (UncompressedCompressState &)state_p;
198450
- state.Finalize(state.current_segment->FinalizeAppend());
198501
+ state.Finalize(state.current_segment->FinalizeAppend(state.append_state));
198451
198502
  }
198452
198503
 
198453
198504
  //===--------------------------------------------------------------------===//
@@ -198517,6 +198568,12 @@ void FixedSizeFetchRow(ColumnSegment &segment, ColumnFetchState &state, row_t ro
198517
198568
  //===--------------------------------------------------------------------===//
198518
198569
  // Append
198519
198570
  //===--------------------------------------------------------------------===//
198571
+ static unique_ptr<CompressionAppendState> FixedSizeInitAppend(ColumnSegment &segment) {
198572
+ auto &buffer_manager = BufferManager::GetBufferManager(segment.db);
198573
+ auto handle = buffer_manager.Pin(segment.block);
198574
+ return make_unique<CompressionAppendState>(move(handle));
198575
+ }
198576
+
198520
198577
  template <class T>
198521
198578
  static void AppendLoop(SegmentStatistics &stats, data_ptr_t target, idx_t target_offset, UnifiedVectorFormat &adata,
198522
198579
  idx_t offset, idx_t count) {
@@ -198559,13 +198616,11 @@ void AppendLoop<list_entry_t>(SegmentStatistics &stats, data_ptr_t target, idx_t
198559
198616
  }
198560
198617
 
198561
198618
  template <class T>
198562
- idx_t FixedSizeAppend(ColumnSegment &segment, SegmentStatistics &stats, UnifiedVectorFormat &data, idx_t offset,
198563
- idx_t count) {
198564
- auto &buffer_manager = BufferManager::GetBufferManager(segment.db);
198565
- auto handle = buffer_manager.Pin(segment.block);
198619
+ idx_t FixedSizeAppend(CompressionAppendState &append_state, ColumnSegment &segment, SegmentStatistics &stats,
198620
+ UnifiedVectorFormat &data, idx_t offset, idx_t count) {
198566
198621
  D_ASSERT(segment.GetBlockOffset() == 0);
198567
198622
 
198568
- auto target_ptr = handle.Ptr();
198623
+ auto target_ptr = append_state.handle.Ptr();
198569
198624
  idx_t max_tuple_count = Storage::BLOCK_SIZE / sizeof(T);
198570
198625
  idx_t copy_count = MinValue<idx_t>(count, max_tuple_count - segment.count);
198571
198626
 
@@ -198588,7 +198643,7 @@ CompressionFunction FixedSizeGetFunction(PhysicalType data_type) {
198588
198643
  FixedSizeAnalyze, FixedSizeFinalAnalyze<T>, UncompressedFunctions::InitCompression,
198589
198644
  UncompressedFunctions::Compress, UncompressedFunctions::FinalizeCompress,
198590
198645
  FixedSizeInitScan, FixedSizeScan<T>, FixedSizeScanPartial<T>, FixedSizeFetchRow<T>,
198591
- UncompressedFunctions::EmptySkip, nullptr, FixedSizeAppend<T>,
198646
+ UncompressedFunctions::EmptySkip, nullptr, FixedSizeInitAppend, FixedSizeAppend<T>,
198592
198647
  FixedSizeFinalizeAppend<T>, nullptr);
198593
198648
  }
198594
198649
 
@@ -200114,7 +200169,8 @@ CompressionFunction StringUncompressed::GetFunction(PhysicalType data_type) {
200114
200169
  UncompressedStringStorage::StringInitScan, UncompressedStringStorage::StringScan,
200115
200170
  UncompressedStringStorage::StringScanPartial, UncompressedStringStorage::StringFetchRow,
200116
200171
  UncompressedFunctions::EmptySkip, UncompressedStringStorage::StringInitSegment,
200117
- UncompressedStringStorage::StringAppend, UncompressedStringStorage::FinalizeAppend);
200172
+ UncompressedStringStorage::StringInitAppend, UncompressedStringStorage::StringAppend,
200173
+ UncompressedStringStorage::FinalizeAppend);
200118
200174
  }
200119
200175
 
200120
200176
  //===--------------------------------------------------------------------===//
@@ -200758,6 +200814,12 @@ void ValidityFetchRow(ColumnSegment &segment, ColumnFetchState &state, row_t row
200758
200814
  //===--------------------------------------------------------------------===//
200759
200815
  // Append
200760
200816
  //===--------------------------------------------------------------------===//
200817
+ static unique_ptr<CompressionAppendState> ValidityInitAppend(ColumnSegment &segment) {
200818
+ auto &buffer_manager = BufferManager::GetBufferManager(segment.db);
200819
+ auto handle = buffer_manager.Pin(segment.block);
200820
+ return make_unique<CompressionAppendState>(move(handle));
200821
+ }
200822
+
200761
200823
  unique_ptr<CompressedSegmentState> ValidityInitSegment(ColumnSegment &segment, block_id_t block_id) {
200762
200824
  auto &buffer_manager = BufferManager::GetBufferManager(segment.db);
200763
200825
  if (block_id == INVALID_BLOCK) {
@@ -200767,8 +200829,8 @@ unique_ptr<CompressedSegmentState> ValidityInitSegment(ColumnSegment &segment, b
200767
200829
  return nullptr;
200768
200830
  }
200769
200831
 
200770
- idx_t ValidityAppend(ColumnSegment &segment, SegmentStatistics &stats, UnifiedVectorFormat &data, idx_t offset,
200771
- idx_t vcount) {
200832
+ idx_t ValidityAppend(CompressionAppendState &append_state, ColumnSegment &segment, SegmentStatistics &stats,
200833
+ UnifiedVectorFormat &data, idx_t offset, idx_t vcount) {
200772
200834
  D_ASSERT(segment.GetBlockOffset() == 0);
200773
200835
  auto &validity_stats = (ValidityStatistics &)*stats.statistics;
200774
200836
 
@@ -200780,10 +200842,8 @@ idx_t ValidityAppend(ColumnSegment &segment, SegmentStatistics &stats, UnifiedVe
200780
200842
  validity_stats.has_no_null = true;
200781
200843
  return append_count;
200782
200844
  }
200783
- auto &buffer_manager = BufferManager::GetBufferManager(segment.db);
200784
- auto handle = buffer_manager.Pin(segment.block);
200785
200845
 
200786
- ValidityMask mask((validity_t *)handle.Ptr());
200846
+ ValidityMask mask((validity_t *)append_state.handle.Ptr());
200787
200847
  for (idx_t i = 0; i < append_count; i++) {
200788
200848
  auto idx = data.sel->get_index(offset + i);
200789
200849
  if (!data.validity.RowIsValidUnsafe(idx)) {
@@ -200833,8 +200893,8 @@ CompressionFunction ValidityUncompressed::GetFunction(PhysicalType data_type) {
200833
200893
  ValidityAnalyze, ValidityFinalAnalyze, UncompressedFunctions::InitCompression,
200834
200894
  UncompressedFunctions::Compress, UncompressedFunctions::FinalizeCompress,
200835
200895
  ValidityInitScan, ValidityScan, ValidityScanPartial, ValidityFetchRow,
200836
- UncompressedFunctions::EmptySkip, ValidityInitSegment, ValidityAppend,
200837
- ValidityFinalizeAppend, ValidityRevertAppend);
200896
+ UncompressedFunctions::EmptySkip, ValidityInitSegment, ValidityInitAppend,
200897
+ ValidityAppend, ValidityFinalizeAppend, ValidityRevertAppend);
200838
200898
  }
200839
200899
 
200840
200900
  } // namespace duckdb
@@ -201408,14 +201468,20 @@ void DataTable::VerifyAppendConstraints(TableCatalogEntry &table, ClientContext
201408
201468
  }
201409
201469
  }
201410
201470
 
201411
- void DataTable::Append(TableCatalogEntry &table, ClientContext &context, DataChunk &chunk) {
201471
+ void DataTable::InitializeLocalAppend(LocalAppendState &state, ClientContext &context) {
201472
+ if (!is_root) {
201473
+ throw TransactionException("Transaction conflict: adding entries to a table that has been altered!");
201474
+ }
201475
+ auto &transaction = Transaction::GetTransaction(context);
201476
+ transaction.storage.InitializeAppend(state, this);
201477
+ }
201478
+
201479
+ void DataTable::LocalAppend(LocalAppendState &state, TableCatalogEntry &table, ClientContext &context,
201480
+ DataChunk &chunk) {
201412
201481
  if (chunk.size() == 0) {
201413
201482
  return;
201414
201483
  }
201415
- // FIXME: could be an assertion instead?
201416
- if (chunk.ColumnCount() != table.StandardColumnCount()) {
201417
- throw InternalException("Mismatch in column count for append");
201418
- }
201484
+ D_ASSERT(chunk.ColumnCount() == table.StandardColumnCount());
201419
201485
  if (!is_root) {
201420
201486
  throw TransactionException("Transaction conflict: adding entries to a table that has been altered!");
201421
201487
  }
@@ -201426,8 +201492,27 @@ void DataTable::Append(TableCatalogEntry &table, ClientContext &context, DataChu
201426
201492
  VerifyAppendConstraints(table, context, chunk);
201427
201493
 
201428
201494
  // append to the transaction local data
201429
- auto &transaction = Transaction::GetTransaction(context);
201430
- transaction.storage.Append(this, chunk);
201495
+ LocalStorage::Append(state, chunk);
201496
+ }
201497
+
201498
+ void DataTable::FinalizeLocalAppend(LocalAppendState &state) {
201499
+ LocalStorage::FinalizeAppend(state);
201500
+ }
201501
+
201502
+ void DataTable::LocalAppend(TableCatalogEntry &table, ClientContext &context, DataChunk &chunk) {
201503
+ LocalAppendState append_state;
201504
+ table.storage->InitializeLocalAppend(append_state, context);
201505
+ table.storage->LocalAppend(append_state, table, context, chunk);
201506
+ table.storage->FinalizeLocalAppend(append_state);
201507
+ }
201508
+
201509
+ void DataTable::LocalAppend(TableCatalogEntry &table, ClientContext &context, ColumnDataCollection &collection) {
201510
+ LocalAppendState append_state;
201511
+ table.storage->InitializeLocalAppend(append_state, context);
201512
+ for (auto &chunk : collection.Chunks()) {
201513
+ table.storage->LocalAppend(append_state, table, context, chunk);
201514
+ }
201515
+ table.storage->FinalizeLocalAppend(append_state);
201431
201516
  }
201432
201517
 
201433
201518
  void DataTable::AppendLock(TableAppendState &state) {
@@ -201446,9 +201531,9 @@ void DataTable::InitializeAppend(Transaction &transaction, TableAppendState &sta
201446
201531
  row_groups->InitializeAppend(transaction, state, append_count);
201447
201532
  }
201448
201533
 
201449
- void DataTable::Append(Transaction &transaction, DataChunk &chunk, TableAppendState &state) {
201534
+ void DataTable::Append(DataChunk &chunk, TableAppendState &state) {
201450
201535
  D_ASSERT(is_root);
201451
- row_groups->Append(transaction, chunk, state, stats);
201536
+ row_groups->Append(chunk, state, stats);
201452
201537
  }
201453
201538
 
201454
201539
  void DataTable::ScanTableSegment(idx_t row_start, idx_t count, const std::function<void(DataChunk &chunk)> &function) {
@@ -202097,27 +202182,33 @@ bool LocalStorage::NextParallelScan(ClientContext &context, DataTable *table, Pa
202097
202182
  return storage->row_groups->NextParallelScan(context, state, scan_state);
202098
202183
  }
202099
202184
 
202100
- void LocalStorage::Append(DataTable *table, DataChunk &chunk) {
202185
+ void LocalStorage::InitializeAppend(LocalAppendState &state, DataTable *table) {
202101
202186
  auto entry = table_storage.find(table);
202102
- LocalTableStorage *storage;
202103
202187
  if (entry == table_storage.end()) {
202104
202188
  auto new_storage = make_shared<LocalTableStorage>(*table);
202105
- storage = new_storage.get();
202189
+ state.storage = new_storage.get();
202106
202190
  table_storage.insert(make_pair(table, move(new_storage)));
202107
202191
  } else {
202108
- storage = entry->second.get();
202192
+ state.storage = entry->second.get();
202109
202193
  }
202194
+ state.storage->row_groups->InitializeAppend(state.append_state);
202195
+ }
202196
+
202197
+ void LocalStorage::Append(LocalAppendState &state, DataChunk &chunk) {
202110
202198
  // append to unique indices (if any)
202199
+ auto storage = state.storage;
202111
202200
  idx_t base_id = MAX_ROW_ID + storage->row_groups->GetTotalRows();
202112
202201
  if (!DataTable::AppendToIndexes(storage->indexes, chunk, base_id)) {
202113
202202
  throw ConstraintException("PRIMARY KEY or UNIQUE constraint violated: duplicated key");
202114
202203
  }
202115
202204
 
202116
202205
  //! Append to the chunk
202117
- TableAppendState state;
202206
+ storage->row_groups->Append(chunk, state.append_state, storage->stats);
202207
+ }
202208
+
202209
+ void LocalStorage::FinalizeAppend(LocalAppendState &state) {
202118
202210
  TransactionData transaction_data(0, 0);
202119
- storage->row_groups->InitializeAppend(transaction_data, state, chunk.size());
202120
- storage->row_groups->Append(transaction_data, chunk, state, storage->stats);
202211
+ state.storage->row_groups->FinalizeAppend(transaction_data, state.append_state);
202121
202212
  }
202122
202213
 
202123
202214
  LocalTableStorage *LocalStorage::GetStorage(DataTable *table) {
@@ -202213,7 +202304,7 @@ void LocalStorage::Flush(DataTable &table, LocalTableStorage &storage) {
202213
202304
  return false;
202214
202305
  }
202215
202306
  // append to base table
202216
- table.Append(transaction, chunk, append_state);
202307
+ table.Append(chunk, append_state);
202217
202308
  return true;
202218
202309
  });
202219
202310
  if (constraint_violated) {
@@ -202238,8 +202329,7 @@ void LocalStorage::Flush(DataTable &table, LocalTableStorage &storage) {
202238
202329
  transaction.PushAppend(&table, append_state.row_start, append_count);
202239
202330
  }
202240
202331
 
202241
- void LocalStorage::Commit(LocalStorage::CommitState &commit_state, Transaction &transaction, WriteAheadLog *log,
202242
- transaction_t commit_id) {
202332
+ void LocalStorage::Commit(LocalStorage::CommitState &commit_state, Transaction &transaction) {
202243
202333
  // commit local storage, iterate over all entries in the table storage map
202244
202334
  for (auto &entry : table_storage) {
202245
202335
  auto table = entry.first;
@@ -206001,27 +206091,33 @@ void ColumnSegment::FetchRow(ColumnFetchState &state, row_t row_id, Vector &resu
206001
206091
  function->fetch_row(*this, state, row_id - this->start, result, result_idx);
206002
206092
  }
206003
206093
 
206094
+ //===--------------------------------------------------------------------===//
206095
+ // Append
206096
+ //===--------------------------------------------------------------------===//
206004
206097
  void ColumnSegment::InitializeAppend(ColumnAppendState &state) {
206005
206098
  D_ASSERT(segment_type == ColumnSegmentType::TRANSIENT);
206099
+ if (!function->init_append) {
206100
+ throw InternalException("Attempting to init append to a segment without init_append method");
206101
+ }
206102
+ state.append_state = function->init_append(*this);
206006
206103
  }
206007
206104
 
206008
- //===--------------------------------------------------------------------===//
206009
- // Append
206010
- //===--------------------------------------------------------------------===//
206011
206105
  idx_t ColumnSegment::Append(ColumnAppendState &state, UnifiedVectorFormat &append_data, idx_t offset, idx_t count) {
206012
206106
  D_ASSERT(segment_type == ColumnSegmentType::TRANSIENT);
206013
206107
  if (!function->append) {
206014
206108
  throw InternalException("Attempting to append to a segment without append method");
206015
206109
  }
206016
- return function->append(*this, stats, append_data, offset, count);
206110
+ return function->append(*state.append_state, *this, stats, append_data, offset, count);
206017
206111
  }
206018
206112
 
206019
- idx_t ColumnSegment::FinalizeAppend() {
206113
+ idx_t ColumnSegment::FinalizeAppend(ColumnAppendState &state) {
206020
206114
  D_ASSERT(segment_type == ColumnSegmentType::TRANSIENT);
206021
206115
  if (!function->finalize_append) {
206022
206116
  throw InternalException("Attempting to call FinalizeAppend on a segment without a finalize_append method");
206023
206117
  }
206024
- return function->finalize_append(*this, stats);
206118
+ auto result_count = function->finalize_append(*this, stats);
206119
+ state.append_state.reset();
206120
+ return result_count;
206025
206121
  }
206026
206122
 
206027
206123
  void ColumnSegment::RevertAppend(idx_t start_row) {
@@ -207272,14 +207368,14 @@ void RowGroup::FetchRow(TransactionData transaction, ColumnFetchState &state, co
207272
207368
  }
207273
207369
  }
207274
207370
 
207275
- void RowGroup::AppendVersionInfo(TransactionData transaction, idx_t row_group_start, idx_t count,
207276
- transaction_t commit_id) {
207371
+ void RowGroup::AppendVersionInfo(TransactionData transaction, idx_t count) {
207372
+ idx_t row_group_start = this->count.load();
207277
207373
  idx_t row_group_end = row_group_start + count;
207374
+ if (row_group_end > RowGroup::ROW_GROUP_SIZE) {
207375
+ row_group_end = RowGroup::ROW_GROUP_SIZE;
207376
+ }
207278
207377
  lock_guard<mutex> lock(row_group_lock);
207279
207378
 
207280
- this->count += count;
207281
- D_ASSERT(this->count <= RowGroup::ROW_GROUP_SIZE);
207282
-
207283
207379
  // create the version_info if it doesn't exist yet
207284
207380
  if (!version_info) {
207285
207381
  version_info = make_unique<VersionNode>();
@@ -207293,7 +207389,7 @@ void RowGroup::AppendVersionInfo(TransactionData transaction, idx_t row_group_st
207293
207389
  if (start == 0 && end == STANDARD_VECTOR_SIZE) {
207294
207390
  // entire vector is encapsulated by append: append a single constant
207295
207391
  auto constant_info = make_unique<ChunkConstantInfo>(this->start + vector_idx * STANDARD_VECTOR_SIZE);
207296
- constant_info->insert_id = commit_id;
207392
+ constant_info->insert_id = transaction.transaction_id;
207297
207393
  constant_info->delete_id = NOT_DELETED_ID;
207298
207394
  version_info->info[vector_idx] = move(constant_info);
207299
207395
  } else {
@@ -207309,9 +207405,10 @@ void RowGroup::AppendVersionInfo(TransactionData transaction, idx_t row_group_st
207309
207405
  // use existing vector
207310
207406
  info = (ChunkVectorInfo *)version_info->info[vector_idx].get();
207311
207407
  }
207312
- info->Append(start, end, commit_id);
207408
+ info->Append(start, end, transaction.transaction_id);
207313
207409
  }
207314
207410
  }
207411
+ this->count = row_group_end;
207315
207412
  }
207316
207413
 
207317
207414
  void RowGroup::CommitAppend(transaction_t commit_id, idx_t row_group_start, idx_t count) {
@@ -207347,8 +207444,7 @@ void RowGroup::RevertAppend(idx_t row_group_start) {
207347
207444
  Verify();
207348
207445
  }
207349
207446
 
207350
- void RowGroup::InitializeAppend(TransactionData transaction, RowGroupAppendState &append_state,
207351
- idx_t remaining_append_count) {
207447
+ void RowGroup::InitializeAppend(RowGroupAppendState &append_state) {
207352
207448
  append_state.row_group = this;
207353
207449
  append_state.offset_in_row_group = this->count;
207354
207450
  // for each column, initialize the append state
@@ -207356,9 +207452,6 @@ void RowGroup::InitializeAppend(TransactionData transaction, RowGroupAppendState
207356
207452
  for (idx_t i = 0; i < columns.size(); i++) {
207357
207453
  columns[i]->InitializeAppend(append_state.states[i]);
207358
207454
  }
207359
- // append the version info for this row_group
207360
- idx_t append_count = MinValue<idx_t>(remaining_append_count, RowGroup::ROW_GROUP_SIZE - this->count);
207361
- AppendVersionInfo(transaction, this->count, append_count, transaction.transaction_id);
207362
207455
  }
207363
207456
 
207364
207457
  void RowGroup::Append(RowGroupAppendState &state, DataChunk &chunk, idx_t append_count) {
@@ -207730,10 +207823,14 @@ void RowGroupCollection::AppendRowGroup(idx_t start_row) {
207730
207823
 
207731
207824
  void RowGroupCollection::Verify() {
207732
207825
  #ifdef DEBUG
207826
+ idx_t current_total_rows = 0;
207733
207827
  for (auto segment = row_groups->GetRootSegment(); segment; segment = segment->next.get()) {
207734
207828
  auto &row_group = (RowGroup &)*segment;
207735
207829
  row_group.Verify();
207830
+ D_ASSERT(row_group.start == this->row_start + current_total_rows);
207831
+ current_total_rows += row_group.count;
207736
207832
  }
207833
+ D_ASSERT(current_total_rows == total_rows.load());
207737
207834
  #endif
207738
207835
  }
207739
207836
 
@@ -207835,14 +207932,22 @@ void RowGroupCollection::Fetch(TransactionData transaction, DataChunk &result, c
207835
207932
  //===--------------------------------------------------------------------===//
207836
207933
  // Append
207837
207934
  //===--------------------------------------------------------------------===//
207935
+ TableAppendState::TableAppendState()
207936
+ : row_group_append_state(*this), total_append_count(0), start_row_group(nullptr), transaction(0, 0), remaining(0) {
207937
+ }
207938
+
207939
+ TableAppendState::~TableAppendState() {
207940
+ D_ASSERT(Exception::UncaughtException() || remaining == 0);
207941
+ }
207942
+
207838
207943
  bool RowGroupCollection::IsEmpty() const {
207839
207944
  return row_groups->GetRootSegment() == nullptr;
207840
207945
  }
207841
207946
 
207842
207947
  void RowGroupCollection::InitializeAppend(TransactionData transaction, TableAppendState &state, idx_t append_count) {
207843
- state.remaining_append_count = append_count;
207844
207948
  state.row_start = total_rows;
207845
207949
  state.current_row = state.row_start;
207950
+ state.total_append_count = 0;
207846
207951
 
207847
207952
  // start writing to the row_groups
207848
207953
  lock_guard<mutex> row_group_lock(row_groups->node_lock);
@@ -207850,19 +207955,29 @@ void RowGroupCollection::InitializeAppend(TransactionData transaction, TableAppe
207850
207955
  // empty row group collection: empty first row group
207851
207956
  AppendRowGroup(row_start);
207852
207957
  }
207853
- auto last_row_group = (RowGroup *)row_groups->GetLastSegment();
207854
- D_ASSERT(this->row_start + total_rows == last_row_group->start + last_row_group->count);
207855
- last_row_group->InitializeAppend(transaction, state.row_group_append_state, state.remaining_append_count);
207856
- total_rows += append_count;
207958
+ state.start_row_group = (RowGroup *)row_groups->GetLastSegment();
207959
+ D_ASSERT(this->row_start + total_rows == state.start_row_group->start + state.start_row_group->count);
207960
+ state.start_row_group->InitializeAppend(state.row_group_append_state);
207961
+ state.remaining = append_count;
207962
+ if (state.remaining > 0) {
207963
+ state.transaction = transaction;
207964
+ state.start_row_group->AppendVersionInfo(transaction, state.remaining);
207965
+ total_rows += state.remaining;
207966
+ }
207967
+ }
207968
+
207969
+ void RowGroupCollection::InitializeAppend(TableAppendState &state) {
207970
+ TransactionData tdata(0, 0);
207971
+ InitializeAppend(tdata, state, 0);
207857
207972
  }
207858
207973
 
207859
- void RowGroupCollection::Append(TransactionData transaction, DataChunk &chunk, TableAppendState &state,
207860
- TableStatistics &stats) {
207974
+ void RowGroupCollection::Append(DataChunk &chunk, TableAppendState &state, TableStatistics &stats) {
207861
207975
  D_ASSERT(chunk.ColumnCount() == types.size());
207862
207976
  chunk.Verify();
207863
207977
 
207864
207978
  idx_t append_count = chunk.size();
207865
207979
  idx_t remaining = chunk.size();
207980
+ state.total_append_count += append_count;
207866
207981
  while (true) {
207867
207982
  auto current_row_group = state.row_group_append_state.row_group;
207868
207983
  // check how much we can fit into the current row_group
@@ -207876,8 +207991,10 @@ void RowGroupCollection::Append(TransactionData transaction, DataChunk &chunk, T
207876
207991
  current_row_group->MergeIntoStatistics(i, *stats.GetStats(i).stats);
207877
207992
  }
207878
207993
  }
207879
- state.remaining_append_count -= append_count;
207880
207994
  remaining -= append_count;
207995
+ if (state.remaining > 0) {
207996
+ state.remaining -= append_count;
207997
+ }
207881
207998
  if (remaining > 0) {
207882
207999
  // we expect max 1 iteration of this loop (i.e. a single chunk should never overflow more than one
207883
208000
  // row_group)
@@ -207891,11 +208008,15 @@ void RowGroupCollection::Append(TransactionData transaction, DataChunk &chunk, T
207891
208008
  chunk.Slice(sel, remaining);
207892
208009
  }
207893
208010
  // append a new row_group
207894
- AppendRowGroup(current_row_group->start + current_row_group->count);
208011
+ auto next_start = current_row_group->start + state.row_group_append_state.offset_in_row_group;
208012
+ AppendRowGroup(next_start);
207895
208013
  // set up the append state for this row_group
207896
208014
  lock_guard<mutex> row_group_lock(row_groups->node_lock);
207897
208015
  auto last_row_group = (RowGroup *)row_groups->GetLastSegment();
207898
- last_row_group->InitializeAppend(transaction, state.row_group_append_state, state.remaining_append_count);
208016
+ last_row_group->InitializeAppend(state.row_group_append_state);
208017
+ if (state.remaining > 0) {
208018
+ last_row_group->AppendVersionInfo(state.transaction, state.remaining);
208019
+ }
207899
208020
  continue;
207900
208021
  } else {
207901
208022
  break;
@@ -207912,6 +208033,22 @@ void RowGroupCollection::Append(TransactionData transaction, DataChunk &chunk, T
207912
208033
  }
207913
208034
  }
207914
208035
 
208036
+ void RowGroupCollection::FinalizeAppend(TransactionData transaction, TableAppendState &state) {
208037
+ D_ASSERT(state.transaction.transaction_id == 0);
208038
+ auto remaining = state.total_append_count;
208039
+ auto row_group = state.start_row_group;
208040
+ while (remaining > 0) {
208041
+ auto append_count = MinValue<idx_t>(remaining, RowGroup::ROW_GROUP_SIZE - row_group->count);
208042
+ row_group->AppendVersionInfo(transaction, append_count);
208043
+ remaining -= append_count;
208044
+ row_group = (RowGroup *)row_group->next.get();
208045
+ }
208046
+ total_rows += state.total_append_count;
208047
+
208048
+ state.total_append_count = 0;
208049
+ state.start_row_group = nullptr;
208050
+ }
208051
+
207915
208052
  void RowGroupCollection::CommitAppend(transaction_t commit_id, idx_t row_start, idx_t count) {
207916
208053
  auto row_group = (RowGroup *)row_groups->GetSegment(row_start);
207917
208054
  D_ASSERT(row_group);
@@ -210929,7 +211066,7 @@ void ReplayState::ReplayInsert() {
210929
211066
  }
210930
211067
 
210931
211068
  // append to the current table
210932
- current_table->storage->Append(*current_table, context, chunk);
211069
+ current_table->storage->LocalAppend(*current_table, context, chunk);
210933
211070
  }
210934
211071
 
210935
211072
  void ReplayState::ReplayDelete() {
@@ -211901,6 +212038,13 @@ void RollbackState::RollbackEntry(UndoFlags type, data_ptr_t data) {
211901
212038
 
211902
212039
  namespace duckdb {
211903
212040
 
212041
+ TransactionData::TransactionData(Transaction &transaction_p) // NOLINT
212042
+ : transaction(&transaction_p), transaction_id(transaction_p.transaction_id), start_time(transaction_p.start_time) {
212043
+ }
212044
+ TransactionData::TransactionData(transaction_t transaction_id_p, transaction_t start_time_p)
212045
+ : transaction(nullptr), transaction_id(transaction_id_p), start_time(start_time_p) {
212046
+ }
212047
+
211904
212048
  Transaction::Transaction(weak_ptr<ClientContext> context_p, transaction_t start_time, transaction_t transaction_id,
211905
212049
  timestamp_t start_timestamp, idx_t catalog_version)
211906
212050
  : context(move(context_p)), start_time(start_time), transaction_id(transaction_id), commit_id(0),
@@ -211980,7 +212124,7 @@ string Transaction::Commit(DatabaseInstance &db, transaction_t commit_id, bool c
211980
212124
  LocalStorage::CommitState commit_state;
211981
212125
  auto storage_commit_state = storage_manager.GenStorageCommitState(*this, checkpoint);
211982
212126
  try {
211983
- storage.Commit(commit_state, *this, log, commit_id);
212127
+ storage.Commit(commit_state, *this);
211984
212128
  undo_buffer.Commit(iterator_state, log, commit_id);
211985
212129
  if (log) {
211986
212130
  // commit any sequences that were used to the WAL