Browse Source
This patch fixes the inconsistent handling of out of range errors in the replica fetcher. Previously we would raise a fatal error if the follower's offset is ahead of the leader's and unclean leader election is not enabled. The behavior was inconsistent depending on the message format. With KIP-101/KIP-279 and the new message format, upon becoming a follower, the replica would use leader epoch information to reconcile the end of the log with the leader and simply truncate. Additionally, with the old format, the check is not really bulletproof for detecting data loss since the unclean leader's end offset might have already caught up to the follower's offset at the time of its initial fetch or when it queries for the current log end offset. With this patch, we simply skip the unclean leader election check and allow the needed truncation to occur. When the truncation offset is below the high watermark, a warning will be logged. This makes the behavior consistent for all message formats and removes a scenario in which an error on one partition can bring the broker down. Reviewers: Ismael Juma <ismael@juma.me.uk>, Jun Rao <junrao@gmail.com>pull/5596/merge
Jason Gustafson
6 years ago
committed by
GitHub
5 changed files with 11 additions and 181 deletions
@ -1,146 +0,0 @@
@@ -1,146 +0,0 @@
|
||||
/** |
||||
* Licensed to the Apache Software Foundation (ASF) under one or more |
||||
* contributor license agreements. See the NOTICE file distributed with |
||||
* this work for additional information regarding copyright ownership. |
||||
* The ASF licenses this file to You under the Apache License, Version 2.0 |
||||
* (the "License"); you may not use this file except in compliance with |
||||
* the License. You may obtain a copy of the License at |
||||
* |
||||
* http://www.apache.org/licenses/LICENSE-2.0 |
||||
* |
||||
* Unless required by applicable law or agreed to in writing, software |
||||
* distributed under the License is distributed on an "AS IS" BASIS, |
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
||||
* See the License for the specific language governing permissions and |
||||
* limitations under the License. |
||||
*/ |
||||
|
||||
package kafka.server |
||||
|
||||
import java.util.concurrent.atomic.AtomicBoolean |
||||
|
||||
import kafka.cluster.BrokerEndPoint |
||||
import kafka.utils.{Exit, TestUtils} |
||||
import kafka.utils.TestUtils.createBrokerConfigs |
||||
import kafka.zk.ZooKeeperTestHarness |
||||
import org.apache.kafka.common.TopicPartition |
||||
import org.apache.kafka.common.internals.FatalExitError |
||||
import org.apache.kafka.common.metrics.Metrics |
||||
import org.apache.kafka.common.protocol.Errors |
||||
import org.apache.kafka.common.record.Records |
||||
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse} |
||||
import org.apache.kafka.common.utils.Time |
||||
import org.junit.{After, Test} |
||||
|
||||
import scala.collection.Map |
||||
import scala.collection.JavaConverters._ |
||||
import scala.concurrent.Future |
||||
|
||||
class ReplicaFetcherThreadFatalErrorTest extends ZooKeeperTestHarness { |
||||
|
||||
private var brokers: Seq[KafkaServer] = null |
||||
@volatile private var shutdownCompleted = false |
||||
|
||||
@After |
||||
override def tearDown() { |
||||
Exit.resetExitProcedure() |
||||
TestUtils.shutdownServers(brokers) |
||||
super.tearDown() |
||||
} |
||||
|
||||
/** |
||||
* Verifies that a follower shuts down if the offset for an `added partition` is out of range and if a fatal |
||||
* exception is thrown from `handleOffsetOutOfRange`. It's a bit tricky to ensure that there are no deadlocks |
||||
* when the shutdown hook is invoked and hence this test. |
||||
*/ |
||||
@Test |
||||
def testFatalErrorInAddPartitions(): Unit = { |
||||
|
||||
// Unlike `TestUtils.createTopic`, this doesn't wait for metadata propagation as the broker shuts down before |
||||
// the metadata is propagated. |
||||
def createTopic(topic: String): Unit = { |
||||
adminZkClient.createTopic(topic, partitions = 1, replicationFactor = 2) |
||||
TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, topic, 0) |
||||
} |
||||
|
||||
val props = createBrokerConfigs(2, zkConnect) |
||||
brokers = props.map(KafkaConfig.fromProps).map(config => createServer(config, { params => |
||||
import params._ |
||||
new ReplicaFetcherThread(threadName, fetcherId, sourceBroker, config, replicaManager, metrics, time, quotaManager) { |
||||
override def handleOffsetOutOfRange(topicPartition: TopicPartition): Long = throw new FatalExitError |
||||
override def addPartitions(partitionAndOffsets: Map[TopicPartition, Long]): Unit = |
||||
super.addPartitions(partitionAndOffsets.mapValues(_ => -1)) |
||||
} |
||||
})) |
||||
createTopic("topic") |
||||
TestUtils.waitUntilTrue(() => shutdownCompleted, "Shutdown of follower did not complete") |
||||
} |
||||
|
||||
/** |
||||
* Verifies that a follower shuts down if the offset of a partition in the fetch response is out of range and if a |
||||
* fatal exception is thrown from `handleOffsetOutOfRange`. It's a bit tricky to ensure that there are no deadlocks |
||||
* when the shutdown hook is invoked and hence this test. |
||||
*/ |
||||
@Test |
||||
def testFatalErrorInProcessFetchRequest(): Unit = { |
||||
val props = createBrokerConfigs(2, zkConnect) |
||||
brokers = props.map(KafkaConfig.fromProps).map(config => createServer(config, { params => |
||||
import params._ |
||||
new ReplicaFetcherThread(threadName, fetcherId, sourceBroker, config, replicaManager, metrics, time, quotaManager) { |
||||
override def handleOffsetOutOfRange(topicPartition: TopicPartition): Long = throw new FatalExitError |
||||
override protected def fetchFromLeader(fetchRequest: FetchRequest.Builder): Seq[(TopicPartition, PD)] = { |
||||
fetchRequest.fetchData.asScala.keys.toSeq.map { tp => |
||||
(tp, new FetchResponse.PartitionData[Records](Errors.OFFSET_OUT_OF_RANGE, |
||||
FetchResponse.INVALID_HIGHWATERMARK, FetchResponse.INVALID_LAST_STABLE_OFFSET, |
||||
FetchResponse.INVALID_LOG_START_OFFSET, null, null)) |
||||
} |
||||
} |
||||
} |
||||
})) |
||||
TestUtils.createTopic(zkClient, "topic", numPartitions = 1, replicationFactor = 2, servers = brokers) |
||||
TestUtils.waitUntilTrue(() => shutdownCompleted, "Shutdown of follower did not complete") |
||||
} |
||||
|
||||
private case class FetcherThreadParams(threadName: String, fetcherId: Int, sourceBroker: BrokerEndPoint, |
||||
replicaManager: ReplicaManager, metrics: Metrics, time: Time, |
||||
quotaManager: ReplicationQuotaManager) |
||||
|
||||
private def createServer(config: KafkaConfig, fetcherThread: FetcherThreadParams => ReplicaFetcherThread): KafkaServer = { |
||||
val time = Time.SYSTEM |
||||
val server = new KafkaServer(config, time) { |
||||
|
||||
override def createReplicaManager(isShuttingDown: AtomicBoolean): ReplicaManager = { |
||||
new ReplicaManager(config, metrics, time, zkClient, kafkaScheduler, logManager, isShuttingDown, |
||||
quotaManagers, new BrokerTopicStats, metadataCache, logDirFailureChannel) { |
||||
|
||||
override protected def createReplicaFetcherManager(metrics: Metrics, time: Time, threadNamePrefix: Option[String], |
||||
quotaManager: ReplicationQuotaManager) = |
||||
new ReplicaFetcherManager(config, this, metrics, time, threadNamePrefix, quotaManager) { |
||||
override def createFetcherThread(fetcherId: Int, sourceBroker: BrokerEndPoint): AbstractFetcherThread = { |
||||
val prefix = threadNamePrefix.map(tp => s"$tp:").getOrElse("") |
||||
val threadName = s"${prefix}ReplicaFetcherThread-$fetcherId-${sourceBroker.id}" |
||||
fetcherThread(FetcherThreadParams(threadName, fetcherId, sourceBroker, replicaManager, metrics, |
||||
time, quotaManager)) |
||||
} |
||||
} |
||||
} |
||||
} |
||||
|
||||
} |
||||
|
||||
Exit.setExitProcedure { (_, _) => |
||||
import scala.concurrent.ExecutionContext.Implicits._ |
||||
// Run in a separate thread like shutdown hooks |
||||
Future { |
||||
server.shutdown() |
||||
shutdownCompleted = true |
||||
} |
||||
// Sleep until interrupted to emulate the fact that `System.exit()` never returns |
||||
Thread.sleep(Long.MaxValue) |
||||
throw new AssertionError |
||||
} |
||||
server.startup() |
||||
server |
||||
} |
||||
|
||||
} |
Loading…
Reference in new issue