forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-24440][source] Announce and combine latest watermarks across S…
…ourceOperators
- Loading branch information
Showing
7 changed files
with
412 additions
and
10 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
64 changes: 64 additions & 0 deletions
64
...k-runtime/src/main/java/org/apache/flink/runtime/source/event/ReportedWatermarkEvent.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,64 @@ | ||
/* | ||
* 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 org.apache.flink.runtime.source.event; | ||
|
||
import org.apache.flink.api.common.eventtime.Watermark; | ||
import org.apache.flink.runtime.operators.coordination.OperatorEvent; | ||
|
||
import java.util.Objects; | ||
|
||
/** | ||
* Reports last emitted {@link Watermark} from a subtask to the {@link | ||
* org.apache.flink.runtime.source.coordinator.SourceCoordinator}. | ||
*/ | ||
public class ReportedWatermarkEvent implements OperatorEvent { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
private final long watermark; | ||
|
||
public ReportedWatermarkEvent(long watermark) { | ||
this.watermark = watermark; | ||
} | ||
|
||
public long getWatermark() { | ||
return watermark; | ||
} | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
ReportedWatermarkEvent that = (ReportedWatermarkEvent) o; | ||
return watermark == that.watermark; | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(watermark); | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return getClass().getSimpleName() + "{" + "watermark=" + watermark + '}'; | ||
} | ||
} |
60 changes: 60 additions & 0 deletions
60
...-runtime/src/main/java/org/apache/flink/runtime/source/event/WatermarkAlignmentEvent.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,60 @@ | ||
/* | ||
* 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 org.apache.flink.runtime.source.event; | ||
|
||
import org.apache.flink.runtime.operators.coordination.OperatorEvent; | ||
|
||
import java.util.Objects; | ||
|
||
/** Signals source operators the maximum watermark that emitted records can have. */ | ||
public class WatermarkAlignmentEvent implements OperatorEvent { | ||
|
||
private static final long serialVersionUID = 1L; | ||
|
||
private final long maxWatermark; | ||
|
||
public WatermarkAlignmentEvent(long maxWatermark) { | ||
this.maxWatermark = maxWatermark; | ||
} | ||
|
||
public long getMaxWatermark() { | ||
return maxWatermark; | ||
} | ||
|
||
@Override | ||
public boolean equals(Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
WatermarkAlignmentEvent that = (WatermarkAlignmentEvent) o; | ||
return maxWatermark == that.maxWatermark; | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(maxWatermark); | ||
} | ||
|
||
@Override | ||
public String toString() { | ||
return "WatermarkAlignmentEvent{" + "maxWatermark=" + maxWatermark + '}'; | ||
} | ||
} |
Oops, something went wrong.