Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Murmur2 partitioner #1468

Merged
merged 33 commits into from
Dec 12, 2017
Merged

Conversation

barrotsteindev
Copy link
Contributor

Add murmur2 partitioner, currently using the fastest algorithm for murmur2, which is not endian neutral. Also implements an endian neutral algorithm which can be used instead.
Fixes part of issue #1424

@edenhill
Copy link
Contributor

Thanks for this, much appreciated and anticipated!

I will review this more careful in a while, but from a quick glance there are some low hanging fruit that needs fixing:

  • remove all white space diffs
  • remove the CONFIGURATION.md changes (it is auto generated from your build options)
  • add a LICENSE.murmur2 file to the top level
  • is there any proof of correctness for the provided implementation?
  • I'd like to see some unit tests to verify it produces the same result as the Java counterpart (look at rd_varint.c and rd_unittest.c to see how unit tests are added)
  • I dont think the new rdkafka_simple_producer_murmur2 example is warranted, about 99% of it is duplicated from existing examples. It might be better to include it in one of the other examples as an option.
  • There are comments in the murmur code that it must be able to perform unaligned reads, this is not portable.
  • There are also endian comments in the murmur code. Does it work on big-endian?
  • Please follow the coding guideline: use 8 whitespaces for tab

@barrotsteindev
Copy link
Contributor Author

I will use the endian neutral hash.
The murmur2hash.c file is from a repo that has no direct mention of a license.
Will fix the tabs and configuration file.
Thanks for your patience I'm fairly new to this.

@barrotsteindev
Copy link
Contributor Author

this should probably be the hash function (MurmurHashNeutral2) which will be used by librdkafka,
since it seems to be endian neutral and can aligned readable

// MurmurHashNeutral2, by Austin Appleby
//
// Same as MurmurHash2, but endian- and alignment-neutral.
// Half the speed though, alas.
*/

would be lovely if another set of eyes would take a look at that implementation to see if there's anything suspicious. I will test it against the java implementation shortly

@barrotsteindev
Copy link
Contributor Author

I hope I addressed all of your requests with these commits

Copy link
Contributor

@edenhill edenhill left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looking better.
Still a few issues to sort out, and some massive whitespace diffs that needs to go away.
Also make sure to reading the coding guidelines regarding formatting and indentation.

// MurmurHash2 was written by Austin Appleby, and is placed in the public
// domain. The author hereby disclaims copyright to this source code.

Permission is hereby granted, free of charge, to any person obtaining a copy
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does the following license really apply to public domain?
Did Austin Appleby specify this license for the code?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yes AppleBy distributed his software under the MIT license.

@@ -89,4 +89,3 @@ rdkafka_zookeeper_example: ../src/librdkafka.a rdkafka_zookeeper_example.c

clean:
rm -f $(EXAMPLES)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove whitespace diff

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove whitespace diff

if (argc != 3) {
fprintf(stderr, "%% Usage: %s <broker> <topic>\n", argv[0]);
return 1;
const char *brokers = NULL; /* Argument: broker list */
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The rdkafka_simple_producer was added because rdkafka_example.c was getting heavy with options trying to show-case everything.
I'd like the simple producer to stay simple.

We can wait with example code until the config-string-based partitioner support is in place (so you could do conf_set("partitioner", "murmur2", ..)).

So please remove the patch for this file.

extern "C" {
#endif

void SetAffinity ( int cpu );
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This doesn't seem to be used anywhere, remove.

src/rdcrc32.h Outdated
@@ -30,10 +30,6 @@
#include <zlib.h>
#endif

#ifdef __cplusplus
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This should not be removed.

src/rdmurmur2.c Outdated
#include "rdmurmur2.h"

int unittest_murmurhashneutral2 (void) {
const char * keysToTest[] = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

indent: 8 whitespaces.

Format: const char *keysToTest[] .. (pay attention to * placement)

src/rdmurmur2.c Outdated
const char * keysToTest[] = {
"kafka",
"amqp",
"giberish123456789"
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since the murmur2 code is prone to alignment and word issues I suggest you also add the following tests:

  • unaligned access
  • word-boundaries (i.e., lengths 7,8,9, 15,16,17)

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

i have fixed all issues but this. What did you mean by word boundaries, so i allocate pointers and start at an address offset by 7 bits from the aligned address?
this is the only thing blocking me right now

src/rdmurmur2.h Outdated
#ifndef __RDMURMUR2___H__
#define __RDMURMUR2___H__

#ifdef __cplusplus
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not needed in C code, remove.


static RD_INLINE rd_murmur2_t rd_murmur2(const char *data, size_t data_len) {
/*
* last bit is set to 0 because the java implementation uses int_32
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Isn't a cast to int32_t sufficient?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

no since the java version flips the first bit to make it positive and we need to maintain that behaviour

@@ -155,6 +156,7 @@
</ItemGroup>
<ItemGroup>
<ClCompile Include="..\src\crc32c.c" />
<ClCompile Include="..\src\MurMurHash2.c" />
<ClCompile Include="..\src\rdaddr.c" />
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what about rdmurmur2.h?

Copy link
Contributor

@edenhill edenhill left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you also please rebase this on current master and fixup / squash the commits?

int32_t partition_cnt,
void *rkt_opaque,
void *msg_opaque) {
if (keylen == 0)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looking at the Java partitioner I believe this should probably be if (!key) since there is a difference between no key (null) and an empty key (which is just an arbitrary key).
If so this is also incorrect on the existing partitioners, but we can't change their behaviour.

@barrotsteindev
Copy link
Contributor Author

barrotsteindev commented Oct 28, 2017 via email

@barrotsteindev
Copy link
Contributor Author

@edenhill I have implemented most fixes except for the word boundary. Could you give a hint because I did not quite understand the unit test you requested. I would really appreciate it.

@tmichaud314
Copy link

@barrotsteindev I would love to see this PR happen and it looks close to completion. Do you foresee this getting merged? Thanks for your efforts here!

@barrotsteindev
Copy link
Contributor Author

i have rebased this pull request and responded to your requests @edenhill

Copy link
Contributor

@edenhill edenhill left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good, but the whitespace diffs needs to be removed and the other formatting comments need to be addressed as well.

@@ -3,24 +3,24 @@
*
* Copyright (c) 2012,2013 Magnus Edenhill
* All rights reserved.
*
*
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove whitespace diffs

@@ -89,4 +89,3 @@ rdkafka_zookeeper_example: ../src/librdkafka.a rdkafka_zookeeper_example.c

clean:
rm -f $(EXAMPLES)

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove whitespace diff

@@ -222,7 +223,7 @@ int rd_kafka_msg_new (rd_kafka_itopic_t *rkt, int32_t force_partition,
int errnox;

/* Create message */
rkm = rd_kafka_msg_new0(rkt, force_partition, msgflags,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove whitespace diff

void *msg_opaque) {
if (!key)
return rd_kafka_msg_partitioner_random(rkt,
key,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove empty lines

@@ -799,4 +829,3 @@ int64_t rd_kafka_message_latency (const rd_kafka_message_t *rkmessage) {

return rd_clock() - rkm->rkm_ts_enq;
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

remove whitespace diff

@barrotsteindev
Copy link
Contributor Author

Hey @edenhill,
I addressed your new review, hope it's satisfactory.
Thanks in advance.

@edenhill
Copy link
Contributor

Looks good, thanks!
Just need to fix these OSX compile warnings and then we're good to merge:
https://travis-ci.org/edenhill/librdkafka/jobs/315112979#L229

@barrotsteindev
Copy link
Contributor Author

@edenhill , I have examined the warnings from the osx build,
and have noticed that they only come from the non endian neutral hash functions, and the aligned only hash function. These functions are not used.
We can either ignore those warnings, since we do not use that code, or I can remove it from the file.
Which action do you prefer?
thanks in advance.

@edenhill
Copy link
Contributor

We should probably use the optimized versions if the platform supports it, but we can fix that later.

@edenhill edenhill merged commit 35f28e2 into confluentinc:master Dec 12, 2017
@edenhill
Copy link
Contributor

Big thanks for all the effort on this!

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

Successfully merging this pull request may close these issues.

None yet

3 participants