github.com/rohankumardubey/aresdb@v0.0.2-0.20190517170215-e54e3ca06b9c/api/debug_handler.go (about) 1 // Copyright (c) 2017-2018 Uber Technologies, Inc. 2 // 3 // Licensed under the Apache License, Version 2.0 (the "License"); 4 // you may not use this file except in compliance with the License. 5 // You may obtain a copy of the License at 6 // 7 // http://www.apache.org/licenses/LICENSE-2.0 8 // 9 // Unless required by applicable law or agreed to in writing, software 10 // distributed under the License is distributed on an "AS IS" BASIS, 11 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 // See the License for the specific language governing permissions and 13 // limitations under the License. 14 15 package api 16 17 import ( 18 "encoding/json" 19 "errors" 20 "fmt" 21 "net/http" 22 "strconv" 23 "strings" 24 25 "github.com/gorilla/mux" 26 "github.com/uber/aresdb/memstore" 27 memCom "github.com/uber/aresdb/memstore/common" 28 "github.com/uber/aresdb/metastore" 29 "github.com/uber/aresdb/utils" 30 "io" 31 ) 32 33 // DebugHandler handles debug operations. 34 type DebugHandler struct { 35 memStore memstore.MemStore 36 // For getting cutoff of a shard. 37 metaStore metastore.MetaStore 38 queryHandler *QueryHandler 39 healthCheckHandler *HealthCheckHandler 40 } 41 42 // NewDebugHandler returns a new DebugHandler. 43 func NewDebugHandler(memStore memstore.MemStore, metaStore metastore.MetaStore, queryHandler *QueryHandler, healthCheckHandler *HealthCheckHandler) *DebugHandler { 44 return &DebugHandler{ 45 memStore: memStore, 46 metaStore: metaStore, 47 queryHandler: queryHandler, 48 healthCheckHandler: healthCheckHandler, 49 } 50 } 51 52 // Register registers http handlers. 53 func (handler *DebugHandler) Register(router *mux.Router) { 54 router.HandleFunc("/health", handler.Health).Methods(http.MethodGet) 55 router.HandleFunc("/health/{onOrOff}", handler.HealthSwitch).Methods(http.MethodPost) 56 router.HandleFunc("/jobs/{jobType}", handler.ShowJobStatus).Methods(http.MethodGet) 57 router.HandleFunc("/devices", handler.ShowDeviceStatus).Methods(http.MethodGet) 58 router.HandleFunc("/host-memory", handler.ShowHostMemory).Methods(http.MethodGet) 59 router.HandleFunc("/{table}/{shard}", handler.ShowShardMeta).Methods(http.MethodGet) 60 router.HandleFunc("/{table}/{shard}/archive", handler.Archive).Methods(http.MethodPost) 61 router.HandleFunc("/{table}/{shard}/backfill", handler.Backfill).Methods(http.MethodPost) 62 router.HandleFunc("/{table}/{shard}/snapshot", handler.Snapshot).Methods(http.MethodPost) 63 router.HandleFunc("/{table}/{shard}/purge", handler.Purge).Methods(http.MethodPost) 64 router.HandleFunc("/{table}/{shard}/batches/{batch}", handler.ShowBatch).Methods(http.MethodGet) 65 router.HandleFunc("/{table}/{shard}/batches/{batch}/vector-parties/{column}", handler.LoadVectorParty).Methods(http.MethodGet) 66 router.HandleFunc("/{table}/{shard}/batches/{batch}/vector-parties/{column}", handler.EvictVectorParty).Methods(http.MethodDelete) 67 router.HandleFunc("/{table}/{shard}/primary-keys", handler.LookupPrimaryKey).Methods(http.MethodGet) 68 router.HandleFunc("/{table}/{shard}/redologs", handler.ListRedoLogs). 69 Methods(http.MethodGet) 70 router.HandleFunc("/{table}/{shard}/redologs/{creationTime}/upsertbatches", handler.ListUpsertBatches). 71 Methods(http.MethodGet) 72 router.HandleFunc("/{table}/{shard}/redologs/{creationTime}/upsertbatches/{offset}", handler.ReadUpsertBatch). 73 Methods(http.MethodGet) 74 router.HandleFunc("/{table}/{shard}/backfill-manager/upsertbatches/{offset}", handler.ReadBackfillQueueUpsertBatch).Methods(http.MethodGet) 75 } 76 77 // Health returns whether the health check is on or off 78 func (handler *DebugHandler) Health(w http.ResponseWriter, r *http.Request) { 79 handler.healthCheckHandler.RLock() 80 disabled := handler.healthCheckHandler.disable 81 handler.healthCheckHandler.RUnlock() 82 status := "on" 83 if disabled { 84 status = "off" 85 } 86 io.WriteString(w, status) 87 } 88 89 // HealthSwitch will turn on health check based on the request. 90 func (handler *DebugHandler) HealthSwitch(w http.ResponseWriter, r *http.Request) { 91 var request HealthSwitchRequest 92 var err error 93 if err = ReadRequest(r, &request); err != nil { 94 RespondWithBadRequest(w, err) 95 return 96 } 97 98 if request.OnOrOff != "on" && request.OnOrOff != "off" { 99 RespondWithBadRequest(w, errors.New("must specify on or off in the url")) 100 return 101 } 102 103 handler.healthCheckHandler.Lock() 104 handler.healthCheckHandler.disable = request.OnOrOff == "off" 105 handler.healthCheckHandler.Unlock() 106 io.WriteString(w, "OK") 107 } 108 109 // ShowBatch will only show batches that is present in memory, it will not request batch 110 // from DiskStore. 111 func (handler *DebugHandler) ShowBatch(w http.ResponseWriter, r *http.Request) { 112 113 var request ShowBatchRequest 114 var response ShowBatchResponse 115 var err error 116 117 err = ReadRequest(r, &request) 118 if err != nil { 119 RespondWithBadRequest(w, err) 120 return 121 } 122 123 if request.NumRows <= 0 || request.NumRows > 100 { 124 request.NumRows = 100 125 } 126 127 if request.StartRow < 0 { 128 request.StartRow = 0 129 } 130 131 response.Body.StartRow = request.StartRow 132 133 schema, err := handler.memStore.GetSchema(request.TableName) 134 if err != nil { 135 RespondWithBadRequest(w, ErrTableDoesNotExist) 136 return 137 } 138 139 schema.RLock() 140 response.Body.Vectors = make([]memCom.SlicedVector, 0, len(schema.Schema.Columns)) 141 response.Body.Columns = make([]string, 0, len(schema.Schema.Columns)) 142 for columnID, column := range schema.Schema.Columns { 143 response.Body.Columns = append(response.Body.Columns, column.Name) 144 response.Body.Types = append(response.Body.Types, column.Type) 145 if column.Deleted { 146 response.Body.Deleted = append(response.Body.Deleted, columnID) 147 } 148 } 149 schema.RUnlock() 150 151 var shard *memstore.TableShard 152 shard, err = handler.memStore.GetTableShard(request.TableName, request.ShardID) 153 if err != nil { 154 RespondWithBadRequest(w, err) 155 return 156 } 157 158 defer func() { 159 shard.Users.Done() 160 if err != nil { 161 RespondWithError(w, err) 162 } else { 163 RespondWithJSONObject(w, response.Body) 164 } 165 }() 166 167 // request archiveBatch 168 if request.BatchID >= 0 { 169 shard.ArchiveStore.RLock() 170 currentVersion := shard.ArchiveStore.CurrentVersion 171 currentVersion.Users.Add(1) 172 shard.ArchiveStore.RUnlock() 173 defer currentVersion.Users.Done() 174 175 archiveBatch := currentVersion.GetBatchForRead(request.BatchID) 176 if archiveBatch == nil { 177 err = ErrBatchDoesNotExist 178 return 179 } 180 defer archiveBatch.RUnlock() 181 // holding archive batch lock will prevent any loading and eviction. 182 response.Body.NumRows, response.Body.Vectors = readRows(archiveBatch.Columns, request.StartRow, request.NumRows) 183 } else { 184 // request liveBatch 185 batchIDs, numRecordsInLastBatch := shard.LiveStore.GetBatchIDs() 186 liveBatch := shard.LiveStore.GetBatchForRead(int32(request.BatchID)) 187 if liveBatch == nil { 188 err = ErrBatchDoesNotExist 189 return 190 } 191 defer liveBatch.RUnlock() 192 193 if batchIDs[len(batchIDs)-1] == int32(request.BatchID) { 194 if request.StartRow >= numRecordsInLastBatch { 195 return 196 } 197 198 if request.NumRows > numRecordsInLastBatch-request.StartRow { 199 request.NumRows = numRecordsInLastBatch - request.StartRow 200 } 201 } 202 response.Body.NumRows, response.Body.Vectors = readRows(liveBatch.Columns, request.StartRow, request.NumRows) 203 } 204 205 // translate enums 206 schema.RLock() 207 for columnID, column := range schema.Schema.Columns { 208 if !column.Deleted && column.IsEnumColumn() && columnID < len(response.Body.Vectors) { 209 vector := &response.Body.Vectors[columnID] 210 err = translateEnums(vector, schema.EnumDicts[column.Name].ReverseDict) 211 } 212 } 213 schema.RUnlock() 214 } 215 216 func readRows(vps []memCom.VectorParty, startRow, numRows int) (n int, vectors []memCom.SlicedVector) { 217 vectors = make([]memCom.SlicedVector, len(vps)) 218 for columnID, vp := range vps { 219 if vp != nil { 220 vectors[columnID] = vp.Slice(startRow, numRows) 221 if len(vectors[columnID].Counts) > 0 { 222 n = vectors[columnID].Counts[len(vectors[columnID].Counts)-1] 223 } 224 } else { 225 vectors[columnID] = memCom.SlicedVector{ 226 Values: []interface{}{nil}, 227 Counts: []int{numRows}, 228 } 229 } 230 } 231 return n, vectors 232 } 233 234 func translateEnums(vector *memCom.SlicedVector, enumCases []string) error { 235 for index, value := range vector.Values { 236 if value != nil { 237 var id int 238 switch v := value.(type) { 239 case uint8: 240 id = int(v) 241 case uint16: 242 id = int(v) 243 default: 244 // this should never happen 245 return utils.StackError(nil, "Wrong data type for enum vector, %T", value) 246 } 247 248 // it is possible when enum change has not arrived in memory yet, 249 // treat it as empty string temporarily. 250 vector.Values[index] = "" 251 if id < len(enumCases) { 252 vector.Values[index] = enumCases[id] 253 } 254 } 255 } 256 return nil 257 } 258 259 // LookupPrimaryKey looks up a key in primary key for given table and shard 260 func (handler *DebugHandler) LookupPrimaryKey(w http.ResponseWriter, r *http.Request) { 261 var request LookupPrimaryKeyRequest 262 err := ReadRequest(r, &request) 263 if err != nil { 264 RespondWithError(w, err) 265 return 266 } 267 268 shard, err := handler.memStore.GetTableShard(request.TableName, request.ShardID) 269 if err != nil { 270 RespondWithError(w, utils.APIError{ 271 Code: http.StatusBadRequest, 272 Message: err.Error(), 273 Cause: err, 274 }) 275 } 276 defer shard.Users.Done() 277 278 keyStrs := strings.Split(request.Key, ",") 279 var found bool 280 var recordID memstore.RecordID 281 recordID, found = shard.LiveStore.LookupKey(keyStrs) 282 if !found { 283 RespondWithError(w, utils.APIError{ 284 Code: http.StatusNotFound, 285 Message: fmt.Sprintf("Key '%s' does not exist or expired", request.Key), 286 }) 287 return 288 } 289 RespondWithJSONObject(w, recordID) 290 } 291 292 // Archive starts an archiving process on demand. 293 func (handler *DebugHandler) Archive(w http.ResponseWriter, r *http.Request) { 294 var request ArchiveRequest 295 err := ReadRequest(r, &request) 296 if err != nil { 297 RespondWithBadRequest(w, err) 298 return 299 } 300 301 // Just check table and shard existence. 302 shard, err := handler.memStore.GetTableShard(request.TableName, request.ShardID) 303 if err != nil { 304 RespondWithBadRequest(w, err) 305 return 306 } 307 shard.Users.Done() 308 309 scheduler := handler.memStore.GetScheduler() 310 err, errChan := scheduler.SubmitJob( 311 scheduler.NewArchivingJob(request.TableName, request.ShardID, request.Body.Cutoff)) 312 if err == nil { 313 go func() { 314 <-errChan 315 }() 316 RespondJSONObjectWithCode(w, http.StatusOK, "Archiving job submitted") 317 } else { 318 RespondJSONObjectWithCode(w, http.StatusMethodNotAllowed, err) 319 } 320 321 } 322 323 // Backfill starts an backfill process on demand. 324 func (handler *DebugHandler) Backfill(w http.ResponseWriter, r *http.Request) { 325 var request BackfillRequest 326 err := ReadRequest(r, &request) 327 if err != nil { 328 RespondWithBadRequest(w, err) 329 return 330 } 331 332 // Just check table and shard existence. 333 shard, err := handler.memStore.GetTableShard(request.TableName, request.ShardID) 334 if err != nil { 335 RespondWithBadRequest(w, err) 336 return 337 } 338 shard.Users.Done() 339 340 scheduler := handler.memStore.GetScheduler() 341 err, errChan := scheduler.SubmitJob( 342 scheduler.NewBackfillJob(request.TableName, request.ShardID)) 343 if err == nil { 344 go func() { 345 <-errChan 346 }() 347 348 RespondJSONObjectWithCode(w, http.StatusOK, "Backfill job submitted") 349 } 350 } 351 352 // Snapshot starts an snapshot process on demand. 353 func (handler *DebugHandler) Snapshot(w http.ResponseWriter, r *http.Request) { 354 var request SnapshotRequest 355 err := ReadRequest(r, &request) 356 if err != nil { 357 RespondWithBadRequest(w, err) 358 return 359 } 360 361 // Just check table and shard existence. 362 shard, err := handler.memStore.GetTableShard(request.TableName, request.ShardID) 363 if err != nil { 364 RespondWithBadRequest(w, err) 365 return 366 } 367 defer shard.Users.Done() 368 369 scheduler := handler.memStore.GetScheduler() 370 err, errChan := scheduler.SubmitJob( 371 scheduler.NewSnapshotJob(request.TableName, request.ShardID)) 372 if err == nil { 373 go func() { 374 <-errChan 375 }() 376 377 RespondJSONObjectWithCode(w, http.StatusOK, "Snapshot job submitted") 378 } 379 } 380 381 // Purge starts an purge process on demand. 382 func (handler *DebugHandler) Purge(w http.ResponseWriter, r *http.Request) { 383 var request PurgeRequest 384 err := ReadRequest(r, &request) 385 if err != nil { 386 RespondWithBadRequest(w, err) 387 return 388 } 389 390 if !request.Body.SafePurge && (request.Body.BatchIDStart < 0 || request.Body.BatchIDEnd < 0 || request.Body.BatchIDStart > request.Body.BatchIDEnd) { 391 RespondWithBadRequest(w, fmt.Errorf("invalid batch range, expects both to be > 0, got [%d, %d)", 392 request.Body.BatchIDStart, request.Body.BatchIDEnd)) 393 return 394 } 395 396 shard, err := handler.memStore.GetTableShard(request.TableName, request.ShardID) 397 if err != nil { 398 RespondWithBadRequest(w, err) 399 return 400 } 401 defer shard.Users.Done() 402 403 if request.Body.SafePurge { 404 retentionDays := shard.Schema.Schema.Config.RecordRetentionInDays 405 if retentionDays > 0 { 406 nowInDay := int(utils.Now().Unix() / 86400) 407 request.Body.BatchIDStart = 0 408 request.Body.BatchIDEnd = nowInDay - retentionDays 409 } else { 410 RespondWithBadRequest(w, utils.APIError{Message: "safe purge attempted on table with infinite retention"}) 411 return 412 } 413 } 414 415 scheduler := handler.memStore.GetScheduler() 416 err, errChan := scheduler.SubmitJob( 417 scheduler.NewPurgeJob(request.TableName, request.ShardID, request.Body.BatchIDStart, request.Body.BatchIDEnd)) 418 if err == nil { 419 go func() { 420 <-errChan 421 }() 422 423 RespondJSONObjectWithCode(w, http.StatusOK, "Purge job submitted") 424 } 425 } 426 427 // ShowShardMeta shows the metadata for a table shard. It won't show the underlying data. 428 func (handler *DebugHandler) ShowShardMeta(w http.ResponseWriter, r *http.Request) { 429 var request ShowShardMetaRequest 430 err := ReadRequest(r, &request) 431 if err != nil { 432 RespondWithBadRequest(w, err) 433 return 434 } 435 shard, err := handler.memStore.GetTableShard(request.TableName, request.ShardID) 436 if err != nil { 437 RespondWithBadRequest(w, err) 438 return 439 } 440 defer shard.Users.Done() 441 RespondWithJSONObject(w, shard) 442 return 443 } 444 445 // ListRedoLogs lists all the redo log files for a given shard. 446 func (handler *DebugHandler) ListRedoLogs(w http.ResponseWriter, r *http.Request) { 447 var request ListRedoLogsRequest 448 err := ReadRequest(r, &request) 449 if err != nil { 450 RespondWithBadRequest(w, err) 451 return 452 } 453 454 shard, err := handler.memStore.GetTableShard(request.TableName, request.ShardID) 455 if err != nil { 456 RespondWithBadRequest(w, err) 457 return 458 } 459 defer shard.Users.Done() 460 redoLogFiles, err := shard.NewRedoLogBrowser().ListLogFiles() 461 462 if err != nil { 463 RespondWithError(w, err) 464 return 465 } 466 467 response := make(ListRedoLogsResponse, len(redoLogFiles)) 468 469 for i, redoLogFile := range redoLogFiles { 470 response[i] = strconv.FormatInt(redoLogFile, 10) 471 } 472 473 RespondWithJSONObject(w, response) 474 return 475 } 476 477 // ListUpsertBatches returns offsets of upsert batches in the redo log file. 478 func (handler *DebugHandler) ListUpsertBatches(w http.ResponseWriter, r *http.Request) { 479 var request ListUpsertBatchesRequest 480 err := ReadRequest(r, &request) 481 if err != nil { 482 RespondWithBadRequest(w, err) 483 return 484 } 485 shard, err := handler.memStore.GetTableShard(request.TableName, request.ShardID) 486 if err != nil { 487 RespondWithBadRequest(w, err) 488 return 489 } 490 491 defer shard.Users.Done() 492 offsets, err := shard.NewRedoLogBrowser().ListUpsertBatch(request.CreationTime) 493 if err != nil { 494 RespondWithError(w, err) 495 return 496 } 497 RespondWithJSONObject(w, &offsets) 498 return 499 } 500 501 // ReadUpsertBatch shows the records of an upsert batch given a redolog file creation time and 502 // upsert batch index within the file. 503 func (handler *DebugHandler) ReadUpsertBatch(w http.ResponseWriter, r *http.Request) { 504 var request ReadRedologUpsertBatchRequest 505 if err := ReadRequest(r, &request); err != nil { 506 RespondWithBadRequest(w, err) 507 return 508 } 509 shard, err := handler.memStore.GetTableShard(request.TableName, request.ShardID) 510 if err != nil { 511 RespondWithBadRequest(w, err) 512 return 513 } 514 defer shard.Users.Done() 515 516 rows, columnNames, numTotalRows, err := shard.NewRedoLogBrowser().ReadData(request.CreationTime, 517 request.Offset, request.Start, request.Length) 518 if err != nil { 519 RespondWithError(w, err) 520 return 521 } 522 response := ReadUpsertBatchResponse{ 523 Draw: request.Draw, 524 Data: rows, 525 ColumnNames: columnNames, 526 RecordsTotal: numTotalRows, 527 RecordsFiltered: numTotalRows, 528 } 529 RespondWithJSONObject(w, &response) 530 return 531 } 532 533 // LoadVectorParty requests a vector party from disk if it is not already in memory 534 func (handler *DebugHandler) LoadVectorParty(w http.ResponseWriter, r *http.Request) { 535 var request LoadVectorPartyRequest 536 if err := ReadRequest(r, &request); err != nil { 537 RespondWithBadRequest(w, err) 538 return 539 } 540 541 if request.BatchID < 0 { 542 RespondWithBadRequest(w, errors.New("Live batch vector party cannot be loaded")) 543 return 544 } 545 546 shard, err := handler.memStore.GetTableShard(request.TableName, request.ShardID) 547 if err != nil { 548 RespondWithBadRequest(w, err) 549 return 550 } 551 defer shard.Users.Done() 552 553 schema := shard.Schema 554 isDeleted := false 555 shard.Schema.RLock() 556 columnID, exist := schema.ColumnIDs[request.ColumnName] 557 if columnID < len(schema.Schema.Columns) { 558 isDeleted = schema.Schema.Columns[columnID].Deleted 559 } 560 shard.Schema.RUnlock() 561 562 if !exist { 563 RespondWithBadRequest(w, ErrColumnDoesNotExist) 564 return 565 } 566 567 if isDeleted { 568 RespondWithBadRequest(w, ErrColumnDeleted) 569 return 570 } 571 572 version := shard.ArchiveStore.GetCurrentVersion() 573 defer version.Users.Done() 574 575 batch := version.RequestBatch(int32(request.BatchID)) 576 vp := batch.RequestVectorParty(columnID) 577 if vp != nil { 578 vp.WaitForDiskLoad() 579 vp.Release() 580 } 581 RespondWithJSONObject(w, nil) 582 } 583 584 // EvictVectorParty evict a vector party from memory. 585 func (handler *DebugHandler) EvictVectorParty(w http.ResponseWriter, r *http.Request) { 586 var request EvictVectorPartyRequest 587 if err := ReadRequest(r, &request); err != nil { 588 RespondWithBadRequest(w, err) 589 return 590 } 591 592 if request.BatchID < 0 { 593 RespondWithBadRequest(w, errors.New("live batch vector party cannot be evicted")) 594 return 595 } 596 597 shard, err := handler.memStore.GetTableShard(request.TableName, request.ShardID) 598 if err != nil { 599 RespondWithBadRequest(w, err) 600 return 601 } 602 defer shard.Users.Done() 603 604 shard.Schema.RLock() 605 columnID, exist := shard.Schema.ColumnIDs[request.ColumnName] 606 shard.Schema.RUnlock() 607 608 if !exist { 609 RespondWithError(w, ErrColumnDoesNotExist) 610 return 611 } 612 613 version := shard.ArchiveStore.GetCurrentVersion() 614 defer version.Users.Done() 615 616 batch := version.RequestBatch(int32(request.BatchID)) 617 // this operation is blocking and needs the user to wait 618 batch.BlockingDelete(columnID) 619 RespondWithJSONObject(w, nil) 620 } 621 622 // ShowJobStatus shows the current archive job status. 623 func (handler *DebugHandler) ShowJobStatus(w http.ResponseWriter, r *http.Request) { 624 var request ShowJobStatusRequest 625 if err := ReadRequest(r, &request); err != nil { 626 RespondWithBadRequest(w, err) 627 return 628 } 629 630 scheduler := handler.memStore.GetScheduler() 631 scheduler.RLock() 632 jsonBuffer, err := json.Marshal(scheduler.GetJobDetails(memCom.JobType(request.JobType))) 633 scheduler.RUnlock() 634 RespondWithJSONBytes(w, jsonBuffer, err) 635 return 636 } 637 638 // ShowDeviceStatus shows the current scheduler status. 639 func (handler *DebugHandler) ShowDeviceStatus(w http.ResponseWriter, r *http.Request) { 640 deviceManager := handler.queryHandler.GetDeviceManager() 641 deviceManager.RLock() 642 jsonBuffer, err := json.Marshal(*deviceManager) 643 deviceManager.RUnlock() 644 645 RespondWithJSONBytes(w, jsonBuffer, err) 646 return 647 } 648 649 // ShowHostMemory shows the current host memory usage 650 func (handler *DebugHandler) ShowHostMemory(w http.ResponseWriter, r *http.Request) { 651 memoryUsageByTableShard, err := handler.memStore.GetMemoryUsageDetails() 652 if err != nil { 653 RespondWithError(w, err) 654 return 655 } 656 RespondWithJSONObject(w, memoryUsageByTableShard) 657 } 658 659 // ReadBackfillQueueUpsertBatch reads upsert batch inside backfill manager backfill queue 660 func (handler *DebugHandler) ReadBackfillQueueUpsertBatch(w http.ResponseWriter, r *http.Request) { 661 var request ReadBackfillQueueUpsertBatchRequest 662 err := ReadRequest(r, &request) 663 if err != nil { 664 RespondWithBadRequest(w, err) 665 return 666 } 667 668 shard, err := handler.memStore.GetTableShard(request.TableName, request.ShardID) 669 if err != nil { 670 RespondWithBadRequest(w, err) 671 return 672 } 673 data, columnNames, err := shard.LiveStore.BackfillManager.ReadUpsertBatch(int(request.Offset), request.Start, request.Length, shard.Schema) 674 shard.Users.Done() 675 676 if err != nil { 677 RespondWithBadRequest(w, err) 678 return 679 } 680 681 response := ReadUpsertBatchResponse{ 682 Data: data, 683 ColumnNames: columnNames, 684 RecordsFiltered: len(data), 685 RecordsTotal: len(data), 686 Draw: request.Draw, 687 } 688 689 RespondWithJSONObject(w, response) 690 return 691 }