-
Notifications
You must be signed in to change notification settings - Fork 13.4k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-36625] add lineage helper class for connector integration
- Loading branch information
1 parent
02510f1
commit 8171f17
Showing
9 changed files
with
253 additions
and
12 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
62 changes: 62 additions & 0 deletions
62
...ng-java/src/main/java/org/apache/flink/streaming/api/lineage/DefaultTypeDatasetFacet.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,62 @@ | ||
/* | ||
* 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.streaming.api.lineage; | ||
|
||
import org.apache.flink.annotation.PublicEvolving; | ||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
|
||
import java.util.Objects; | ||
|
||
/** Default implementation of {@link TypeDatasetFacet}. */ | ||
@PublicEvolving | ||
public class DefaultTypeDatasetFacet implements TypeDatasetFacet { | ||
|
||
public static final String TYPE_FACET_NAME = "type"; | ||
|
||
private final TypeInformation typeInformation; | ||
|
||
public DefaultTypeDatasetFacet(TypeInformation typeInformation) { | ||
this.typeInformation = typeInformation; | ||
} | ||
|
||
public TypeInformation getTypeInformation() { | ||
return typeInformation; | ||
} | ||
|
||
public boolean equals(Object o) { | ||
if (this == o) { | ||
return true; | ||
} | ||
if (o == null || getClass() != o.getClass()) { | ||
return false; | ||
} | ||
DefaultTypeDatasetFacet that = (DefaultTypeDatasetFacet) o; | ||
return Objects.equals(typeInformation, that.typeInformation); | ||
} | ||
|
||
@Override | ||
public int hashCode() { | ||
return Objects.hash(typeInformation); | ||
} | ||
|
||
@Override | ||
public String name() { | ||
return TYPE_FACET_NAME; | ||
} | ||
} |
50 changes: 50 additions & 0 deletions
50
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/lineage/LineageUtil.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,50 @@ | ||
/* | ||
* 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.streaming.api.lineage; | ||
|
||
import org.apache.flink.api.connector.source.Boundedness; | ||
|
||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.stream.Collectors; | ||
|
||
public class LineageUtil { | ||
|
||
public static LineageDataset datasetOf( | ||
String name, String namespace, TypeDatasetFacet typeDatasetFacet) { | ||
return datasetOf(name, namespace, Collections.singletonList(typeDatasetFacet)); | ||
} | ||
|
||
public static LineageDataset datasetOf( | ||
String name, String namespace, List<LineageDatasetFacet> facets) { | ||
return new DefaultLineageDataset( | ||
name, | ||
namespace, | ||
facets.stream().collect(Collectors.toMap(LineageDatasetFacet::name, item -> item))); | ||
} | ||
|
||
public static SourceLineageVertex sourceLineageVertexOf( | ||
Boundedness boundedness, LineageDataset dataset) { | ||
return new DefaultSourceLineageVertex(boundedness, Collections.singletonList(dataset)); | ||
} | ||
|
||
public static LineageVertex lineageVertexOf(LineageDataset dataset) { | ||
return new DefaultLineageVertex(Collections.singletonList(dataset)); | ||
} | ||
} |
29 changes: 29 additions & 0 deletions
29
...streaming-java/src/main/java/org/apache/flink/streaming/api/lineage/TypeDatasetFacet.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,29 @@ | ||
/* | ||
* 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.streaming.api.lineage; | ||
|
||
import org.apache.flink.annotation.PublicEvolving; | ||
import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
|
||
/** Facet definition to contain type information of source and sink. */ | ||
@PublicEvolving | ||
public interface TypeDatasetFacet extends LineageDatasetFacet { | ||
|
||
TypeInformation getTypeInformation(); | ||
} |
34 changes: 34 additions & 0 deletions
34
...g-java/src/main/java/org/apache/flink/streaming/api/lineage/TypeDatasetFacetProvider.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,34 @@ | ||
/* | ||
* 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.streaming.api.lineage; | ||
|
||
import org.apache.flink.annotation.PublicEvolving; | ||
|
||
import java.util.Optional; | ||
|
||
/** Contains method to extract {@link TypeDatasetFacet}. */ | ||
@PublicEvolving | ||
public interface TypeDatasetFacetProvider { | ||
|
||
/** | ||
* Returns a type dataset facet or `Optional.empty` in case an implementing class is not able to | ||
* resolve type. | ||
*/ | ||
Optional<TypeDatasetFacet> getTypeDatasetFacet(); | ||
} |
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
...-streaming-java/src/test/java/org/apache/flink/streaming/api/lineage/LineageUtilTest.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.streaming.api.lineage; | ||
|
||
import org.apache.flink.api.common.typeinfo.Types; | ||
import org.apache.flink.api.connector.source.Boundedness; | ||
|
||
import org.junit.jupiter.api.Test; | ||
|
||
import static org.assertj.core.api.Assertions.assertThat; | ||
|
||
/** Testing for lineage util. */ | ||
public class LineageUtilTest { | ||
private static final String TEST_NAME = "testName"; | ||
private static final String TEST_NAMESPACE = "testNameSpace"; | ||
|
||
@Test | ||
public void testDataSetOf() { | ||
DefaultTypeDatasetFacet typeDatasetFacet = new DefaultTypeDatasetFacet(Types.BIG_INT); | ||
LineageDataset dataset = LineageUtil.datasetOf(TEST_NAME, TEST_NAMESPACE, typeDatasetFacet); | ||
|
||
assertThat(dataset.name()).isEqualTo(TEST_NAME); | ||
assertThat(dataset.namespace()).isEqualTo(TEST_NAMESPACE); | ||
assertThat(dataset.facets()).size().isEqualTo(1); | ||
assertThat(dataset.facets().get(typeDatasetFacet.name())).isEqualTo(typeDatasetFacet); | ||
} | ||
|
||
@Test | ||
public void testSourceLineageVertexOf() { | ||
LineageDataset dataset = | ||
LineageUtil.datasetOf( | ||
TEST_NAME, TEST_NAMESPACE, new DefaultTypeDatasetFacet(Types.BIG_INT)); | ||
SourceLineageVertex sourceLineageVertex = | ||
LineageUtil.sourceLineageVertexOf(Boundedness.CONTINUOUS_UNBOUNDED, dataset); | ||
|
||
assertThat(sourceLineageVertex.boundedness()).isEqualTo(Boundedness.CONTINUOUS_UNBOUNDED); | ||
assertThat(sourceLineageVertex.datasets()).containsExactly(dataset); | ||
} | ||
|
||
@Test | ||
public void testLineageVertexOf() { | ||
LineageDataset dataset = | ||
LineageUtil.datasetOf( | ||
TEST_NAME, TEST_NAMESPACE, new DefaultTypeDatasetFacet(Types.BIG_INT)); | ||
LineageVertex lineageVertex = LineageUtil.lineageVertexOf(dataset); | ||
assertThat(lineageVertex.datasets()).containsExactly(dataset); | ||
} | ||
} |
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