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

Can't produce msg when kill broker #513

Closed
DavidLiuXh opened this issue Jan 20, 2016 · 28 comments
Closed

Can't produce msg when kill broker #513

DavidLiuXh opened this issue Jan 20, 2016 · 28 comments

Comments

@DavidLiuXh
Copy link

1.There are three broker 1, 2, 3;
2. Topic test_topic_1, It only has a Replication;
3. Run producer;
4. kill broker 1;
5. The producer still send msg to broker 1;

@edenhill
Copy link
Contributor

Please reproduce with debug=topic,metadata,broker

@edenhill
Copy link
Contributor

If your replication-factor is 1 and broker 1 is your leader and then goes down then no other broker will be able to take over the topic+partition since it hasn't been replicated.

@DavidLiuXh
Copy link
Author

I think rdkafka will auto send msg to broker 2 or 3.
I use rd_kafka_topic_partition_available to test partition of the broker 1, but result is 1

@edenhill
Copy link
Contributor

It will only send to the current leader broker, and your leader broker went down without any replicas to take over.

@DavidLiuXh
Copy link
Author

I use rd_kafka_topic_partition_available to test partition of the broker 1, but result is 1

edenhill added a commit that referenced this issue Jan 20, 2016
@edenhill
Copy link
Contributor

Update to latest master and see if that fixes it.

@DavidLiuXh
Copy link
Author

I test rd_kafka_topic_partition_available, but result still is 1

@edenhill
Copy link
Contributor

Can you paste the metadata for your topic when the broker is down?
(e.g., kafkacat -b broker -L -t yourtopic or rdkafka_example -b broker -L -t yourtopic),

@DavidLiuXh
Copy link
Author

Sorry, I rebuild librdkafka and use rd_kafka_topic_partition_available to test partition of the broker 1, result is 0.

@DavidLiuXh
Copy link
Author

But I restart broker 1 and found a certain probability that the all result is 0 when use rd_kafka_topic_partition_available to test all partitions of the broker 1, 2, 3.

but the result of "rdkafka_example -b broker -L -t yourtopic" is ok.
Metadata for t_p_7 (from broker -1: localhost:9192/bootstrap):
3 brokers:
broker 0 at 127.0.0.1:9192
broker 1 at 127.0.0.1:9193
broker 2 at 127.0.0.1:9194
1 topics:
topic "t_p_7" with 3 partitions:
partition 2, leader 1, replicas: 1, isrs: 1
partition 1, leader 0, replicas: 0, isrs: 0
partition 0, leader 2, replicas: 2, isrs: 2

@edenhill
Copy link
Contributor

So you are saying partition_available() returns 0 for all partitions when broker 1 is down, even for partitions that have another leader than broker 1?

Can you reproduce this with debug=topic,metadata,broker enabled?

@DavidLiuXh
Copy link
Author

I found a certain probability that the all result is 0 when use rd_kafka_topic_partition_available to test all partitions of the broker 1, 2, 3 when broker 1 is restart

@edenhill
Copy link
Contributor

Running with the above debug would let us troubleshoot and see why.

@DavidLiuXh
Copy link
Author

1453365931.611 RDKAFKA-7-BRKMAIN: rdkafka#producer-1: :0/internal: Enter main broker thread
1453365931.611 RDKAFKA-7-STATE: rdkafka#producer-1: :0/internal: Broker changed state INIT -> UP
1453365931.611 RDKAFKA-7-BROKER: rdkafka#producer-1: localhost:9192/bootstrap: Added new broker with NodeId -1
1453365931.611 RDKAFKA-7-BRKMAIN: rdkafka#producer-1: localhost:9192/bootstrap: Enter main broker thread
1453365931.611 RDKAFKA-7-CONNECT: rdkafka#producer-1: localhost:9192/bootstrap: broker in state INIT connecting
1453365931.611 RDKAFKA-7-BROKER: rdkafka#producer-1: localhost:9193/bootstrap: Added new broker with NodeId -1
1453365931.611 RDKAFKA-7-BROKER: rdkafka#producer-1: localhost:9194/bootstrap: Added new broker with NodeId -1
1453365931.611 RDKAFKA-7-TOPIC: rdkafka#producer-1: New local topic: t_p_7
1453365931.611 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap: Request metadata for t_p_7: application requested
1453365931.611 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap: Request metadata: scheduled: not in broker thread
1453365931.612 RDKAFKA-7-CONNECT: rdkafka#producer-1: localhost:9192/bootstrap: Connecting to ipv4#127.0.0.1:9192 (plaintext) with socket 3
1453365931.612 RDKAFKA-7-STATE: rdkafka#producer-1: localhost:9192/bootstrap: Broker changed state INIT -> CONNECT
1453365931.612 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap: Request metadata for t_p_7: application requested
1453365931.612 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap: Request metadata for t_p_7: application requested
1453365931.612 RDKAFKA-7-BRKMAIN: rdkafka#producer-1: localhost:9193/bootstrap: Enter main broker thread
1453365931.612 RDKAFKA-7-CONNECT: rdkafka#producer-1: localhost:9193/bootstrap: broker in state INIT connecting
1453365931.612 RDKAFKA-7-CONNECT: rdkafka#producer-1: localhost:9193/bootstrap: Connecting to ipv4#127.0.0.1:9193 (plaintext) with socket 4
1453365931.612 RDKAFKA-7-BRKMAIN: rdkafka#producer-1: localhost:9194/bootstrap: Enter main broker thread
1453365931.612 RDKAFKA-7-CONNECT: rdkafka#producer-1: localhost:9194/bootstrap: broker in state INIT connecting
1453365931.612 RDKAFKA-7-STATE: rdkafka#producer-1: localhost:9193/bootstrap: Broker changed state INIT -> CONNECT
1453365931.612 RDKAFKA-7-CONNECT: rdkafka#producer-1: localhost:9194/bootstrap: Connecting to ipv4#127.0.0.1:9194 (plaintext) with socket 5
1453365931.612 RDKAFKA-7-STATE: rdkafka#producer-1: localhost:9194/bootstrap: Broker changed state INIT -> CONNECT
1453365931.622 RDKAFKA-7-CONNECT: rdkafka#producer-1: localhost:9192/bootstrap: Connected to ipv4#127.0.0.1:9192
1453365931.622 RDKAFKA-7-CONNECTED: rdkafka#producer-1: localhost:9192/bootstrap: Connected
1453365931.622 RDKAFKA-7-STATE: rdkafka#producer-1: localhost:9192/bootstrap: Broker changed state CONNECT -> UP
1453365931.622 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap: Request metadata for locally known topics: connected
1453365931.622 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap: Request metadata for locally known topics: connected
1453365931.622 RDKAFKA-7-CONNECT: rdkafka#producer-1: localhost:9193/bootstrap: Connected to ipv4#127.0.0.1:9193
1453365931.622 RDKAFKA-7-CONNECTED: rdkafka#producer-1: localhost:9193/bootstrap: Connected
1453365931.622 RDKAFKA-7-STATE: rdkafka#producer-1: localhost:9193/bootstrap: Broker changed state CONNECT -> UP
1453365931.622 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9193/bootstrap: Request metadata for locally known topics: connected
1453365931.622 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9193/bootstrap: Request metadata for locally known topics: connected
1453365931.622 RDKAFKA-7-CONNECT: rdkafka#producer-1: localhost:9194/bootstrap: Connected to ipv4#127.0.0.1:9194
1453365931.622 RDKAFKA-7-CONNECTED: rdkafka#producer-1: localhost:9194/bootstrap: Connected
1453365931.622 RDKAFKA-7-STATE: rdkafka#producer-1: localhost:9194/bootstrap: Broker changed state CONNECT -> UP
1453365931.622 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9194/bootstrap: Request metadata for locally known topics: connected
1453365931.622 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9194/bootstrap: Request metadata for locally known topics: connected
1453365931.642 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap: ===== Received metadata =====
1453365931.642 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap: 3 brokers, 1 topics
1453365931.642 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap:   Broker #0/3: 127.0.0.1:9192 NodeId 0
1453365931.642 RDKAFKA-7-BROKER: rdkafka#producer-1: 127.0.0.1:9192/0: Added new broker with NodeId 0
1453365931.642 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap:   Broker #1/3: 127.0.0.1:9193 NodeId 1
1453365931.642 RDKAFKA-7-BRKMAIN: rdkafka#producer-1: 127.0.0.1:9192/0: Enter main broker thread
1453365931.642 RDKAFKA-7-CONNECT: rdkafka#producer-1: 127.0.0.1:9192/0: broker in state INIT connecting
1453365931.642 RDKAFKA-7-BROKER: rdkafka#producer-1: 127.0.0.1:9193/1: Added new broker with NodeId 1
1453365931.642 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap:   Broker #2/3: 127.0.0.1:9194 NodeId 2
1453365931.642 RDKAFKA-7-BRKMAIN: rdkafka#producer-1: 127.0.0.1:9193/1: Enter main broker thread
1453365931.642 RDKAFKA-7-CONNECT: rdkafka#producer-1: 127.0.0.1:9193/1: broker in state INIT connecting
1453365931.653 RDKAFKA-7-BROKER: rdkafka#producer-1: 127.0.0.1:9194/2: Added new broker with NodeId 2
1453365931.654 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap:   Topic #0/1: t_p_7 with 3 partitions
1453365931.654 RDKAFKA-7-BRKMAIN: rdkafka#producer-1: 127.0.0.1:9194/2: Enter main broker thread
1453365931.654 RDKAFKA-7-CONNECT: rdkafka#producer-1: 127.0.0.1:9194/2: broker in state INIT connecting
1453365931.654 RDKAFKA-7-CONNECT: rdkafka#producer-1: 127.0.0.1:9192/0: Connecting to ipv4#127.0.0.1:9192 (plaintext) with socket 7
1453365931.654 RDKAFKA-7-STATE: rdkafka#producer-1: Topic t_p_7 changed state unknown -> exists
1453365931.654 RDKAFKA-7-PARTCNT: rdkafka#producer-1: Topic t_p_7 partition count changed from 0 to 3
1453365931.654 RDKAFKA-7-CONNECT: rdkafka#producer-1: 127.0.0.1:9193/1: Connecting to ipv4#127.0.0.1:9193 (plaintext) with socket 8
1453365931.654 RDKAFKA-7-CONNECT: rdkafka#producer-1: 127.0.0.1:9194/2: Connecting to ipv4#127.0.0.1:9194 (plaintext) with socket 6
1453365931.654 RDKAFKA-7-STATE: rdkafka#producer-1: 127.0.0.1:9192/0: Broker changed state INIT -> CONNECT
1453365931.654 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap:   Topic t_p_7 partition 2 Leader 1
1453365931.654 RDKAFKA-7-BRKDELGT: rdkafka#producer-1: Broker 127.0.0.1:9193/1 is now leader for topic t_p_7 [2] with 0 messages (0 bytes) queued
1453365931.654 RDKAFKA-7-BRKMIGR: rdkafka#producer-1: Migrating topic t_p_7 [2] from (none) to 127.0.0.1:9193/1
1453365931.654 RDKAFKA-7-STATE: rdkafka#producer-1: 127.0.0.1:9193/1: Broker changed state INIT -> CONNECT
1453365931.654 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap:   Topic t_p_7 partition 1 Leader 0
1453365931.654 RDKAFKA-7-STATE: rdkafka#producer-1: 127.0.0.1:9194/2: Broker changed state INIT -> CONNECT
1453365931.654 RDKAFKA-7-TOPBRK: rdkafka#producer-1: 127.0.0.1:9193/1: Topic t_p_7 [2]: joining broker
1453365931.654 RDKAFKA-7-BRKDELGT: rdkafka#producer-1: Broker 127.0.0.1:9192/0 is now leader for topic t_p_7 [1] with 0 messages (0 bytes) queued
1453365931.654 RDKAFKA-7-BRKMIGR: rdkafka#producer-1: Migrating topic t_p_7 [1] from (none) to 127.0.0.1:9192/0
1453365931.654 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap:   Topic t_p_7 partition 0 Leader 2
1453365931.654 RDKAFKA-7-BRKDELGT: rdkafka#producer-1: Broker 127.0.0.1:9194/2 is now leader for topic t_p_7 [0] with 0 messages (0 bytes) queued
1453365931.654 RDKAFKA-7-BRKMIGR: rdkafka#producer-1: Migrating topic t_p_7 [0] from (none) to 127.0.0.1:9194/2
1453365931.654 RDKAFKA-7-PARTCNT: rdkafka#producer-1: Partitioning 0 unassigned messages in topic t_p_7 to 3 partitions
1453365931.654 RDKAFKA-7-UAS: rdkafka#producer-1: 0/0 messages were partitioned in topic t_p_7
1453365931.654 RDKAFKA-7-TOPBRK: rdkafka#producer-1: 127.0.0.1:9194/2: Topic t_p_7 [0]: joining broker
1453365931.654 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap: Requested topic t_p_7 seen in metadata
1453365931.654 RDKAFKA-7-TOPBRK: rdkafka#producer-1: 127.0.0.1:9192/0: Topic t_p_7 [1]: joining broker
1453365931.654 RDKAFKA-7-DESTROY: rdkafka#producer-1: Terminating instance
1453365931.654 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9192/bootstrap: ===== Received metadata =====
1453365931.654 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9193/bootstrap: ===== Received metadata =====
1453365931.654 RDKAFKA-7-METADATA: rdkafka#producer-1: localhost:9194/bootstrap: ===== Received metadata =====
1453365931.654 RDKAFKA-7-DESTROY: rdkafka#producer-1: Destroy internal
1453365931.654 RDKAFKA-7-DESTROY: rdkafka#producer-1: Remove all topics
1453365931.654 RDKAFKA-7-PARTCNT: rdkafka#producer-1: Topic t_p_7 partition count changed from 3 to 0
1453365931.654 RDKAFKA-7-BRKDELGT: rdkafka#producer-1: Broker 127.0.0.1:9194/2 no longer leader for topic t_p_7 [0]
1453365931.654 RDKAFKA-7-BRKDELGT: rdkafka#producer-1: No broker is leader for topic t_p_7 [0]
1453365931.654 RDKAFKA-7-BRKMIGR: rdkafka#producer-1: Migrating topic t_p_7 [0] from 127.0.0.1:9194/2 to (none)
1453365931.654 RDKAFKA-7-BRKDELGT: rdkafka#producer-1: Broker 127.0.0.1:9192/0 no longer leader for topic t_p_7 [1]
1453365931.654 RDKAFKA-7-BRKDELGT: rdkafka#producer-1: No broker is leader for topic t_p_7 [1]
1453365931.654 RDKAFKA-7-BRKMIGR: rdkafka#producer-1: Migrating topic t_p_7 [1] from 127.0.0.1:9192/0 to (none)
1453365931.654 RDKAFKA-7-TOPBRK: rdkafka#producer-1: 127.0.0.1:9194/2: Topic t_p_7 [0]: leaving broker (next leader (none))
1453365931.654 RDKAFKA-7-BRKDELGT: rdkafka#producer-1: Broker 127.0.0.1:9193/1 no longer leader for topic t_p_7 [2]
1453365931.654 RDKAFKA-7-BRKDELGT: rdkafka#producer-1: No broker is leader for topic t_p_7 [2]
1453365931.654 RDKAFKA-7-BRKMIGR: rdkafka#producer-1: Migrating topic t_p_7 [2] from 127.0.0.1:9193/1 to (none)
1453365931.654 RDKAFKA-7-TOPBRK: rdkafka#producer-1: 127.0.0.1:9192/0: Topic t_p_7 [1]: leaving broker (next leader (none))
1453365931.654 RDKAFKA-7-TOPPARREMOVE: rdkafka#producer-1: REMOVING toppar t_p_7 [1]
1453365931.654 RDKAFKA-7-TOPPARREMOVE: rdkafka#producer-1: REMOVING toppar t_p_7 [-1]
1453365931.654 RDKAFKA-7-BROKERFAIL: rdkafka#producer-1: localhost:9192/bootstrap: failed: err: Local: Broker handle destroyed: (errno: Interrupted system call)
1453365931.654 RDKAFKA-7-STATE: rdkafka#producer-1: localhost:9192/bootstrap: Broker changed state UP -> DOWN
1453365931.654 RDKAFKA-7-TOPPARREMOVE: rdkafka#producer-1: REMOVING toppar t_p_7 [0]
1453365931.654 RDKAFKA-7-TOPBRK: rdkafka#producer-1: 127.0.0.1:9193/1: Topic t_p_7 [2]: leaving broker (next leader (none))
1453365931.654 RDKAFKA-7-TOPPARREMOVE: rdkafka#producer-1: REMOVING toppar t_p_7 [2]
1453365931.654 RDKAFKA-7-BROKERFAIL: rdkafka#producer-1: localhost:9193/bootstrap: failed: err: Local: Broker handle destroyed: (errno: Interrupted system call)
1453365931.654 RDKAFKA-7-BROKERFAIL: rdkafka#producer-1: localhost:9194/bootstrap: failed: err: Local: Broker handle destroyed: (errno: Interrupted system call)
1453365931.654 RDKAFKA-7-STATE: rdkafka#producer-1: localhost:9193/bootstrap: Broker changed state UP -> DOWN
1453365931.654 RDKAFKA-7-STATE: rdkafka#producer-1: localhost:9194/bootstrap: Broker changed state UP -> DOWN
1453365931.662 RDKAFKA-7-BROKERFAIL: rdkafka#producer-1: :0/internal: failed: err: Local: Broker handle destroyed: (errno: Success)
1453365931.662 RDKAFKA-7-STATE: rdkafka#producer-1: :0/internal: Broker changed state UP -> DOWN
1453365931.664 RDKAFKA-7-CONNECT: rdkafka#producer-1: 127.0.0.1:9192/0: Connected to ipv4#127.0.0.1:9192
1453365931.664 RDKAFKA-7-CONNECTED: rdkafka#producer-1: 127.0.0.1:9192/0: Connected
1453365931.664 RDKAFKA-7-STATE: rdkafka#producer-1: 127.0.0.1:9192/0: Broker changed state CONNECT -> UP
1453365931.664 RDKAFKA-7-METADATA: rdkafka#producer-1: 127.0.0.1:9192/0: Request metadata for locally known topics: connected
1453365931.664 RDKAFKA-7-METADATA: rdkafka#producer-1: 127.0.0.1:9192/0: Request metadata for locally known topics: connected
1453365931.664 RDKAFKA-7-CONNECT: rdkafka#producer-1: 127.0.0.1:9194/2: Connected to ipv4#127.0.0.1:9194
1453365931.664 RDKAFKA-7-CONNECTED: rdkafka#producer-1: 127.0.0.1:9194/2: Connected
1453365931.664 RDKAFKA-7-STATE: rdkafka#producer-1: 127.0.0.1:9194/2: Broker changed state CONNECT -> UP
1453365931.664 RDKAFKA-7-METADATA: rdkafka#producer-1: 127.0.0.1:9194/2: Request metadata for locally known topics: connected
1453365931.664 RDKAFKA-7-METADATA: rdkafka#producer-1: 127.0.0.1:9194/2: Request metadata for locally known topics: connected
1453365931.664 RDKAFKA-7-CONNECT: rdkafka#producer-1: 127.0.0.1:9193/1: Connected to ipv4#127.0.0.1:9193
1453365931.664 RDKAFKA-7-CONNECTED: rdkafka#producer-1: 127.0.0.1:9193/1: Connected
1453365931.664 RDKAFKA-7-STATE: rdkafka#producer-1: 127.0.0.1:9193/1: Broker changed state CONNECT -> UP
1453365931.664 RDKAFKA-7-METADATA: rdkafka#producer-1: 127.0.0.1:9193/1: Request metadata for locally known topics: connected
1453365931.664 RDKAFKA-7-METADATA: rdkafka#producer-1: 127.0.0.1:9193/1: Request metadata for locally known topics: connected
1453365931.665 RDKAFKA-7-METADATA: rdkafka#producer-1: 127.0.0.1:9192/0: ===== Received metadata =====
1453365931.665 RDKAFKA-7-BROKERFAIL: rdkafka#producer-1: 127.0.0.1:9192/0: failed: err: Local: Broker handle destroyed: (errno: Resource temporarily unavailable)
1453365931.665 RDKAFKA-7-STATE: rdkafka#producer-1: 127.0.0.1:9192/0: Broker changed state UP -> DOWN
1453365931.665 RDKAFKA-7-METADATA: rdkafka#producer-1: 127.0.0.1:9193/1: ===== Received metadata =====
1453365931.665 RDKAFKA-7-BROKERFAIL: rdkafka#producer-1: 127.0.0.1:9193/1: failed: err: Local: Broker handle destroyed: (errno: Resource temporarily unavailable)
1453365931.665 RDKAFKA-7-STATE: rdkafka#producer-1: 127.0.0.1:9193/1: Broker changed state UP -> DOWN
1453365931.666 RDKAFKA-7-METADATA: rdkafka#producer-1: 127.0.0.1:9194/2: ===== Received metadata =====
1453365931.666 RDKAFKA-7-BROKERFAIL: rdkafka#producer-1: 127.0.0.1:9194/2: failed: err: Local: Broker handle destroyed: (errno: Resource temporarily unavailable)
1453365931.666 RDKAFKA-7-STATE: rdkafka#producer-1: 127.0.0.1:9194/2: Broker changed state UP -> DOWN

@edenhill
Copy link
Contributor

Could you do some printouts from your partitioner too so we can match the debug log the what you are experiencing?

Also, in the above log the producer is destroyed almost immediately. Is this intentional?

@DavidLiuXh
Copy link
Author

  1. My partitioner function:

    
    int32_t QbusProducer::QbusProducerImp::partitionHashFunc(const rd_kafka_topic_t *rkt,
            const void *keydata,
            size_t keylen,
            int32_t partition_cnt,
            void *rkt_opaque,
            void *msg_opaque) {
    int32_t hit_partition = 0;
    if (keylen > 0 && NULL != keydata) {
        while (true) {
            const char* key = static_cast(keydata);
            DEBUG(__FUNCTION__ << " | KEY: " << std::string(key, keylen));
            //use djb hash
            unsigned int hash = 5381;
            for (size_t i = 0 ; i < keylen; i++) {
                hash = ((hash << 5) + hash) + key[i];
            }
            hit_partition = hash % partition_cnt;
            if (1 == rd_kafka_topic_partition_available(rkt, hit_partition)) {
                break;
            } else {
                DEBUG(__FUNCTION__ << " | retry select parition | current invailed partiton: " << hit_partition);
            }
        }
    } else {
        while (true) {
            hit_partition = rd_kafka_msg_partitioner_random(rkt,
                        keydata,
                        keylen,
                        partition_cnt,
                        rkt_opaque,
                        msg_opaque);
            if (1 == rd_kafka_topic_partition_available(rkt, hit_partition)) {
                break;
            } else {
                DEBUG(__FUNCTION__ << " | retry select parition | current invailed partiton: " << hit_partition);
            }
        }
    }
    return hit_partition;
    }
    
  2. print again and again:

    | retry select parition | current invailed partiton: 0
    | retry select parition | current invailed partiton: 1
    | retry select parition | current invailed partiton: 2

@edenhill
Copy link
Contributor

That while (true) loop will busy-loop forever if there are no leaders for a topic's partitions, you should give up after partition_cnt retries and return RD_KAFKA_PARTITION_UA.

@edenhill
Copy link
Contributor

I need to see the partitioner printouts mixed with the rdkafka debug log, otherwise it is impossible to sync up when it happens.

@DavidLiuXh
Copy link
Author

see log
rdkafka.txt

@edenhill
Copy link
Contributor

I dont see any log lines from your partitioner ("current invailed partition..")

@DavidLiuXh
Copy link
Author

For convenience you see, I output rdkafka log to a separate file.
producer.txt

@edenhill
Copy link
Contributor

This is the problem:
" | RD_KAFKA_LOG | level: 7 | fac: METADATA | msg: localhost:9194/bootstrap: Error in metadata reply for topic t_p_7 (PartCnt 0): Broker: Leader not available"

When broker 2 comes up again and librdkafka queries it for metadata it responds with an empty topic information and error "Leader not available". This makes librdkafka remove leaders for all partitions and that's why partition_available() will return false.

I'll try to figure out what's best to do in this case.

@edenhill
Copy link
Contributor

This should be fixed now, can you give it a try?
Thanks

@DavidLiuXh
Copy link
Author

Temporarily not reproduce, I will always watch it.
Thanks.

@edenhill
Copy link
Contributor

Perfect, let me know how it goes.
Thanks

@mmanoj
Copy link

mmanoj commented Jun 7, 2016

@edenhill can you guide me how can I specify replication-factor while I create topic using librdkafka I'm using example program rdkafka_example.c

@edenhill
Copy link
Contributor

edenhill commented Jun 7, 2016

@mmanoj That's not possible, you can only do that using the kafka-topics.sh script in the kafka distro.

@mmanoj
Copy link

mmanoj commented Jun 7, 2016

Can I create using script it and use via program ? what happen if I create it using program and it not replicate ?

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

No branches or pull requests

3 participants