[common] Add support for JDK 9+ (#194)
parent
6aa9f43a4e
commit
679b940341
@ -0,0 +1,297 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.alibaba.ververica.cdc.debezium.internal;
|
||||||
|
|
||||||
|
import org.apache.flink.annotation.Internal;
|
||||||
|
import org.apache.flink.streaming.api.functions.source.SourceFunction;
|
||||||
|
import org.apache.flink.util.Collector;
|
||||||
|
|
||||||
|
import com.alibaba.ververica.cdc.debezium.DebeziumDeserializationSchema;
|
||||||
|
import io.debezium.connector.SnapshotRecord;
|
||||||
|
import io.debezium.data.Envelope;
|
||||||
|
import io.debezium.engine.ChangeEvent;
|
||||||
|
import io.debezium.engine.DebeziumEngine;
|
||||||
|
import org.apache.commons.collections.CollectionUtils;
|
||||||
|
import org.apache.kafka.connect.data.Schema;
|
||||||
|
import org.apache.kafka.connect.data.Struct;
|
||||||
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import java.util.ArrayDeque;
|
||||||
|
import java.util.List;
|
||||||
|
import java.util.Map;
|
||||||
|
import java.util.Queue;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* A Handler that convert change messages from {@link DebeziumEngine} to data in Flink. Considering
|
||||||
|
* Debezium in different mode has different strategies to hold the lock, e.g. snapshot, the handler
|
||||||
|
* also needs different strategy. In snapshot phase, the handler needs to hold the lock until the
|
||||||
|
* snapshot finishes. But in non-snapshot phase, the handler only needs to hold the lock when
|
||||||
|
* emitting the records.
|
||||||
|
*
|
||||||
|
* @param <T> The type of elements produced by the handler.
|
||||||
|
*/
|
||||||
|
@Internal
|
||||||
|
public class DebeziumChangeFetcher<T> {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(DebeziumChangeFetcher.class);
|
||||||
|
|
||||||
|
private final SourceFunction.SourceContext<T> sourceContext;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The lock that guarantees that record emission and state updates are atomic, from the view of
|
||||||
|
* taking a checkpoint.
|
||||||
|
*/
|
||||||
|
private final Object checkpointLock;
|
||||||
|
|
||||||
|
/** The schema to convert from Debezium's messages into Flink's objects. */
|
||||||
|
private final DebeziumDeserializationSchema<T> deserialization;
|
||||||
|
|
||||||
|
/** A collector to emit records in batch (bundle). */
|
||||||
|
private final DebeziumCollector debeziumCollector;
|
||||||
|
|
||||||
|
private final DebeziumOffset debeziumOffset;
|
||||||
|
|
||||||
|
private final DebeziumOffsetSerializer stateSerializer;
|
||||||
|
|
||||||
|
private final String heartbeatTopicPrefix;
|
||||||
|
|
||||||
|
private boolean isInDbSnapshotPhase;
|
||||||
|
|
||||||
|
private final Handover handover;
|
||||||
|
|
||||||
|
private volatile boolean isRunning = true;
|
||||||
|
|
||||||
|
// ---------------------------------------------------------------------------------------
|
||||||
|
// Metrics
|
||||||
|
// ---------------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
/** Timestamp of change event. If the event is a snapshot event, the timestamp is 0L. */
|
||||||
|
private volatile long messageTimestamp = 0L;
|
||||||
|
|
||||||
|
/** The last record processing time. */
|
||||||
|
private volatile long processTime = 0L;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* currentFetchEventTimeLag = FetchTime - messageTimestamp, where the FetchTime is the time the
|
||||||
|
* record fetched into the source operator.
|
||||||
|
*/
|
||||||
|
private volatile long fetchDelay = 0L;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* emitDelay = EmitTime - messageTimestamp, where the EmitTime is the time the record leaves the
|
||||||
|
* source operator.
|
||||||
|
*/
|
||||||
|
private volatile long emitDelay = 0L;
|
||||||
|
|
||||||
|
// ------------------------------------------------------------------------
|
||||||
|
|
||||||
|
public DebeziumChangeFetcher(
|
||||||
|
SourceFunction.SourceContext<T> sourceContext,
|
||||||
|
DebeziumDeserializationSchema<T> deserialization,
|
||||||
|
boolean isInDbSnapshotPhase,
|
||||||
|
String heartbeatTopicPrefix,
|
||||||
|
Handover handover) {
|
||||||
|
this.sourceContext = sourceContext;
|
||||||
|
this.checkpointLock = sourceContext.getCheckpointLock();
|
||||||
|
this.deserialization = deserialization;
|
||||||
|
this.isInDbSnapshotPhase = isInDbSnapshotPhase;
|
||||||
|
this.heartbeatTopicPrefix = heartbeatTopicPrefix;
|
||||||
|
this.debeziumCollector = new DebeziumCollector();
|
||||||
|
this.debeziumOffset = new DebeziumOffset();
|
||||||
|
this.stateSerializer = DebeziumOffsetSerializer.INSTANCE;
|
||||||
|
this.handover = handover;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Take a snapshot of the Debezium handler state.
|
||||||
|
*
|
||||||
|
* <p>Important: This method must be called under the checkpoint lock.
|
||||||
|
*/
|
||||||
|
public byte[] snapshotCurrentState() throws Exception {
|
||||||
|
// this method assumes that the checkpoint lock is held
|
||||||
|
assert Thread.holdsLock(checkpointLock);
|
||||||
|
if (debeziumOffset.sourceOffset == null || debeziumOffset.sourcePartition == null) {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
return stateSerializer.serialize(debeziumOffset);
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Process change messages from the {@link Handover} and collect the processed messages by
|
||||||
|
* {@link Collector}.
|
||||||
|
*/
|
||||||
|
public void runFetchLoop() throws Exception {
|
||||||
|
try {
|
||||||
|
// begin snapshot database phase
|
||||||
|
if (isInDbSnapshotPhase) {
|
||||||
|
List<ChangeEvent<SourceRecord, SourceRecord>> events = handover.pollNext();
|
||||||
|
|
||||||
|
synchronized (checkpointLock) {
|
||||||
|
LOG.info(
|
||||||
|
"Database snapshot phase can't perform checkpoint, acquired Checkpoint lock.");
|
||||||
|
handleBatch(events);
|
||||||
|
while (isRunning && isInDbSnapshotPhase) {
|
||||||
|
handleBatch(handover.pollNext());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
LOG.info("Received record from streaming binlog phase, released checkpoint lock.");
|
||||||
|
}
|
||||||
|
|
||||||
|
// begin streaming binlog phase
|
||||||
|
while (isRunning) {
|
||||||
|
// If the handover is closed or has errors, exit.
|
||||||
|
// If there is no streaming phase, the handover will be closed by the engine.
|
||||||
|
handleBatch(handover.pollNext());
|
||||||
|
}
|
||||||
|
} catch (Handover.ClosedException e) {
|
||||||
|
// ignore
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void close() {
|
||||||
|
isRunning = false;
|
||||||
|
handover.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
// ---------------------------------------------------------------------------------------
|
||||||
|
// Metric getter
|
||||||
|
// ---------------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
public long getFetchDelay() {
|
||||||
|
return fetchDelay;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getEmitDelay() {
|
||||||
|
return emitDelay;
|
||||||
|
}
|
||||||
|
|
||||||
|
public long getIdleTime() {
|
||||||
|
return System.currentTimeMillis() - processTime;
|
||||||
|
}
|
||||||
|
|
||||||
|
// ---------------------------------------------------------------------------------------
|
||||||
|
// Helper
|
||||||
|
// ---------------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
private void handleBatch(List<ChangeEvent<SourceRecord, SourceRecord>> changeEvents)
|
||||||
|
throws Exception {
|
||||||
|
if (CollectionUtils.isEmpty(changeEvents)) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
this.processTime = System.currentTimeMillis();
|
||||||
|
|
||||||
|
for (ChangeEvent<SourceRecord, SourceRecord> event : changeEvents) {
|
||||||
|
SourceRecord record = event.value();
|
||||||
|
updateMessageTimestamp(record);
|
||||||
|
fetchDelay = processTime - messageTimestamp;
|
||||||
|
|
||||||
|
if (isHeartbeatEvent(record)) {
|
||||||
|
// keep offset update
|
||||||
|
synchronized (checkpointLock) {
|
||||||
|
debeziumOffset.setSourcePartition(record.sourcePartition());
|
||||||
|
debeziumOffset.setSourceOffset(record.sourceOffset());
|
||||||
|
}
|
||||||
|
// drop heartbeat events
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
deserialization.deserialize(record, debeziumCollector);
|
||||||
|
|
||||||
|
if (!isSnapshotRecord(record)) {
|
||||||
|
LOG.debug("Snapshot phase finishes.");
|
||||||
|
isInDbSnapshotPhase = false;
|
||||||
|
}
|
||||||
|
|
||||||
|
// emit the actual records. this also updates offset state atomically
|
||||||
|
emitRecordsUnderCheckpointLock(
|
||||||
|
debeziumCollector.records, record.sourcePartition(), record.sourceOffset());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void emitRecordsUnderCheckpointLock(
|
||||||
|
Queue<T> records, Map<String, ?> sourcePartition, Map<String, ?> sourceOffset) {
|
||||||
|
// Emit the records. Use the checkpoint lock to guarantee
|
||||||
|
// atomicity of record emission and offset state update.
|
||||||
|
// The synchronized checkpointLock is reentrant. It's safe to sync again in snapshot mode.
|
||||||
|
synchronized (checkpointLock) {
|
||||||
|
T record;
|
||||||
|
while ((record = records.poll()) != null) {
|
||||||
|
emitDelay = System.currentTimeMillis() - messageTimestamp;
|
||||||
|
sourceContext.collect(record);
|
||||||
|
}
|
||||||
|
// update offset to state
|
||||||
|
debeziumOffset.setSourcePartition(sourcePartition);
|
||||||
|
debeziumOffset.setSourceOffset(sourceOffset);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void updateMessageTimestamp(SourceRecord record) {
|
||||||
|
Schema schema = record.valueSchema();
|
||||||
|
Struct value = (Struct) record.value();
|
||||||
|
if (schema.field(Envelope.FieldName.SOURCE) == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
Struct source = value.getStruct(Envelope.FieldName.SOURCE);
|
||||||
|
if (source.schema().field(Envelope.FieldName.TIMESTAMP) == null) {
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
Long tsMs = source.getInt64(Envelope.FieldName.TIMESTAMP);
|
||||||
|
if (tsMs != null) {
|
||||||
|
this.messageTimestamp = tsMs;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean isHeartbeatEvent(SourceRecord record) {
|
||||||
|
String topic = record.topic();
|
||||||
|
return topic != null && topic.startsWith(heartbeatTopicPrefix);
|
||||||
|
}
|
||||||
|
|
||||||
|
private boolean isSnapshotRecord(SourceRecord record) {
|
||||||
|
Struct value = (Struct) record.value();
|
||||||
|
if (value != null) {
|
||||||
|
Struct source = value.getStruct(Envelope.FieldName.SOURCE);
|
||||||
|
SnapshotRecord snapshotRecord = SnapshotRecord.fromSource(source);
|
||||||
|
// even if it is the last record of snapshot, i.e. SnapshotRecord.LAST
|
||||||
|
// we can still recover from checkpoint and continue to read the binlog,
|
||||||
|
// because the checkpoint contains binlog position
|
||||||
|
return SnapshotRecord.TRUE == snapshotRecord;
|
||||||
|
}
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
// ---------------------------------------------------------------------------------------
|
||||||
|
|
||||||
|
private class DebeziumCollector implements Collector<T> {
|
||||||
|
|
||||||
|
private final Queue<T> records = new ArrayDeque<>();
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void collect(T record) {
|
||||||
|
records.add(record);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() {}
|
||||||
|
}
|
||||||
|
}
|
@ -1,28 +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 com.alibaba.ververica.cdc.debezium.internal;
|
|
||||||
|
|
||||||
import org.apache.flink.annotation.Internal;
|
|
||||||
|
|
||||||
/** A reporter that can report errors to handler. */
|
|
||||||
@Internal
|
|
||||||
public interface ErrorReporter {
|
|
||||||
|
|
||||||
void reportError(Throwable error);
|
|
||||||
}
|
|
@ -0,0 +1,197 @@
|
|||||||
|
/*
|
||||||
|
* 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 com.alibaba.ververica.cdc.debezium.internal;
|
||||||
|
|
||||||
|
import org.apache.flink.annotation.Internal;
|
||||||
|
import org.apache.flink.util.ExceptionUtils;
|
||||||
|
|
||||||
|
import io.debezium.engine.ChangeEvent;
|
||||||
|
import org.apache.kafka.connect.source.SourceRecord;
|
||||||
|
import org.slf4j.Logger;
|
||||||
|
import org.slf4j.LoggerFactory;
|
||||||
|
|
||||||
|
import javax.annotation.concurrent.GuardedBy;
|
||||||
|
import javax.annotation.concurrent.ThreadSafe;
|
||||||
|
|
||||||
|
import java.io.Closeable;
|
||||||
|
import java.util.Collections;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
import static org.apache.flink.util.Preconditions.checkNotNull;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* The Handover is a utility to hand over data (a buffer of records) and exception from a
|
||||||
|
* <i>producer</i> thread to a <i>consumer</i> thread. It effectively behaves like a "size one
|
||||||
|
* blocking queue", with some extras around exception reporting, closing, and waking up thread
|
||||||
|
* without {@link Thread#interrupt() interrupting} threads.
|
||||||
|
*
|
||||||
|
* <p>This class is used in the Flink Debezium Engine Consumer to hand over data and exceptions
|
||||||
|
* between the thread that runs the DebeziumEngine class and the main thread.
|
||||||
|
*
|
||||||
|
* <p>The Handover can also be "closed", signalling from one thread to the other that it the thread
|
||||||
|
* has terminated.
|
||||||
|
*/
|
||||||
|
@ThreadSafe
|
||||||
|
@Internal
|
||||||
|
public class Handover implements Closeable {
|
||||||
|
|
||||||
|
private static final Logger LOG = LoggerFactory.getLogger(Handover.class);
|
||||||
|
private final Object lock = new Object();
|
||||||
|
|
||||||
|
@GuardedBy("lock")
|
||||||
|
private List<ChangeEvent<SourceRecord, SourceRecord>> next;
|
||||||
|
|
||||||
|
@GuardedBy("lock")
|
||||||
|
private Throwable error;
|
||||||
|
|
||||||
|
private boolean wakeupProducer;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Polls the next element from the Handover, possibly blocking until the next element is
|
||||||
|
* available. This method behaves similar to polling from a blocking queue.
|
||||||
|
*
|
||||||
|
* <p>If an exception was handed in by the producer ({@link #reportError(Throwable)}), then that
|
||||||
|
* exception is thrown rather than an element being returned.
|
||||||
|
*
|
||||||
|
* @return The next element (buffer of records, never null).
|
||||||
|
* @throws ClosedException Thrown if the Handover was {@link #close() closed}.
|
||||||
|
* @throws Exception Rethrows exceptions from the {@link #reportError(Throwable)} method.
|
||||||
|
*/
|
||||||
|
public List<ChangeEvent<SourceRecord, SourceRecord>> pollNext() throws Exception {
|
||||||
|
synchronized (lock) {
|
||||||
|
while (next == null && error == null) {
|
||||||
|
lock.wait();
|
||||||
|
}
|
||||||
|
List<ChangeEvent<SourceRecord, SourceRecord>> n = next;
|
||||||
|
if (n != null) {
|
||||||
|
next = null;
|
||||||
|
lock.notifyAll();
|
||||||
|
return n;
|
||||||
|
} else {
|
||||||
|
ExceptionUtils.rethrowException(error, error.getMessage());
|
||||||
|
|
||||||
|
// this statement cannot be reached since the above method always throws an
|
||||||
|
// exception this is only here to silence the compiler and any warnings
|
||||||
|
return Collections.emptyList();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Hands over an element from the producer. If the Handover already has an element that was not
|
||||||
|
* yet picked up by the consumer thread, this call blocks until the consumer picks up that
|
||||||
|
* previous element.
|
||||||
|
*
|
||||||
|
* <p>This behavior is similar to a "size one" blocking queue.
|
||||||
|
*
|
||||||
|
* @param element The next element to hand over.
|
||||||
|
* @throws InterruptedException Thrown, if the thread is interrupted while blocking for the
|
||||||
|
* Handover to be empty.
|
||||||
|
*/
|
||||||
|
public void produce(final List<ChangeEvent<SourceRecord, SourceRecord>> element)
|
||||||
|
throws InterruptedException {
|
||||||
|
|
||||||
|
checkNotNull(element);
|
||||||
|
|
||||||
|
synchronized (lock) {
|
||||||
|
while (next != null && !wakeupProducer) {
|
||||||
|
lock.wait();
|
||||||
|
}
|
||||||
|
|
||||||
|
wakeupProducer = false;
|
||||||
|
|
||||||
|
// an error marks this as closed for the producer
|
||||||
|
if (error != null) {
|
||||||
|
ExceptionUtils.rethrow(error, error.getMessage());
|
||||||
|
} else {
|
||||||
|
// if there is no error, then this is open and can accept this element
|
||||||
|
next = element;
|
||||||
|
lock.notifyAll();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Reports an exception. The consumer will throw the given exception immediately, if it is
|
||||||
|
* currently blocked in the {@link #pollNext()} method, or the next time it calls that method.
|
||||||
|
*
|
||||||
|
* <p>After this method has been called, no call to either {@link #produce( List)} or {@link
|
||||||
|
* #pollNext()} will ever return regularly any more, but will always return exceptionally.
|
||||||
|
*
|
||||||
|
* <p>If another exception was already reported, this method does nothing.
|
||||||
|
*
|
||||||
|
* <p>For the producer, the Handover will appear as if it was {@link #close() closed}.
|
||||||
|
*
|
||||||
|
* @param t The exception to report.
|
||||||
|
*/
|
||||||
|
public void reportError(Throwable t) {
|
||||||
|
checkNotNull(t);
|
||||||
|
|
||||||
|
synchronized (lock) {
|
||||||
|
LOG.error("Reporting error:", t);
|
||||||
|
// do not override the initial exception
|
||||||
|
if (error == null) {
|
||||||
|
error = t;
|
||||||
|
}
|
||||||
|
next = null;
|
||||||
|
lock.notifyAll();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return whether there is an error.
|
||||||
|
*
|
||||||
|
* @return whether there is an error
|
||||||
|
*/
|
||||||
|
public boolean hasError() {
|
||||||
|
return error != null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Closes the handover. Both the {@link #produce(List)} method and the {@link #pollNext()} will
|
||||||
|
* throw a {@link ClosedException} on any currently blocking and future invocations.
|
||||||
|
*
|
||||||
|
* <p>If an exception was previously reported via the {@link #reportError(Throwable)} method,
|
||||||
|
* that exception will not be overridden. The consumer thread will throw that exception upon
|
||||||
|
* calling {@link #pollNext()}, rather than the {@code ClosedException}.
|
||||||
|
*/
|
||||||
|
@Override
|
||||||
|
public void close() {
|
||||||
|
synchronized (lock) {
|
||||||
|
next = null;
|
||||||
|
wakeupProducer = false;
|
||||||
|
|
||||||
|
if (error == null) {
|
||||||
|
error = new ClosedException();
|
||||||
|
}
|
||||||
|
lock.notifyAll();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// ------------------------------------------------------------------------
|
||||||
|
|
||||||
|
/**
|
||||||
|
* An exception thrown by the Handover in the {@link #pollNext()} or {@link #produce(List)}
|
||||||
|
* method, after the Handover was closed via {@link #close()}.
|
||||||
|
*/
|
||||||
|
public static final class ClosedException extends Exception {
|
||||||
|
|
||||||
|
private static final long serialVersionUID = 1L;
|
||||||
|
}
|
||||||
|
}
|
Loading…
Reference in New Issue