diff --git a/execgen/main.go b/execgen/main.go index 3f0eacc..0f2efa9 100644 --- a/execgen/main.go +++ b/execgen/main.go @@ -184,7 +184,7 @@ func main() { if err != nil { // Write out incorrect source for easier debugging. b = buf.Bytes() - fmt.Printf("Code formatting failed with Go parse error: %s", out, err) + fmt.Printf("Code formatting failed with Go parse error: %s", err) } wr := os.Stdout diff --git a/hashjoin.go b/hashjoin.go new file mode 100644 index 0000000..af93d53 --- /dev/null +++ b/hashjoin.go @@ -0,0 +1,393 @@ +package exectoy + +// todo(changangela) hash table should be able to rehash or determine bucket size dynamically +const hashTableBucketSize = 1 << 16 + +type hashTableInt struct { + first []int + next []int + + keys intColumn + values []intColumn + + nOutCols int + count int + bucketSize int + + // allocated represents the reserved number of keys that the hashTableInt + // can store in its current state. + allocated int +} + +func makeHashTableInt(bucketSize int, nOutCols int) *hashTableInt { + values := make([]intColumn, nOutCols) + for i := 0; i < nOutCols; i++ { + values[i] = make(intColumn, 1) + } + + hashTable := &hashTableInt{ + first: make([]int, bucketSize), + next: make([]int, 1), + keys: make([]int, 1), + values: values, + nOutCols: nOutCols, + bucketSize: bucketSize, + allocated: 1, + } + + return hashTable +} + +func (hashTable *hashTableInt) hashInt(key int) int { + // todo(changangela) replace with a better hash function or use a library + return key % hashTable.bucketSize +} + +func (hashTable *hashTableInt) grow(amount int) { + hashTable.next = append(hashTable.next, make([]int, amount)...) + hashTable.keys = append(hashTable.keys, make(intColumn, amount)...) + + for i := 0; i < hashTable.nOutCols; i++ { + hashTable.values[i] = append(hashTable.values[i], make(intColumn, amount)...) + } + hashTable.allocated += amount +} + +func (hashTable *hashTableInt) insert(hashKey int, key int) (id int) { + hashTable.count++ + + id = hashTable.count + hashTable.next[id] = hashTable.first[hashKey] + hashTable.first[hashKey] = id + hashTable.keys[id] = key + + return id +} + +// hashJoinBuilder is a naive builder that operates on the build relation batch +// by batch and builds the hash table incrementally. +type hashJoinBuilder struct { + hashTable *hashTableInt + + batchSize int + + bucket []int + groupId []int +} + +func makeHashJoinBuilder(batchSize int, hashTable *hashTableInt) *hashJoinBuilder { + return &hashJoinBuilder{ + hashTable: hashTable, + batchSize: batchSize, + bucket: make([]int, batchSize), + groupId: make([]int, batchSize), + } +} + +func (builder *hashJoinBuilder) computeBucket(eqCol intColumn) { + for i := 0; i < builder.batchSize; i++ { + builder.bucket[i] = builder.hashTable.hashInt(eqCol[i]) + } +} + +func (builder *hashJoinBuilder) insertBatch(eqCol intColumn) { + builder.hashTable.grow(builder.batchSize) + + for i := 0; i < builder.batchSize; i++ { + builder.groupId[i] = builder.hashTable.insert(builder.bucket[i], eqCol[i]) + } +} + +func (builder *hashJoinBuilder) spread(outCol intColumn, valIdx int) { + valCol := builder.hashTable.values[valIdx] + + for i:= 0; i < builder.batchSize; i++ { + valCol[builder.groupId[i]] = outCol[i] + } +} + +func (builder *hashJoinBuilder) start(flow dataFlow, eqColIdx int, outCols []int) { + eqCol := flow.b[eqColIdx].(intColumn)[:batchRowLen] + + builder.computeBucket(eqCol) + builder.insertBatch(eqCol) + + for idx, colIdx := range outCols { + outCol := flow.b[colIdx].(intColumn)[:batchRowLen] + builder.spread(outCol, idx) + } +} + +type hashJoinProber struct { + hashTable *hashTableInt + + batchSize int + + bucket []int + groupId []int + toCheck []int +} + +func makeHashJoinProber(batchSize int, hashTable *hashTableInt) *hashJoinProber { + return &hashJoinProber{ + hashTable: hashTable, + batchSize: batchSize, + + bucket: make([]int, batchSize), + groupId: make([]int, batchSize), + toCheck: make([]int, batchSize), + } +} + +func (prober *hashJoinProber) computeBucket(eqCol intColumn) { + for i := 0; i < prober.batchSize; i++ { + prober.bucket[i] = prober.hashTable.hashInt(eqCol[i]) + } +} + +func (prober *hashJoinProber) lookupInitial() { + for i := 0; i < prober.batchSize; i++ { + prober.groupId[i] = prober.hashTable.first[prober.bucket[i]] + prober.toCheck[i] = i + } +} + +func (prober *hashJoinProber) check(eqCol intColumn) int { + nToCheck := len(prober.toCheck) + differs := make([]int, 0) + + for i := 0; i < nToCheck; i++ { + // id of 0 is reversed to represent end of next chain + if id := prober.groupId[prober.toCheck[i]]; id != 0 { + if prober.hashTable.keys[id] != eqCol[prober.toCheck[i]] { + differs = append(differs, prober.toCheck[i]) + } + } + } + + prober.toCheck = differs + + return len(prober.toCheck) +} + +func (prober *hashJoinProber) findNext(nToCheck int) { + for i := 0; i < nToCheck; i++ { + prober.groupId[prober.toCheck[i]] = prober.hashTable.next[prober.groupId[prober.toCheck[i]]] + } +} + +func (prober *hashJoinProber) start(flow dataFlow, eqColIdx int, outCols []int, outFlow *dataFlow) { + eqCol := flow.b[eqColIdx].(intColumn)[:batchRowLen] + + prober.computeBucket(eqCol) + prober.lookupInitial() + + nToCheck := len(prober.toCheck) + + for nToCheck > 0 { + nToCheck = prober.check(eqCol) + prober.findNext(nToCheck) + } + + groupId := prober.groupId + results := make([]int, 0) + resultsIdx := make([]int, 0) + + for i := 0; i < flow.n; i++ { + if groupId[i] != 0 { + results = append(results, groupId[i]) + resultsIdx = append(resultsIdx, i) + } + } + + nBuildOutCols := prober.hashTable.nOutCols + nResults := len(results) + + for j := 0; j < nBuildOutCols; j++ { + outCol := outFlow.b[j].(intColumn)[:batchRowLen] + valCol := prober.hashTable.values[j] + + for i := 0; i < nResults; i++ { + outCol[i] = valCol[results[i]] + } + } + + for j, colIdx := range outCols { + outCol := outFlow.b[j + nBuildOutCols].(intColumn)[:batchRowLen] + valCol := flow.b[colIdx].(intColumn)[:batchRowLen] + + for i := 0; i < nResults; i++ { + outCol[i] = valCol[resultsIdx[i]] + } + } + + outFlow.n = nResults +} + +// hashJoinGroupBuilder is an alternative builder that pre-loads all build +// relation batches before building the hash table. +type hashJoinGroupBuilder struct { + hashTable *hashTableInt + + totalSize int + + // bucket stores the hashed value of every key in the build relation, and + // thereby contributes additional memory in comparison to the + // hashJoinBuilder. + bucket []int +} + +func (hashTable *hashTableInt) growGroup(amount int) { + hashTable.keys = append(hashTable.keys, make(intColumn, amount)...) + + for i := 0; i < hashTable.nOutCols; i++ { + hashTable.values[i] = append(hashTable.values[i], make(intColumn, amount)...) + } + + hashTable.allocated += amount +} + +func (hashTable *hashTableInt) growNext(amount int) { + hashTable.next = append(hashTable.next, make([]int, amount)...) +} + +func (hashTable *hashTableInt) insertKey(hashKey int, id int) { + hashTable.count++ + hashTable.next[id] = hashTable.first[hashKey] + hashTable.first[hashKey] = id +} + +func makeHashJoinGroupBuilder(hashTable *hashTableInt) *hashJoinGroupBuilder { + return &hashJoinGroupBuilder{ + hashTable: hashTable, + } +} + +func (builder *hashJoinGroupBuilder) insertBatch(flow dataFlow, eqColIdx int, outCols []int, batchSize int) { + builder.hashTable.growGroup(batchSize) + + eqCol := flow.b[eqColIdx].(intColumn)[:batchRowLen] + + for i := 0; i < batchSize; i++ { + builder.hashTable.keys[i + builder.totalSize + 1] = eqCol[i] + } + + for idx, colIdx := range outCols { + outCol := flow.b[colIdx].(intColumn)[:batchRowLen] + builder.spread(outCol, idx, batchSize) + } + + builder.totalSize += batchSize +} + +func (builder *hashJoinGroupBuilder) spread(outCol intColumn, valIdx int, batchSize int) { + valCol := builder.hashTable.values[valIdx] + + for i:= 0; i < batchSize; i++ { + valCol[i + builder.totalSize + 1] = outCol[i] + } +} + +func (builder *hashJoinGroupBuilder) computeBucket() { + builder.bucket = make([]int, builder.totalSize) + + for i := 0; i < builder.totalSize; i++ { + builder.bucket[i] = builder.hashTable.hashInt(builder.hashTable.keys[i + 1]) + } +} + +func (builder *hashJoinGroupBuilder) insertKeys() { + builder.hashTable.growNext(builder.totalSize) + + for i := 0; i < builder.totalSize; i++ { + builder.hashTable.insertKey(builder.bucket[i], i + 1) + } +} + +func (builder *hashJoinGroupBuilder) start(source ExecOp, eqColIdx int, outCols []int) { + for{ + flow := source.Next() + batchSize := flow.n + + if batchSize == 0 { + break + } + + builder.insertBatch(flow, eqColIdx, outCols, batchSize) + } + + builder.computeBucket() + builder.insertKeys() +} + +// hashJoinerIntInt performs a hashJoin joining on two integer columns where the +// left table represents the build relation. It does not work with N - N joins. +type hashJoinerIntInt struct { + hashTable *hashTableInt + + left ExecOp + right ExecOp + + leftEqColIdx int + rightEqColIdx int + + leftCols []int + rightCols []int + + outFlow dataFlow +} + +func (hashJoiner *hashJoinerIntInt) Init() { + nOutputCols := len(hashJoiner.leftCols) + len(hashJoiner.rightCols) + if nOutputCols == 0 { + panic("no output cols") + } + + hashJoiner.outFlow.b = make(batch, nOutputCols) + for i := range hashJoiner.outFlow.b { + hashJoiner.outFlow.b[i] = make(intColumn, batchRowLen) + } + + // todo(changangela): we currently assume build relation has unique join + // column and is on the left + + hashJoiner.hashTable = makeHashTableInt(hashTableBucketSize, len(hashJoiner.leftCols)) + + hashJoiner.build() +} + +func (hashJoiner *hashJoinerIntInt) Next() dataFlow { + flow := hashJoiner.right.Next() + batchSize := flow.n + + if batchSize == 0 { + return flow + } + + batchProber := makeHashJoinProber(batchSize, hashJoiner.hashTable) + batchProber.start(flow, hashJoiner.rightEqColIdx, hashJoiner.rightCols, &hashJoiner.outFlow) + + return hashJoiner.outFlow +} + +// build performs the build phase of the hash join using the left relation. +// Different builders used different heuristics for the build phase allowing us +// to evaluate cpu-memory trade-offs. +func (hashJoiner *hashJoinerIntInt) build() { + // build phase using the hashJoinGroupBuilder + builder := makeHashJoinGroupBuilder(hashJoiner.hashTable) + builder.start(hashJoiner.left, hashJoiner.leftEqColIdx, hashJoiner.leftCols) + + // build phase using the hashJoinBuilder + //for { + // flow := hashJoiner.left.Next() + // batchSize := flow.n + // + // if batchSize == 0 { + // return + // } + // + // batchBuilder := makeHashJoinBuilder(batchSize, hashJoiner.hashTable) + // batchBuilder.start(flow, hashJoiner.leftEqColIdx, hashJoiner.leftCols) + //} +} \ No newline at end of file diff --git a/mergejoin.go b/mergejoin.go index a9ec092..b1d7018 100644 --- a/mergejoin.go +++ b/mergejoin.go @@ -183,7 +183,7 @@ func (m *mergeJoinIntIntOp) bufferMatchGroup(val int, flow dataFlow, colIdx int, // It's hard because this whole process can span batch boundaries. // The algorithm should be: // for each col: - // add value to buffer until group's over or batch ends. + // add value to buffer until groupId's over or batch ends. // if batch ended early, repeat. for i, c := range cols { batchBuf[i] = append(batchBuf[i].(intColumn), flow.b[c].(intColumn)[matchIdx]) diff --git a/operator.go b/operator.go index bf3c5d8..90de8c4 100644 --- a/operator.go +++ b/operator.go @@ -71,7 +71,70 @@ func (s *repeatableBatchSource) Init() { } } +type finiteBatchSource struct { + s *repeatableBatchSource + usableCount int +} + +func (f *finiteBatchSource) Init() { + f.s.Init() +} + +func (f *finiteBatchSource) Next() dataFlow { + f.usableCount-- + if f.usableCount > 0 { + return f.s.Next() + } else { + return dataFlow{} + } +} + +type uniformDistinctBatchSource struct { + numOutputCols int + internalCol intColumn + internalSel intColumn + usableCount int + curIdx int +} + +func (u *uniformDistinctBatchSource) Init() { + u.internalSel = make(intColumn, batchRowLen) + + u.internalCol = make(intColumn, hashTableBucketSize) + for i := 0; i < hashTableBucketSize; i++ { + u.internalCol[i] = i; + } +} + +func (u *uniformDistinctBatchSource) Next() dataFlow { + u.usableCount-- + if u.usableCount > 0 { + batch := make(batch, u.numOutputCols) + for i := 0; i < u.numOutputCols; i++ { + batch[i] = intColumn(u.internalCol[u.curIdx: u.curIdx + batchRowLen]) + } + + u.curIdx += batchRowLen + if u.curIdx >= hashTableBucketSize { + u.curIdx = 0 + } + + flow := dataFlow{ + b: batch, + sel: u.internalSel, + useSel: false, + n: batchRowLen, + } + + return flow + } else { + return dataFlow{} + } +} + var _ ExecOp = &repeatableBatchSource{} +var _ ExecOp = &finiteBatchSource{} +var _ ExecOp = &uniformDistinctBatchSource{} /* diff --git a/operator_bench_test.go b/operator_bench_test.go index 4b8380c..58c33ca 100644 --- a/operator_bench_test.go +++ b/operator_bench_test.go @@ -1,19 +1,47 @@ package exectoy import ( + "fmt" "math/rand" "testing" ) -func randomizeSource(s *repeatableBatchSource) { +func randomizeSource(s *repeatableBatchSource, upper int) { seed := int64(12345) rngesus := rand.New(rand.NewSource(seed)) for i := 0; i < s.numOutputCols*batchRowLen; i++ { - s.internalBatch[i/batchRowLen].(intColumn)[i%batchRowLen] = rngesus.Int() % 128 + s.internalBatch[i/batchRowLen].(intColumn)[i%batchRowLen] = rngesus.Int() % upper } } +func makeRandomizeSource(numCols, upper int) ExecOp { + source := &repeatableBatchSource{ + numOutputCols: numCols, + } + source.Init() + randomizeSource(source, upper) + return source +} + +func makeRandomizeFiniteSource(numCols, useCount, upper int) ExecOp { + source := makeRandomizeSource(numCols, upper) + finiteSource := &finiteBatchSource{ + s: source.(*repeatableBatchSource), + usableCount: useCount, + } + return finiteSource +} + +func makeUniformDistinctSource(numCols int, useCount int) ExecOp { + source := &uniformDistinctBatchSource { + numOutputCols: numCols, + usableCount: useCount, + } + source.Init() + return source +} + func randomizeTupleBatchSouce(s *repeatableTupleBatchSource) { seed := int64(12345) rngesus := rand.New(rand.NewSource(seed)) @@ -27,7 +55,7 @@ func BenchmarkZeroOp(b *testing.B) { var source repeatableBatchSource source.numOutputCols = 4 source.Init() - randomizeSource(&source) + randomizeSource(&source, 128) zeroOp := &zeroIntOp{ input: &source, @@ -45,7 +73,7 @@ func BenchmarkFilterIntLessThanConstOperator(b *testing.B) { var source repeatableBatchSource source.numOutputCols = 4 source.Init() - randomizeSource(&source) + randomizeSource(&source, 128) var fop selLTIntIntConstOp fop.input = &source @@ -64,7 +92,7 @@ func BenchmarkProjPlusIntIntConst(b *testing.B) { var source repeatableBatchSource source.numOutputCols = 4 source.Init() - randomizeSource(&source) + randomizeSource(&source, 128) var rop projPlusIntIntConstOp rop.input = &source @@ -125,7 +153,7 @@ func BenchmarkProjPlusIntInt(b *testing.B) { var source repeatableBatchSource source.numOutputCols = 4 source.Init() - randomizeSource(&source) + randomizeSource(&source, 128) var rop projPlusIntIntOp rop.input = &source @@ -145,7 +173,7 @@ func BenchmarkRenderChain(b *testing.B) { var source repeatableBatchSource source.numOutputCols = 4 source.Init() - randomizeSource(&source) + randomizeSource(&source, 128) var rop projPlusIntIntOp rop.input = &source @@ -176,7 +204,7 @@ func BenchmarkSelectIntPlusConstLTInt(b *testing.B) { var source repeatableBatchSource source.numOutputCols = 4 source.Init() - randomizeSource(&source) + randomizeSource(&source, 128) // first project n -> n+1 projOp := projPlusIntIntConstOp{ @@ -219,7 +247,7 @@ func BenchmarkSortedDistinct(b *testing.B) { numOutputCols: 5, } source.Init() - randomizeSource(source) + randomizeSource(source, 128) zeroOp := &zeroIntOp{ input: source, @@ -253,13 +281,13 @@ func BenchmarkMergeJoin(b *testing.B) { numOutputCols: 4, } sourceL.Init() - randomizeSource(sourceL) + randomizeSource(sourceL, 128) sourceR := &repeatableBatchSource{ numOutputCols: 4, } sourceR.Init() - randomizeSource(sourceR) + randomizeSource(sourceR, 128) mj := &mergeJoinIntIntOp{ left: sourceL, @@ -276,3 +304,66 @@ func BenchmarkMergeJoin(b *testing.B) { mj.Next() } } + +func BenchmarkHashJoin(b *testing.B) { + // todo(changangela) since all numbers are between 0-127, the hash table is + // not being efficiently used + + bcs := []struct { + name string + numCols int + sourceL ExecOp + sourceR ExecOp + reset func(nBatches int, sourceL ExecOp, sourceR ExecOp) + } { + { + // left and right source join columns are both pseudo random integers + name: "random source", + numCols: 4, + sourceL: makeRandomizeFiniteSource(4, 0, hashTableBucketSize), + sourceR: makeRandomizeSource(4, hashTableBucketSize), + reset: func(nBatches int, sourceL ExecOp, sourceR ExecOp) { + sourceL.(*finiteBatchSource).usableCount = nBatches + }, + }, + { + // left and right source join columns are both uniformly + // incrementing integers (1:1 join) + name: "uniformly distinct source", + numCols: 4, + sourceL: makeUniformDistinctSource(4, 0), + sourceR: makeUniformDistinctSource(4, 0), + reset: func(nBatches int, sourceL ExecOp, sourceR ExecOp) { + sourceL.(*uniformDistinctBatchSource).usableCount = nBatches + sourceR.(*uniformDistinctBatchSource).usableCount = nBatches + }, + }, + } + + for _, bc := range bcs { + b.Run(fmt.Sprintf("name=%v", bc.name), func(b *testing.B) { + for _, nBatches := range []int{0, 1 << 2, 1 << 4, 1 << 8, 1 << 12, 1 << 16} { + b.Run(fmt.Sprintf("rows=%d", nBatches*batchRowLen), func(b *testing.B) { + b.SetBytes(int64(8 * nBatches * batchRowLen * bc.numCols * 2)) + b.ResetTimer() + for i := 0; i < b.N; i++ { + bc.reset(nBatches, bc.sourceL, bc.sourceR) + hj := &hashJoinerIntInt{ + left: bc.sourceL, + right: bc.sourceR, + leftEqColIdx: 1, + rightEqColIdx: 1, + leftCols: []int{1, 2}, + rightCols: []int{2, 3}, + } + hj.Init() + + for i := 0; i < nBatches; i++ { + hj.Next() + } + } + }) + } + }) + } +} diff --git a/operator_test.go b/operator_test.go index dda3c02..413aae3 100644 --- a/operator_test.go +++ b/operator_test.go @@ -228,3 +228,153 @@ func TestMergeJoin(t *testing.T) { } } } + +func TestHashTable(t *testing.T) { + tcs := []struct { + batches [][]int + bucketSize int + expectedFirst []int + expectedNext []int + expectedKeys []int + } { + { + batches: [][]int { + []int{0, 1, 2, 3, 4}, + []int {5, 6, 7, 8, 9}, + }, + bucketSize: 5, + expectedFirst: []int {6, 7, 8, 9, 10}, + expectedNext: []int {0, 0, 0, 0, 0, 0, 1, 2, 3, 4, 5}, + expectedKeys: []int {0, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9}, + }, + } + + for _, tc := range tcs { + ht := makeHashTableInt(tc.bucketSize, 0) + for _, batch := range tc.batches { + batchSize := len(batch) + builder := makeHashJoinBuilder(batchSize, ht) + builder.computeBucket(batch) + builder.insertBatch(batch) + } + + if !reflect.DeepEqual(tc.expectedFirst, ht.first[:len(tc.expectedFirst)]) { + t.Errorf("expected first: %v, got %v", tc.expectedFirst, ht.first) + } + + if !reflect.DeepEqual(tc.expectedNext, ht.next[:len(tc.expectedNext)]) { + t.Errorf("expected next: %v, got %v", tc.expectedNext, ht.next) + } + + if !reflect.DeepEqual(tc.expectedKeys, []int(ht.keys)[:len(tc.expectedKeys)]) { + t.Errorf("expected keys: %v, got %v", tc.expectedKeys, ht.keys) + } + } +} + +func TestHashJoin(t *testing.T) { + tcs := []struct { + leftEqColIdx int + rightEqColIdx int + leftNCols int + rightNCols int + leftCols []int + rightCols []int + leftTuples []tuple + rightTuples []tuple + expected []tuple + }{ + { + leftEqColIdx: 0, + rightEqColIdx: 1, + leftNCols: 4, + rightNCols: 4, + leftTuples: []tuple{ + tuple{1, 2, 3, 4}, + tuple{5, 2, 3, 5}, + }, + rightTuples: []tuple{ + tuple{1, 5, 3, 4}, + tuple{1, 6, 3, 5}, + }, + leftCols: []int{0, 1}, + rightCols: []int{1, 2}, + expected: []tuple{ + tuple{5, 2, 5, 3}, + }, + }, + { + leftEqColIdx: 0, + rightEqColIdx: 1, + leftNCols: 4, + rightNCols: 4, + leftTuples: []tuple{ + tuple{1, 2, 3, 4}, + tuple{5, 3, 3, 5}, + tuple{4, 2, 1, 8}, + }, + rightTuples: []tuple{ + tuple{1, 5, 3, 4}, + tuple{1, 6, 3, 5}, + tuple{1, 4, 5, 6}, + tuple{1, 5, 4, 4}, + }, + leftCols: []int{0, 1}, + rightCols: []int{1, 2}, + expected: []tuple{ + tuple{5, 3, 5, 3}, + tuple{4, 2, 4, 5}, + tuple{5, 3, 5, 4}, + }, + }, + } + + for _, tc := range tcs { + leftSource := &staticTupleSource{ + tuples: tc.leftTuples, + } + + lColOp := &columnarizeOp{ + input: leftSource, + numCols: tc.leftNCols, + } + lColOp.Init() + + rightSource := &staticTupleSource{ + tuples: tc.rightTuples, + } + rColOp := &columnarizeOp{ + input: rightSource, + numCols: tc.rightNCols, + } + rColOp.Init() + + hj := &hashJoinerIntInt{ + left: lColOp, + right: rColOp, + leftEqColIdx: tc.leftEqColIdx, + rightEqColIdx: tc.rightEqColIdx, + leftCols: tc.leftCols, + rightCols: tc.rightCols, + } + hj.Init() + + mop := &materializeOp{ + input: hj, + cols: []int{0, 1, 2, 3}, + } + mop.Init() + + var actual []tuple + for { + tuple := mop.NextTuple() + if tuple == nil { + break + } + actual = append(actual, tuple) + } + if !reflect.DeepEqual(tc.expected, actual) { + t.Errorf("expected %v, got %v", tc.expected, actual) + } + } +} \ No newline at end of file