github.com/rohankumardubey/aresdb@v0.0.2-0.20190517170215-e54e3ca06b9c/query/time_series_aggregate.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 query 16 17 // #cgo LDFLAGS: -L${SRCDIR}/../lib -lalgorithm 18 // #include "time_series_aggregate.h" 19 import "C" 20 import ( 21 "github.com/uber/aresdb/utils" 22 "strconv" 23 "unsafe" 24 25 "github.com/uber/aresdb/cgoutils" 26 "github.com/uber/aresdb/memstore" 27 memCom "github.com/uber/aresdb/memstore/common" 28 "github.com/uber/aresdb/query/common" 29 "github.com/uber/aresdb/query/expr" 30 ) 31 32 // DataTypeToCDataType mapps from memstore data type to c data types 33 var DataTypeToCDataType = map[memCom.DataType]C.enum_DataType{ 34 memCom.Bool: C.Bool, 35 memCom.Int8: C.Int8, 36 memCom.Uint8: C.Uint8, 37 memCom.Int16: C.Int16, 38 memCom.Uint16: C.Uint16, 39 memCom.Int32: C.Int32, 40 memCom.Int64: C.Int64, 41 memCom.Uint32: C.Uint32, 42 memCom.Float32: C.Float32, 43 memCom.SmallEnum: C.Uint8, 44 memCom.BigEnum: C.Uint16, 45 memCom.GeoPoint: C.GeoPoint, 46 memCom.UUID: C.UUID, 47 } 48 49 // UnaryExprTypeToCFunctorType maps from unary operator to C UnaryFunctorType 50 var UnaryExprTypeToCFunctorType = map[expr.Token]C.enum_UnaryFunctorType{ 51 expr.NOT: C.Not, 52 expr.UNARY_MINUS: C.Negate, 53 expr.IS_NULL: C.IsNull, 54 expr.IS_NOT_NULL: C.IsNotNull, 55 expr.BITWISE_NOT: C.BitwiseNot, 56 expr.GET_WEEK_START: C.GetWeekStart, 57 expr.GET_MONTH_START: C.GetMonthStart, 58 expr.GET_QUARTER_START: C.GetQuarterStart, 59 expr.GET_YEAR_START: C.GetYearStart, 60 expr.GET_DAY_OF_MONTH: C.GetDayOfMonth, 61 expr.GET_DAY_OF_YEAR: C.GetDayOfYear, 62 expr.GET_MONTH_OF_YEAR: C.GetMonthOfYear, 63 expr.GET_QUARTER_OF_YEAR: C.GetQuarterOfYear, 64 expr.GET_HLL_VALUE: C.GetHLLValue, 65 } 66 67 // BinaryExprTypeToCFunctorType maps from binary operator to C BinaryFunctorType 68 var BinaryExprTypeToCFunctorType = map[expr.Token]C.enum_BinaryFunctorType{ 69 expr.AND: C.And, 70 expr.OR: C.Or, 71 expr.EQ: C.Equal, 72 expr.NEQ: C.NotEqual, 73 expr.LT: C.LessThan, 74 expr.LTE: C.LessThanOrEqual, 75 expr.GT: C.GreaterThan, 76 expr.GTE: C.GreaterThanOrEqual, 77 expr.ADD: C.Plus, 78 expr.SUB: C.Minus, 79 expr.MUL: C.Multiply, 80 expr.DIV: C.Divide, 81 expr.MOD: C.Mod, 82 expr.BITWISE_AND: C.BitwiseAnd, 83 expr.BITWISE_OR: C.BitwiseOr, 84 expr.BITWISE_XOR: C.BitwiseXor, 85 expr.FLOOR: C.Floor, 86 expr.CONVERT_TZ: C.Plus, 87 // TODO: expr.BITWISE_LEFT_SHIFT ? 88 // TODO: expr.BITWISE_RIGHT_SHIFT ? 89 } 90 91 type rootAction func(functorType uint32, stream unsafe.Pointer, device int, inputs []C.InputVector, exp expr.Expr) 92 93 func makeForeignColumnInput(columnIndex int, recordIDs unsafe.Pointer, table foreignTable, timezoneLookup unsafe.Pointer, timezoneLookupSize int) C.InputVector { 94 var vector C.InputVector 95 var foreignColumnVector C.ForeignColumnVector 96 97 vpSlices := make([]C.VectorPartySlice, len(table.batches)) 98 var dataType memCom.DataType 99 var defaultValue memCom.DataValue 100 for batchIndex, batch := range table.batches { 101 column := batch[columnIndex] 102 dataType = column.valueType 103 defaultValue = column.defaultValue 104 vpSlices[batchIndex] = makeVectorPartySlice(batch[columnIndex]) 105 } 106 107 foreignColumnVector.RecordIDs = (*C.RecordID)(recordIDs) 108 if len(vpSlices) > 0 { 109 foreignColumnVector.Batches = (*C.VectorPartySlice)(unsafe.Pointer(&vpSlices[0])) 110 } 111 foreignColumnVector.BaseBatchID = (C.int32_t)(memstore.BaseBatchID) 112 foreignColumnVector.NumBatches = (C.int32_t)(len(table.batches)) 113 foreignColumnVector.NumRecordsInLastBatch = (C.int32_t)(table.numRecordsInLastBatch) 114 foreignColumnVector.DataType = DataTypeToCDataType[dataType] 115 foreignColumnVector.DefaultValue = makeDefaultValue(defaultValue) 116 foreignColumnVector.TimezoneLookup = (*C.int16_t)(timezoneLookup) 117 foreignColumnVector.TimezoneLookupSize = (C.int16_t)(timezoneLookupSize) 118 119 *(*C.ForeignColumnVector)(unsafe.Pointer(&vector.Vector)) = foreignColumnVector 120 vector.Type = C.ForeignColumnInput 121 return vector 122 } 123 124 func makeDefaultValue(value memCom.DataValue) C.DefaultValue { 125 var defaultValue C.DefaultValue 126 defaultValue.HasDefault = (C.bool)(value.Valid) 127 if value.Valid { 128 switch value.DataType { 129 case memCom.Bool: 130 *(*C.bool)(unsafe.Pointer(&defaultValue.Value)) = (C.bool)(value.BoolVal) 131 case memCom.Int8: 132 *(*C.int32_t)(unsafe.Pointer(&defaultValue.Value)) = (C.int32_t)(*(*int8)(value.OtherVal)) 133 case memCom.Int16: 134 *(*C.int32_t)(unsafe.Pointer(&defaultValue.Value)) = (C.int32_t)(*(*int16)(value.OtherVal)) 135 case memCom.Int32: 136 *(*C.int32_t)(unsafe.Pointer(&defaultValue.Value)) = (C.int32_t)(*(*int32)(value.OtherVal)) 137 case memCom.SmallEnum: 138 fallthrough 139 case memCom.Uint8: 140 *(*C.uint32_t)(unsafe.Pointer(&defaultValue.Value)) = (C.uint32_t)(*(*uint8)(value.OtherVal)) 141 case memCom.BigEnum: 142 fallthrough 143 case memCom.Uint16: 144 *(*C.uint32_t)(unsafe.Pointer(&defaultValue.Value)) = (C.uint32_t)(*(*uint16)(value.OtherVal)) 145 case memCom.Uint32: 146 *(*C.uint32_t)(unsafe.Pointer(&defaultValue.Value)) = (C.uint32_t)(*(*uint32)(value.OtherVal)) 147 case memCom.Float32: 148 *(*C.float)(unsafe.Pointer(&defaultValue.Value)) = (C.float)(*(*float32)(value.OtherVal)) 149 case memCom.Int64: 150 *(*C.int64_t)(unsafe.Pointer(&defaultValue.Value)) = (C.int64_t)(*(*int64)(value.OtherVal)) 151 case memCom.GeoPoint: 152 *(*C.GeoPointT)(unsafe.Pointer(&defaultValue.Value)) = *(*C.GeoPointT)(value.OtherVal) 153 case memCom.UUID: 154 *(*C.UUIDT)(unsafe.Pointer(&defaultValue.Value)) = *(*C.UUIDT)(value.OtherVal) 155 default: 156 // Otherwise it's the default value type we don't support yet, setting it to null to be safe. 157 defaultValue.HasDefault = false 158 } 159 } 160 return defaultValue 161 } 162 163 func makeVectorPartySlice(column deviceVectorPartySlice) C.VectorPartySlice { 164 var vpSlice C.VectorPartySlice 165 var basePtr unsafe.Pointer 166 var startingIndex int 167 var nullsOffset uint32 168 var valuesOffset uint32 169 170 if !column.counts.isNull() { 171 basePtr = utils.MemAccess(column.counts.getPointer(), column.countStartIndex*4) 172 } 173 174 if !column.nulls.isNull() { 175 startingIndex = column.nullStartIndex % 8 176 nulls := utils.MemAccess(column.nulls.getPointer(), column.nullStartIndex/8) 177 if basePtr == nil { 178 basePtr = nulls 179 } else { 180 nullsOffset = uint32(utils.MemDist(nulls, basePtr)) 181 } 182 } 183 184 if !column.values.isNull() { 185 values := utils.MemAccess(column.values.getPointer(), 186 column.valueStartIndex*memCom.DataTypeBits(column.valueType)/8) 187 if basePtr == nil { 188 basePtr = values 189 } else { 190 valuesOffset = uint32(utils.MemDist(values, basePtr)) 191 } 192 } 193 194 vpSlice.BasePtr = (*C.uint8_t)(basePtr) 195 vpSlice.NullsOffset = (C.uint32_t)(nullsOffset) 196 vpSlice.ValuesOffset = (C.uint32_t)(valuesOffset) 197 vpSlice.StartingIndex = (C.uint8_t)(startingIndex) 198 199 vpSlice.Length = (C.uint32_t)(column.length) 200 vpSlice.DataType = DataTypeToCDataType[column.valueType] 201 vpSlice.DefaultValue = makeDefaultValue(column.defaultValue) 202 return vpSlice 203 } 204 205 func makeVectorPartySliceInput(column deviceVectorPartySlice) C.InputVector { 206 var vector C.InputVector 207 *(*C.VectorPartySlice)(unsafe.Pointer(&vector.Vector)) = makeVectorPartySlice(column) 208 vector.Type = C.VectorPartyInput 209 return vector 210 } 211 212 func makeConstantInput(val interface{}, isValid bool) C.InputVector { 213 var constVector C.ConstantVector 214 constVector.IsValid = C.bool(isValid) 215 216 switch val.(type) { 217 case float64, float32: 218 floatVal := val.(float64) 219 *(*C.float)(unsafe.Pointer(&constVector.Value)) = C.float(floatVal) 220 constVector.DataType = C.ConstFloat 221 case *expr.GeopointLiteral: 222 geopoint := val.(*expr.GeopointLiteral).Val 223 *(*C.GeoPointT)(unsafe.Pointer(&constVector.Value)) = *(*C.GeoPointT)(unsafe.Pointer(&geopoint[0])) 224 constVector.DataType = C.ConstGeoPoint 225 case *expr.NumberLiteral: 226 t := val.(*expr.NumberLiteral) 227 if t.Type() == expr.Float { 228 *(*C.float)(unsafe.Pointer(&constVector.Value)) = C.float(t.Val) 229 constVector.DataType = C.ConstFloat 230 } else { 231 *(*C.int32_t)(unsafe.Pointer(&constVector.Value)) = C.int32_t(t.Int) 232 constVector.DataType = C.ConstInt 233 } 234 default: 235 intVal := val.(int) 236 *(*C.int32_t)(unsafe.Pointer(&constVector.Value)) = C.int32_t(intVal) 237 constVector.DataType = C.ConstInt 238 } 239 240 var vector C.InputVector 241 *(*C.ConstantVector)(unsafe.Pointer(&vector.Vector)) = constVector 242 vector.Type = C.ConstantInput 243 return vector 244 } 245 246 func makeScratchSpaceInput(values unsafe.Pointer, nulls unsafe.Pointer, dataType C.enum_DataType) C.InputVector { 247 var scratchSpaceVector C.ScratchSpaceVector 248 scratchSpaceVector.Values = (*C.uint8_t)(values) 249 scratchSpaceVector.NullsOffset = (C.uint32_t)(utils.MemDist(nulls, values)) 250 scratchSpaceVector.DataType = dataType 251 252 var vector C.InputVector 253 *(*C.ScratchSpaceVector)(unsafe.Pointer(&vector.Vector)) = scratchSpaceVector 254 vector.Type = C.ScratchSpaceInput 255 256 return vector 257 } 258 259 func makeMeasureVectorOutput(measureVector unsafe.Pointer, outputDataType C.enum_DataType, aggFunc C.enum_AggregateFunction) C.OutputVector { 260 var measureOutputVector C.MeasureOutputVector 261 262 measureOutputVector.Values = (*C.uint32_t)(measureVector) 263 measureOutputVector.DataType = outputDataType 264 measureOutputVector.AggFunc = aggFunc 265 266 var vector C.OutputVector 267 *(*C.MeasureOutputVector)(unsafe.Pointer(&vector.Vector)) = measureOutputVector 268 vector.Type = C.MeasureOutput 269 return vector 270 } 271 272 func makeDimensionVectorOutput(dimensionVector unsafe.Pointer, valueOffset, nullOffset int, dataType C.enum_DataType) C.OutputVector { 273 var dimensionOutputVector C.DimensionOutputVector 274 275 dimensionOutputVector.DimValues = (*C.uint8_t)(utils.MemAccess(dimensionVector, valueOffset)) 276 dimensionOutputVector.DimNulls = (*C.uint8_t)(utils.MemAccess(dimensionVector, nullOffset)) 277 dimensionOutputVector.DataType = dataType 278 279 var vector C.OutputVector 280 *(*C.DimensionOutputVector)(unsafe.Pointer(&vector.Vector)) = dimensionOutputVector 281 vector.Type = C.DimensionOutput 282 return vector 283 } 284 285 func makeScratchSpaceOutput(values unsafe.Pointer, nulls unsafe.Pointer, dataType C.enum_DataType) C.OutputVector { 286 var scratchSpaceVector C.ScratchSpaceVector 287 scratchSpaceVector.Values = (*C.uint8_t)(values) 288 scratchSpaceVector.NullsOffset = (C.uint32_t)(utils.MemDist(nulls, values)) 289 scratchSpaceVector.DataType = dataType 290 291 var vector C.OutputVector 292 *(*C.ScratchSpaceVector)(unsafe.Pointer(&vector.Vector)) = scratchSpaceVector 293 vector.Type = C.ScratchSpaceOutput 294 295 return vector 296 } 297 298 func makeDimensionColumnVector(dimensionVector, hashVector, indexVector unsafe.Pointer, numDims common.DimCountsPerDimWidth, vectorCapacity int) C.DimensionColumnVector { 299 var dimensionColumnVector C.DimensionColumnVector 300 dimensionColumnVector.DimValues = (*C.uint8_t)(dimensionVector) 301 dimensionColumnVector.HashValues = (*C.uint64_t)(hashVector) 302 dimensionColumnVector.IndexVector = (*C.uint32_t)(indexVector) 303 304 dimensionColumnVector.VectorCapacity = (C.int)(vectorCapacity) 305 for i := 0; i < len(numDims); i++ { 306 dimensionColumnVector.NumDimsPerDimWidth[i] = (C.uint8_t)(numDims[i]) 307 } 308 return dimensionColumnVector 309 } 310 311 func getOutputDataType(exprType expr.Type, outputWidthInByte int) C.enum_DataType { 312 if outputWidthInByte == 4 { 313 switch exprType { 314 case expr.Float: 315 return C.Float32 316 case expr.Unsigned: 317 return C.Uint32 318 default: 319 return C.Int32 320 } 321 } else { 322 switch exprType { 323 case expr.Float: 324 return C.Float64 325 // For reducing the measure output iterator cardinality. 326 case expr.Unsigned: 327 return C.Int64 328 default: 329 return C.Int64 330 } 331 } 332 } 333 334 func initIndexVector(vector unsafe.Pointer, start, size int, stream unsafe.Pointer, device int) { 335 C.InitIndexVector((*C.uint32_t)(vector), (C.uint32_t)(start), (C.int)(size), stream, (C.int)(device)) 336 } 337 338 func (bc *oopkBatchContext) filterAction(functorType uint32, stream unsafe.Pointer, device int, inputs []C.InputVector, exp expr.Expr) { 339 numForeignTables := len(bc.foreignTableRecordIDsD) 340 // If current batch size is already 0, short circuit to avoid issuing a noop cuda call. 341 if bc.size <= 0 { 342 return 343 } 344 345 foreignTableRecordIDs := unsafe.Pointer(nil) 346 if numForeignTables > 0 { 347 foreignTableRecordIDs = unsafe.Pointer(&bc.foreignTableRecordIDsD[0].pointer) 348 } 349 if len(inputs) == 1 { 350 bc.size = int(doCGoCall(func() C.CGoCallResHandle { 351 return C.UnaryFilter(inputs[0], (*C.uint32_t)(bc.indexVectorD.getPointer()), 352 (*C.uint8_t)(bc.predicateVectorD.getPointer()), 353 (C.int)(bc.size), (**C.RecordID)(foreignTableRecordIDs), 354 (C.int)(numForeignTables), 355 (*C.uint32_t)(bc.baseCountD.getPointer()), (C.uint32_t)(bc.startRow), functorType, stream, C.int(device)) 356 })) 357 } else if len(inputs) == 2 { 358 bc.size = int(doCGoCall(func() C.CGoCallResHandle { 359 return C.BinaryFilter(inputs[0], inputs[1], (*C.uint32_t)(bc.indexVectorD.getPointer()), 360 (*C.uint8_t)(bc.predicateVectorD.getPointer()), 361 (C.int)(bc.size), (**C.RecordID)(foreignTableRecordIDs), (C.int)(numForeignTables), 362 (*C.uint32_t)(bc.baseCountD.getPointer()), (C.uint32_t)(bc.startRow), functorType, stream, C.int(device)) 363 })) 364 } 365 } 366 367 func (bc *oopkBatchContext) makeWriteToMeasureVectorAction(aggFunc C.enum_AggregateFunction, outputWidthInByte int) rootAction { 368 return func(functorType uint32, stream unsafe.Pointer, device int, inputs []C.InputVector, exp expr.Expr) { 369 // If current batch size is already 0, short circuit to avoid issuing a noop cuda call. 370 if bc.size <= 0 { 371 return 372 } 373 measureVector := utils.MemAccess(bc.measureVectorD[0].getPointer(), bc.resultSize*outputWidthInByte) 374 // write measure out to measureVectorD[1] for hll query 375 if aggFunc == C.AGGR_HLL { 376 measureVector = bc.measureVectorD[1].getPointer() 377 } 378 outputVector := makeMeasureVectorOutput(measureVector, getOutputDataType(exp.Type(), outputWidthInByte), aggFunc) 379 380 if len(inputs) == 1 { 381 doCGoCall(func() C.CGoCallResHandle { 382 return C.UnaryTransform(inputs[0], outputVector, (*C.uint32_t)(bc.indexVectorD.getPointer()), 383 (C.int)(bc.size), (*C.uint32_t)(bc.baseCountD.getPointer()), (C.uint32_t)(bc.startRow), functorType, stream, C.int(device)) 384 }) 385 } else if len(inputs) == 2 { 386 doCGoCall(func() C.CGoCallResHandle { 387 return C.BinaryTransform(inputs[0], inputs[1], outputVector, 388 (*C.uint32_t)(bc.indexVectorD.getPointer()), (C.int)(bc.size), (*C.uint32_t)(bc.baseCountD.getPointer()), (C.uint32_t)(bc.startRow), 389 functorType, stream, C.int(device)) 390 }) 391 } 392 } 393 } 394 395 func (bc *oopkBatchContext) makeWriteToDimensionVectorAction(valueOffset, nullOffset, prevResultSize int) rootAction { 396 return func(functorType uint32, stream unsafe.Pointer, device int, inputs []C.InputVector, exp expr.Expr) { 397 // If current batch size is already 0, short circuit to avoid issuing a noop cuda call. 398 if bc.size <= 0 { 399 return 400 } 401 402 dataType := getDimensionDataType(exp) 403 dataBytes := getDimensionDataBytes(exp) 404 outputVector := makeDimensionVectorOutput( 405 bc.dimensionVectorD[0].getPointer(), 406 // move dimensionVectorD to the start position of current batch 407 // dimension vector start position + bc.resultSize * dataBytes 408 // null vector start position + bc.resultSize 409 valueOffset+dataBytes*prevResultSize, 410 nullOffset+prevResultSize, 411 DataTypeToCDataType[dataType]) 412 413 if len(inputs) == 1 { 414 doCGoCall(func() C.CGoCallResHandle { 415 return C.UnaryTransform(inputs[0], outputVector, (*C.uint32_t)(bc.indexVectorD.getPointer()), 416 (C.int)(bc.size), (*C.uint32_t)(bc.baseCountD.getPointer()), (C.uint32_t)(bc.startRow), functorType, 417 stream, C.int(device)) 418 }) 419 } else if len(inputs) == 2 { 420 doCGoCall(func() C.CGoCallResHandle { 421 return C.BinaryTransform(inputs[0], inputs[1], outputVector, 422 (*C.uint32_t)(bc.indexVectorD.getPointer()), (C.int)(bc.size), 423 (*C.uint32_t)(bc.baseCountD.getPointer()), (C.uint32_t)(bc.startRow), 424 functorType, stream, C.int(device)) 425 }) 426 } 427 } 428 } 429 430 func makeCuckooHashIndex(primaryKeyData memstore.PrimaryKeyData, deviceData unsafe.Pointer) C.CuckooHashIndex { 431 var cuckooHashIndex C.CuckooHashIndex 432 cuckooHashIndex.buckets = (*C.uint8_t)(deviceData) 433 for index, seed := range primaryKeyData.Seeds { 434 cuckooHashIndex.seeds[index] = (C.uint32_t)(seed) 435 } 436 cuckooHashIndex.keyBytes = (C.int)(primaryKeyData.KeyBytes) 437 cuckooHashIndex.numHashes = (C.int)(len(primaryKeyData.Seeds)) 438 cuckooHashIndex.numBuckets = (C.int)(primaryKeyData.NumBuckets) 439 return cuckooHashIndex 440 } 441 442 func (bc *oopkBatchContext) prepareForeignRecordIDs(mainTableJoinColumnIndex int, joinTableID int, table foreignTable, 443 stream unsafe.Pointer, device int) { 444 // If current batch size is already 0, short circuit to avoid issuing a noop cuda call. 445 if bc.size <= 0 { 446 return 447 } 448 449 column := bc.columns[mainTableJoinColumnIndex] 450 inputVector := makeVectorPartySliceInput(column) 451 hashIndex := makeCuckooHashIndex(table.hostPrimaryKeyData, table.devicePrimaryKeyPtr.getPointer()) 452 doCGoCall(func() C.CGoCallResHandle { 453 return C.HashLookup( 454 inputVector, (*C.RecordID)(bc.foreignTableRecordIDsD[joinTableID].getPointer()), 455 (*C.uint32_t)(bc.indexVectorD.getPointer()), (C.int)(bc.size), (*C.uint32_t)(bc.baseCountD.getPointer()), 456 (C.uint32_t)(bc.startRow), hashIndex, stream, C.int(device)) 457 }) 458 } 459 460 // processExpression does AST tree dfs traversal and apply root action on the root level, 461 // rootAction includes filterAction, writeToDimensionVectorAction and makeWriteToMeasureVectorAction 462 func (bc *oopkBatchContext) processExpression(exp, parentExp expr.Expr, tableScanners []*TableScanner, foreignTables []*foreignTable, 463 stream unsafe.Pointer, device int, action rootAction) C.InputVector { 464 switch e := exp.(type) { 465 case *expr.ParenExpr: 466 return bc.processExpression(e.Expr, e, tableScanners, foreignTables, stream, device, action) 467 case *expr.VarRef: 468 columnIndex := tableScanners[e.TableID].ColumnsByIDs[e.ColumnID] 469 var inputVector C.InputVector 470 // main table 471 if e.TableID == 0 { 472 column := bc.columns[columnIndex] 473 inputVector = makeVectorPartySliceInput(column) 474 } else { 475 var timezoneLookup unsafe.Pointer 476 var timezoneLookupDSize int 477 switch pe := parentExp.(type) { 478 case *expr.BinaryExpr: 479 if pe.Op == expr.CONVERT_TZ { 480 timezoneLookup = bc.timezoneLookupD.getPointer() 481 timezoneLookupDSize = bc.timezoneLookupDSize 482 } 483 default: 484 } 485 inputVector = makeForeignColumnInput(columnIndex, bc.foreignTableRecordIDsD[e.TableID-1].getPointer(), *foreignTables[e.TableID-1], timezoneLookup, timezoneLookupDSize) 486 } 487 488 if action != nil { 489 action(C.Noop, stream, device, []C.InputVector{inputVector}, e) 490 return C.InputVector{} 491 } 492 return inputVector 493 case *expr.NumberLiteral, *expr.GeopointLiteral: 494 var inputVector C.InputVector 495 inputVector = makeConstantInput(e, true) 496 if action != nil { 497 action(C.Noop, stream, device, []C.InputVector{inputVector}, e) 498 return C.InputVector{} 499 } 500 return inputVector 501 case *expr.UnaryExpr: 502 inputVector := bc.processExpression(e.Expr, e, tableScanners, foreignTables, stream, device, nil) 503 functorType, exist := UnaryExprTypeToCFunctorType[e.Op] 504 if !exist { 505 functorType = C.Noop 506 } 507 508 if action != nil { 509 action(functorType, stream, device, []C.InputVector{inputVector}, e) 510 return C.InputVector{} 511 } 512 513 values, nulls := bc.allocateStackFrame() 514 dataType := getOutputDataType(e.Type(), 4) 515 var outputVector = makeScratchSpaceOutput(values.getPointer(), nulls.getPointer(), dataType) 516 517 C.UnaryTransform(inputVector, outputVector, (*C.uint32_t)(bc.indexVectorD.getPointer()), 518 (C.int)(bc.size), (*C.uint32_t)(bc.baseCountD.getPointer()), (C.uint32_t)(bc.startRow), functorType, stream, C.int(device)) 519 520 if inputVector.Type == C.ScratchSpaceInput { 521 bc.shrinkStackFrame() 522 } 523 bc.appendStackFrame(values, nulls) 524 525 return makeScratchSpaceInput(values.getPointer(), nulls.getPointer(), dataType) 526 case *expr.BinaryExpr: 527 lhsInputVector := bc.processExpression(e.LHS, e, tableScanners, foreignTables, stream, device, nil) 528 529 rhsInputVector := bc.processExpression(e.RHS, e, tableScanners, foreignTables, stream, device, nil) 530 functorType, exist := BinaryExprTypeToCFunctorType[e.Op] 531 if !exist { 532 return makeConstantInput(0.0, false) 533 } 534 535 if action != nil { 536 action(functorType, stream, device, []C.InputVector{lhsInputVector, rhsInputVector}, e) 537 return C.InputVector{} 538 } 539 540 values, nulls := bc.allocateStackFrame() 541 outputDataType := getOutputDataType(e.Type(), 4) 542 var outputVector = makeScratchSpaceOutput(values.getPointer(), nulls.getPointer(), outputDataType) 543 C.BinaryTransform(lhsInputVector, rhsInputVector, outputVector, 544 (*C.uint32_t)(bc.indexVectorD.getPointer()), (C.int)(bc.size), (*C.uint32_t)(bc.baseCountD.getPointer()), 545 (C.uint32_t)(bc.startRow), 546 functorType, stream, C.int(device)) 547 548 if rhsInputVector.Type == C.ScratchSpaceInput { 549 bc.shrinkStackFrame() 550 } 551 552 if lhsInputVector.Type == C.ScratchSpaceInput { 553 bc.shrinkStackFrame() 554 } 555 556 bc.appendStackFrame(values, nulls) 557 558 return makeScratchSpaceInput(values.getPointer(), nulls.getPointer(), outputDataType) 559 default: 560 return C.InputVector{} 561 } 562 } 563 564 func makeGeoShapeBatch(shapesLatLongs devicePointer, numShapes, totalNumPoints int) C.GeoShapeBatch { 565 var geoShapes C.GeoShapeBatch 566 geoShapes.LatLongs = (*C.uint8_t)(shapesLatLongs.getPointer()) 567 totalWords := (numShapes + 31) / 32 568 geoShapes.TotalNumPoints = (C.int32_t)(totalNumPoints) 569 geoShapes.TotalWords = (C.uint8_t)(totalWords) 570 return geoShapes 571 } 572 573 func (bc *oopkBatchContext) makeGeoPointInputVector(pointTableID int, pointColumnIndex int, foreignTables []*foreignTable) C.InputVector { 574 if pointTableID == 0 { 575 return makeVectorPartySliceInput(bc.columns[pointColumnIndex]) 576 } 577 return makeForeignColumnInput(pointColumnIndex, 578 bc.foreignTableRecordIDsD[pointTableID-1].getPointer(), 579 *foreignTables[pointTableID-1], nil, 0) 580 } 581 582 func (bc *oopkBatchContext) writeGeoShapeDim(geo *geoIntersection, 583 outputPredicate devicePointer, dimValueOffset, dimNullOffset int, sizeBeforeGeoFilter, prevResultSize int, stream unsafe.Pointer, device int) { 584 if bc.size <= 0 || geo.shapeLatLongs.isNull() { 585 return 586 } 587 588 // geo dimension always take 1 byte and has type uint8 589 // compiler should have checked the number of geo shapes for join is less than 256 590 var dimensionOutputVector C.DimensionOutputVector 591 dimensionVector := bc.dimensionVectorD[0].getPointer() 592 // move dimensionVectorD to the start position of current batch 593 // dimension vector start position + prevResultSize * dataBytes 594 // null vector start position + prevResultSize 595 dimensionOutputVector.DimValues = (*C.uint8_t)(utils.MemAccess(dimensionVector, dimValueOffset+prevResultSize)) 596 dimensionOutputVector.DimNulls = (*C.uint8_t)(utils.MemAccess(dimensionVector, dimNullOffset+prevResultSize)) 597 dimensionOutputVector.DataType = C.Uint8 598 599 totalWords := (geo.numShapes + 31) / 32 600 doCGoCall(func() C.CGoCallResHandle { 601 return C.WriteGeoShapeDim((C.int)(totalWords), dimensionOutputVector, (C.int)(sizeBeforeGeoFilter), 602 (*C.uint32_t)(outputPredicate.getPointer()), stream, (C.int)(device)) 603 }) 604 } 605 606 func (bc *oopkBatchContext) geoIntersect(geo *geoIntersection, pointColumnIndex int, 607 foreignTables []*foreignTable, 608 outputPredicte devicePointer, stream unsafe.Pointer, device int) { 609 // If current batch size is already 0, short circuit to avoid issuing a noop cuda call. 610 if bc.size <= 0 || geo.shapeLatLongs.isNull() { 611 return 612 } 613 614 numForeignTables := len(bc.foreignTableRecordIDsD) 615 foreignTableRecordIDs := unsafe.Pointer(nil) 616 if numForeignTables > 0 { 617 foreignTableRecordIDs = unsafe.Pointer(&bc.foreignTableRecordIDsD[0].pointer) 618 } 619 geoShapes := makeGeoShapeBatch(geo.shapeLatLongs, geo.numShapes, geo.totalNumPoints) 620 points := bc.makeGeoPointInputVector(geo.pointTableID, pointColumnIndex, foreignTables) 621 bc.size = int(doCGoCall(func() C.CGoCallResHandle { 622 return C.GeoBatchIntersects( 623 geoShapes, points, (*C.uint32_t)(bc.indexVectorD.getPointer()), 624 (C.int)(bc.size), (C.uint32_t)(bc.startRow), (**C.RecordID)(foreignTableRecordIDs), 625 (C.int)(numForeignTables), (*C.uint32_t)(outputPredicte.getPointer()), 626 (C.bool)(geo.inOrOut), stream, (C.int)(device)) 627 })) 628 } 629 630 func (bc *oopkBatchContext) hll(numDims common.DimCountsPerDimWidth, isLastBatch bool, stream unsafe.Pointer, device int) ( 631 hllVector, dimRegCount devicePointer, hllVectorSize int64) { 632 prevDimOut := makeDimensionColumnVector(bc.dimensionVectorD[0].getPointer(), bc.hashVectorD[0].getPointer(), 633 bc.dimIndexVectorD[0].getPointer(), numDims, bc.resultCapacity) 634 curDimOut := makeDimensionColumnVector(bc.dimensionVectorD[1].getPointer(), bc.hashVectorD[1].getPointer(), 635 bc.dimIndexVectorD[1].getPointer(), numDims, bc.resultCapacity) 636 prevValuesOut, curValuesOut := (*C.uint32_t)(bc.measureVectorD[0].getPointer()), (*C.uint32_t)(bc.measureVectorD[1].getPointer()) 637 bc.resultSize = int(doCGoCall(func() C.CGoCallResHandle { 638 return C.HyperLogLog(prevDimOut, curDimOut, 639 prevValuesOut, curValuesOut, 640 (C.int)(bc.resultSize), (C.int)(bc.size), (C.bool)(isLastBatch), 641 (**C.uint8_t)(unsafe.Pointer(&hllVector.pointer)), (*C.size_t)(unsafe.Pointer(&hllVectorSize)), 642 (**C.uint16_t)(unsafe.Pointer(&dimRegCount.pointer)), stream, (C.int)(device)) 643 })) 644 // TODO: we also need a way to report this allocation in C++ code. Maybe can be done via calling a golang function from c++ 645 hllVector.device = device 646 hllVector.allocated = true 647 dimRegCount.device = device 648 dimRegCount.allocated = true 649 return 650 } 651 652 func (bc *oopkBatchContext) sortByKey(numDims common.DimCountsPerDimWidth, stream unsafe.Pointer, device int) { 653 keys := makeDimensionColumnVector(bc.dimensionVectorD[0].getPointer(), bc.hashVectorD[0].getPointer(), 654 bc.dimIndexVectorD[0].getPointer(), numDims, bc.resultCapacity) 655 doCGoCall(func() C.CGoCallResHandle { 656 // sort the previous result with current batch together 657 return C.Sort(keys, (C.int)(bc.resultSize+bc.size), stream, C.int(device)) 658 }) 659 } 660 661 func (bc *oopkBatchContext) reduceByKey(numDims common.DimCountsPerDimWidth, valueWidth int, aggFunc C.enum_AggregateFunction, stream unsafe.Pointer, 662 device int) { 663 inputKeys := makeDimensionColumnVector( 664 bc.dimensionVectorD[0].getPointer(), bc.hashVectorD[0].getPointer(), bc.dimIndexVectorD[0].getPointer(), numDims, bc.resultCapacity) 665 outputKeys := makeDimensionColumnVector( 666 bc.dimensionVectorD[1].getPointer(), bc.hashVectorD[1].getPointer(), bc.dimIndexVectorD[1].getPointer(), numDims, bc.resultCapacity) 667 inputValues, outputValues := (*C.uint8_t)(bc.measureVectorD[0].getPointer()), (*C.uint8_t)(bc.measureVectorD[1].getPointer()) 668 bc.resultSize = int(doCGoCall(func() C.CGoCallResHandle { 669 return C.Reduce(inputKeys, inputValues, outputKeys, outputValues, (C.int)(valueWidth), (C.int)(bc.resultSize+bc.size), aggFunc, 670 stream, C.int(device)) 671 })) 672 } 673 674 func (bc *oopkBatchContext) expand(numDims common.DimCountsPerDimWidth, lenWanted int, stream unsafe.Pointer, device int) { 675 inputKeys := makeDimensionColumnVector( 676 bc.dimensionVectorD[0].getPointer(), bc.hashVectorD[0].getPointer(), bc.dimIndexVectorD[0].getPointer(), numDims, bc.resultCapacity) 677 outputKeys := makeDimensionColumnVector( 678 bc.dimensionVectorD[1].getPointer(), bc.hashVectorD[1].getPointer(), bc.dimIndexVectorD[1].getPointer(), numDims, bc.resultCapacity) 679 680 bc.resultSize = int(doCGoCall(func() C.CGoCallResHandle { 681 return C.Expand(inputKeys, outputKeys, (*C.uint32_t)(bc.baseCountD.getPointer()), (*C.uint32_t)(bc.indexVectorD.getPointer()), 682 C.int(lenWanted), C.int(bc.resultSize), stream, C.int(device)) 683 })) 684 } 685 686 func (bc *oopkBatchContext) allocateStackFrame() (values, nulls devicePointer) { 687 // width bytes * bc.size (value buffer) + 1 byte * bc.size (null buffer) 688 valuesPointer := deviceAllocate((4+1)*bc.size, bc.device) 689 nullsPointer := valuesPointer.offset(4 * bc.size) 690 return valuesPointer, nullsPointer 691 } 692 693 func (bc *oopkBatchContext) shrinkStackFrame() { 694 var stackFrame [2]devicePointer 695 stackFrame, bc.exprStackD = bc.exprStackD[len(bc.exprStackD)-1], bc.exprStackD[:len(bc.exprStackD)-1] 696 deviceFreeAndSetNil(&stackFrame[0]) 697 } 698 699 func (bc *oopkBatchContext) appendStackFrame(values, nulls devicePointer) { 700 bc.exprStackD = append(bc.exprStackD, [2]devicePointer{values, nulls}) 701 } 702 703 func (qc *AQLQueryContext) createCutoffTimeFilter(cutoff uint32) expr.Expr { 704 column := &expr.VarRef{ 705 Val: qc.Query.TimeFilter.Column, 706 ExprType: expr.Unsigned, 707 TableID: 0, 708 // time column is always 0 709 ColumnID: 0, 710 } 711 712 return &expr.BinaryExpr{ 713 ExprType: expr.Boolean, 714 Op: expr.GTE, 715 LHS: column, 716 RHS: &expr.NumberLiteral{ 717 Int: int(cutoff), 718 Expr: strconv.FormatInt(int64(cutoff), 10), 719 ExprType: expr.Unsigned, 720 }, 721 } 722 } 723 724 // doCGoCall does the cgo call by converting CGoCallResHandle to C.int and *C.char and calls doCGoCall. 725 // The reason to have this wrapper is because CGo types are bound to package name, thereby even C.int are different types 726 // under different packages. 727 func doCGoCall(f func() C.CGoCallResHandle) uintptr { 728 return cgoutils.DoCGoCall(func() (uintptr, unsafe.Pointer) { 729 ret := f() 730 return uintptr(ret.res), unsafe.Pointer(ret.pStrErr) 731 }) 732 } 733 734 // bootstrapDevice is the go wrapper of BootstrapDevice. It will panic and crash the server if any exceptions are thrown 735 // in this function. 736 func bootstrapDevice() { 737 doCGoCall(func() C.CGoCallResHandle { 738 return C.BootstrapDevice() 739 }) 740 }