TestMemBlockIterator.cc 3.0 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110
  1. /**
  2. * Licensed to the Apache Software Foundation (ASF) under one
  3. * or more contributor license agreements. See the NOTICE file
  4. * distributed with this work for additional information
  5. * regarding copyright ownership. The ASF licenses this file
  6. * to you under the Apache License, Version 2.0 (the
  7. * "License"); you may not use this file except in compliance
  8. * with the License. You may obtain a copy of the License at
  9. *
  10. * http://www.apache.org/licenses/LICENSE-2.0
  11. *
  12. * Unless required by applicable law or agreed to in writing, software
  13. * distributed under the License is distributed on an "AS IS" BASIS,
  14. * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  15. * See the License for the specific language governing permissions and
  16. * limitations under the License.
  17. */
  18. #include "lib/commons.h"
  19. #include "test_commons.h"
  20. #include "lib/MapOutputSpec.h"
  21. #include "lib/MemoryBlock.h"
  22. namespace NativeTask {
  23. TEST(MemoryBlockIterator, test) {
  24. const uint32_t BUFFER_LENGTH = 100;
  25. char * bytes = new char[BUFFER_LENGTH];
  26. MemoryBlock block(bytes, BUFFER_LENGTH);
  27. const uint32_t KV_SIZE = 60;
  28. block.allocateKVBuffer(KV_SIZE);
  29. block.allocateKVBuffer(KV_SIZE);
  30. MemBlockIterator iter(&block);
  31. uint32_t keyCount = 0;
  32. while (iter.next()) {
  33. KVBuffer * kv = iter.getKVBuffer();
  34. ASSERT_EQ(block.getKVBuffer(keyCount), kv);
  35. keyCount++;
  36. }
  37. delete [] bytes;
  38. }
  39. class MemoryBlockFactory {
  40. public:
  41. static MemoryBlock * create(std::vector<int> & keys) {
  42. const uint32_t BUFFER_LENGTH = 1000;
  43. char * bytes = new char[BUFFER_LENGTH];
  44. MemoryBlock * block1 = new MemoryBlock(bytes, BUFFER_LENGTH);
  45. const uint32_t KV_SIZE = 16;
  46. for (uint32_t i = 0; i < keys.size(); i++) {
  47. uint32_t index = keys[i];
  48. KVBuffer * kv = block1->allocateKVBuffer(KV_SIZE);
  49. kv->keyLength = 4;
  50. kv->valueLength = 4;
  51. uint32_t * key = (uint32_t *)kv->getKey();
  52. *key = bswap(index);
  53. }
  54. return block1;
  55. }
  56. };
  57. TEST(MemoryBlockIterator, compare) {
  58. std::vector<int> vector1;
  59. vector1.push_back(2);
  60. vector1.push_back(4);
  61. vector1.push_back(6);
  62. std::vector<int> vector2;
  63. vector2.push_back(1);
  64. vector2.push_back(3);
  65. vector2.push_back(5);
  66. ComparatorPtr bytesComparator = NativeTask::get_comparator(BytesType, NULL);
  67. MemoryBlock * block1 = MemoryBlockFactory::create(vector1);
  68. MemoryBlock * block2 = MemoryBlockFactory::create(vector2);
  69. block1->sort(CPPSORT, bytesComparator);
  70. block2->sort(CPPSORT, bytesComparator);
  71. MemBlockIterator * iter1 = new MemBlockIterator(block1);
  72. MemBlockIterator * iter2 = new MemBlockIterator(block2);
  73. MemBlockComparator comparator(bytesComparator);
  74. ASSERT_EQ(false, comparator(iter1, iter2));
  75. iter1->next();
  76. ASSERT_EQ(true, comparator(iter1, iter2));
  77. iter2->next();
  78. ASSERT_EQ(false, comparator(iter1, iter2));
  79. delete iter2;
  80. delete iter1;
  81. delete [] block2->base();
  82. delete [] block1->base();
  83. delete block2;
  84. delete block1;
  85. }
  86. } // namespace NativeTask