-
Notifications
You must be signed in to change notification settings - Fork 13.5k
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
[FLINK-36625] add lineage helper class for connector integration #25712
base: master
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
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; | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,53 @@ | ||
/* | ||
* 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.Internal; | ||
import org.apache.flink.api.connector.source.Boundedness; | ||
|
||
import java.util.Collections; | ||
import java.util.List; | ||
import java.util.stream.Collectors; | ||
|
||
/** Util class for creating default LineageDataset and LineageVertex. */ | ||
@Internal | ||
public class LineageUtils { | ||
|
||
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)); | ||
} | ||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,32 @@ | ||
/* | ||
* 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 javax.annotation.Nonnull; | ||
|
||
/** Facet definition to contain type information of source and sink. */ | ||
@PublicEvolving | ||
public interface TypeDatasetFacet extends LineageDatasetFacet { | ||
|
||
@Nonnull | ||
TypeInformation getTypeInformation(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. |
||
} |
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(); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see this is a new interface but it is not referenced in the fix, what is the thinking here? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This interface is mainly for some connector in which return type is provided by internal class rather than the source/sink directly. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Thanks for the explanation. It would be good to understand when a connector author should use this interface. Could we add documentation around when this and the utility classes could/ should be used ? |
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,65 @@ | ||
/* | ||
* 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 LineageUtilsTest { | ||
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 = | ||
LineageUtils.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 = | ||
LineageUtils.datasetOf( | ||
TEST_NAME, TEST_NAMESPACE, new DefaultTypeDatasetFacet(Types.BIG_INT)); | ||
SourceLineageVertex sourceLineageVertex = | ||
LineageUtils.sourceLineageVertexOf(Boundedness.CONTINUOUS_UNBOUNDED, dataset); | ||
|
||
assertThat(sourceLineageVertex.boundedness()).isEqualTo(Boundedness.CONTINUOUS_UNBOUNDED); | ||
assertThat(sourceLineageVertex.datasets()).containsExactly(dataset); | ||
} | ||
|
||
@Test | ||
public void testLineageVertexOf() { | ||
LineageDataset dataset = | ||
LineageUtils.datasetOf( | ||
TEST_NAME, TEST_NAMESPACE, new DefaultTypeDatasetFacet(Types.BIG_INT)); | ||
LineageVertex lineageVertex = LineageUtils.lineageVertexOf(dataset); | ||
assertThat(lineageVertex.datasets()).containsExactly(dataset); | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I wonder if these datasetOf constructor methods can be put in DefaultLineageDataset or a class that creates DefaultLineageDataset , maybe a DefaultLineageDatasetProvider
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It is mainly to make the LineageDataset creation easier and simplify the code in each of connectors.