1 | /* |
2 | * librdkafka - Apache Kafka C library |
3 | * |
4 | * Copyright (c) 2012-2015, Magnus Edenhill |
5 | * All rights reserved. |
6 | * |
7 | * Redistribution and use in source and binary forms, with or without |
8 | * modification, are permitted provided that the following conditions are met: |
9 | * |
10 | * 1. Redistributions of source code must retain the above copyright notice, |
11 | * this list of conditions and the following disclaimer. |
12 | * 2. Redistributions in binary form must reproduce the above copyright notice, |
13 | * this list of conditions and the following disclaimer in the documentation |
14 | * and/or other materials provided with the distribution. |
15 | * |
16 | * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" |
17 | * AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE |
18 | * IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE |
19 | * ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR CONTRIBUTORS BE |
20 | * LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR |
21 | * CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF |
22 | * SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR PROFITS; OR BUSINESS |
23 | * INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN |
24 | * CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) |
25 | * ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE |
26 | * POSSIBILITY OF SUCH DAMAGE. |
27 | */ |
28 | |
29 | #include "rd.h" |
30 | #include "rdunittest.h" |
31 | #include "rdmurmur2.h" |
32 | #include "rdendian.h" |
33 | |
34 | |
35 | /* MurmurHash2, by Austin Appleby |
36 | * |
37 | * With librdkafka modifications combinining aligned/unaligned variants |
38 | * into the same function. |
39 | */ |
40 | |
41 | #define MM_MIX(h,k,m) { k *= m; k ^= k >> r; k *= m; h *= m; h ^= k; } |
42 | |
43 | /*----------------------------------------------------------------------------- |
44 | // Based on MurmurHashNeutral2, by Austin Appleby |
45 | // |
46 | // Same as MurmurHash2, but endian- and alignment-neutral. |
47 | // Half the speed though, alas. |
48 | // |
49 | */ |
50 | uint32_t rd_murmur2 (const void *key, size_t len) { |
51 | const uint32_t seed = 0x9747b28c; |
52 | const uint32_t m = 0x5bd1e995; |
53 | const int r = 24; |
54 | uint32_t h = seed ^ (uint32_t)len; |
55 | const unsigned char *tail; |
56 | |
57 | if (likely(((intptr_t)key & 0x3) == 0)) { |
58 | /* Input is 32-bit word aligned. */ |
59 | const uint32_t *data = (const uint32_t *)key; |
60 | |
61 | while (len >= 4) { |
62 | uint32_t k = htole32(*(uint32_t *)data); |
63 | |
64 | MM_MIX(h,k,m); |
65 | |
66 | data++; |
67 | len -= 4; |
68 | } |
69 | |
70 | tail = (const unsigned char *)data; |
71 | |
72 | } else { |
73 | /* Unaligned slower variant */ |
74 | const unsigned char *data = (const unsigned char *)key; |
75 | |
76 | while (len >= 4) { |
77 | uint32_t k; |
78 | |
79 | k = data[0]; |
80 | k |= data[1] << 8; |
81 | k |= data[2] << 16; |
82 | k |= data[3] << 24; |
83 | |
84 | MM_MIX(h,k,m); |
85 | |
86 | data += 4; |
87 | len -= 4; |
88 | } |
89 | |
90 | tail = data; |
91 | } |
92 | |
93 | /* Read remaining sub-word */ |
94 | switch(len) |
95 | { |
96 | case 3: h ^= tail[2] << 16; |
97 | case 2: h ^= tail[1] << 8; |
98 | case 1: h ^= tail[0]; |
99 | h *= m; |
100 | }; |
101 | |
102 | h ^= h >> 13; |
103 | h *= m; |
104 | h ^= h >> 15; |
105 | |
106 | /* Last bit is set to 0 because the java implementation uses int_32 |
107 | * and then sets to positive number flipping last bit to 1. */ |
108 | return h; |
109 | } |
110 | |
111 | |
112 | /** |
113 | * @brief Unittest for rd_murmur2() |
114 | */ |
115 | int unittest_murmur2 (void) { |
116 | const char *short_unaligned = "1234" ; |
117 | const char *unaligned = "PreAmbleWillBeRemoved,ThePrePartThatIs" ; |
118 | const char *keysToTest[] = { |
119 | "kafka" , |
120 | "giberish123456789" , |
121 | short_unaligned, |
122 | short_unaligned+1, |
123 | short_unaligned+2, |
124 | short_unaligned+3, |
125 | unaligned, |
126 | unaligned+1, |
127 | unaligned+2, |
128 | unaligned+3, |
129 | "" , |
130 | NULL, |
131 | }; |
132 | |
133 | const int32_t java_murmur2_results[] = { |
134 | 0xd067cf64, // kafka |
135 | 0x8f552b0c, // giberish123456789 |
136 | 0x9fc97b14, // short_unaligned |
137 | 0xe7c009ca, // short_unaligned+1 |
138 | 0x873930da, // short_unaligned+2 |
139 | 0x5a4b5ca1, // short_unaligned+3 |
140 | 0x78424f1c, // unaligned |
141 | 0x4a62b377, // unaligned+1 |
142 | 0xe0e4e09e, // unaligned+2 |
143 | 0x62b8b43f, // unaligned+3 |
144 | 0x106e08d9, // "" |
145 | 0x106e08d9, // NULL |
146 | }; |
147 | |
148 | size_t i; |
149 | for (i = 0; i < RD_ARRAYSIZE(keysToTest); i++) { |
150 | uint32_t h = rd_murmur2(keysToTest[i], |
151 | keysToTest[i] ? |
152 | strlen(keysToTest[i]) : 0); |
153 | RD_UT_ASSERT((int32_t)h == java_murmur2_results[i], |
154 | "Calculated murmur2 hash 0x%x for \"%s\", " |
155 | "expected 0x%x" , |
156 | h, keysToTest[i], java_murmur2_results[i]); |
157 | } |
158 | RD_UT_PASS(); |
159 | } |
160 | |