Skip to content

Commit

Permalink
fix
Browse files Browse the repository at this point in the history
  • Loading branch information
snuyanzin committed Aug 16, 2024
1 parent bbbf2ae commit 9c07fda
Show file tree
Hide file tree
Showing 9 changed files with 137 additions and 145 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -23,22 +23,21 @@
import org.apache.flink.table.functions.SqlLikeUtils;
import org.apache.flink.table.operations.utils.LikeOp;

import javax.annotation.Nullable;

import java.util.Collection;
import java.util.Objects;

import static org.apache.flink.table.api.internal.TableResultUtils.buildStringArrayResult;

@Internal
public abstract class AbstractOneColumnShowOperation implements ShowOperation {
private final String catalogName;
private final String databaseName;
private final String preposition;
private final LikeOp likeOp;
private final @Nullable String catalogName;
private final @Nullable String preposition;
private final @Nullable LikeOp likeOp;

public AbstractOneColumnShowOperation(
String catalogName, String databaseName, String preposition, LikeOp likeOp) {
public AbstractOneColumnShowOperation(String catalogName, String preposition, LikeOp likeOp) {
this.catalogName = catalogName;
this.databaseName = databaseName;
this.preposition = preposition;
this.likeOp = likeOp;
}
Expand All @@ -49,69 +48,59 @@ public AbstractOneColumnShowOperation(

public abstract String getColumnName();

public AbstractOneColumnShowOperation() {
this(null, null, null, null);
}

String getCatalogName() {
return catalogName;
}

String getDatabaseName() {
return databaseName;
String getPreposition() {
return preposition;
}

LikeOp getLikeOp() {
return likeOp;
}

String getPreposition() {
return preposition;
}

@Override
public TableResultInternal execute(Context ctx) {
final Collection<String> views = retrieveDataForTableResult(ctx);
final String[] rows;
if (likeOp != null) {
rows =
views.stream()
.filter(row -> likeFilter(row, likeOp))
.sorted()
.toArray(String[]::new);
} else {
rows = views.stream().sorted().toArray(String[]::new);
}
final String[] rows =
views.stream()
.filter(row -> likeFilter(row, likeOp))
.sorted()
.toArray(String[]::new);
return buildStringArrayResult(getColumnName(), rows);
}

private boolean likeFilter(String row, LikeOp likeOp) {
public boolean likeFilter(String row, @Nullable LikeOp likeOp) {
if (likeOp == null) {
return true;
}
final boolean notLike = likeOp.isNot();
final boolean isILike = likeOp.isILike();
final String sqlLikePattern = likeOp.getLikePattern();
boolean likeMatch =
final String likePattern = likeOp.getLikePattern();
final boolean likeMatch =
isILike
? SqlLikeUtils.ilike(row, sqlLikePattern, "\\")
: SqlLikeUtils.like(row, sqlLikePattern, "\\");
? SqlLikeUtils.ilike(row, likePattern, "\\")
: SqlLikeUtils.like(row, likePattern, "\\");
return notLike != likeMatch;
}

@Override
public String asSummaryString() {
StringBuilder builder = new StringBuilder().append(getOperationName());
if (preposition != null) {
if (databaseName != null) {
builder.append(String.format(" %s %s.%s", preposition, catalogName, databaseName));
} else {
builder.append(String.format(" %s %s", preposition, catalogName));
}
builder.append(" ").append(getPrepositionSummaryString());
}
if (likeOp != null) {
builder.append(" ").append(likeOp);
}
return builder.toString();
}

public String getPrepositionSummaryString() {
return String.format("%s %s", preposition, catalogName);
}

@Override
public boolean equals(Object o) {
if (this == o) {
Expand All @@ -122,14 +111,13 @@ public boolean equals(Object o) {
}
AbstractOneColumnShowOperation that = (AbstractOneColumnShowOperation) o;
return Objects.equals(catalogName, that.catalogName)
&& Objects.equals(databaseName, that.databaseName)
&& Objects.equals(preposition, that.preposition)
&& Objects.equals(likeOp, that.likeOp);
}

@Override
public int hashCode() {
return Objects.hash(catalogName, databaseName, preposition, likeOp);
return Objects.hash(catalogName, preposition, likeOp);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,8 +61,7 @@ public String asSummaryString() {
@Override
public TableResultInternal execute(Context ctx) {
// DESCRIBE <table> is a synonym for SHOW COLUMNS without LIKE pattern.
ShowColumnsOperation showColumns =
new ShowColumnsOperation(sqlIdentifier, null, false, false, "FROM");
ShowColumnsOperation showColumns = new ShowColumnsOperation(sqlIdentifier, "FROM", null);
return showColumns.execute(ctx);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -27,14 +27,16 @@
import org.apache.flink.table.catalog.ObjectIdentifier;
import org.apache.flink.table.catalog.ResolvedSchema;
import org.apache.flink.table.catalog.WatermarkSpec;
import org.apache.flink.table.functions.SqlLikeUtils;
import org.apache.flink.table.operations.utils.LikeOp;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.LogicalType;

import javax.annotation.Nullable;

import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
Expand All @@ -45,62 +47,20 @@

/** Show columns from [[catalog.]database.]table. */
@Internal
public class ShowColumnsOperation implements ShowOperation {
public class ShowColumnsOperation extends AbstractOneColumnShowOperation {

private final ObjectIdentifier tableIdentifier;
private final boolean useLike;
private final boolean notLike;
private final @Nullable String likePattern;
private final String preposition;

public ShowColumnsOperation(
ObjectIdentifier tableIdentifier,
@Nullable String likePattern,
boolean useLike,
boolean notLike,
String preposition) {
ObjectIdentifier tableIdentifier, String preposition, @Nullable LikeOp likeOp) {
super(tableIdentifier.getCatalogName(), preposition, likeOp);
this.tableIdentifier = tableIdentifier;
this.likePattern = likePattern;
this.useLike = useLike;
this.notLike = notLike;
this.preposition = preposition;
}

public String getLikePattern() {
return likePattern;
}

public String getPreposition() {
return preposition;
}

public boolean isUseLike() {
return useLike;
}

public boolean isNotLike() {
return notLike;
}

public ObjectIdentifier getTableIdentifier() {
return tableIdentifier;
}

@Override
public String asSummaryString() {
if (useLike) {
if (notLike) {
return String.format(
"SHOW COLUMNS %s %s %s LIKE '%s'",
preposition, tableIdentifier.asSummaryString(), "NOT", likePattern);
}
return String.format(
"SHOW COLUMNS %s %s LIKE '%s'",
preposition, tableIdentifier.asSummaryString(), likePattern);
}
return String.format("SHOW COLUMNS %s %s", preposition, tableIdentifier.asSummaryString());
}

@Override
public TableResultInternal execute(Context ctx) {
Optional<ContextResolvedTable> result = ctx.getCatalogManager().getTable(tableIdentifier);
Expand All @@ -113,14 +73,11 @@ public TableResultInternal execute(Context ctx) {

ResolvedSchema schema = result.get().getResolvedSchema();
Object[][] rows = generateTableColumnsRows(schema);
if (useLike) {
LikeOp likeOp = getLikeOp();
if (likeOp != null) {
rows =
Arrays.stream(rows)
.filter(
row ->
notLike
!= SqlLikeUtils.like(
row[0].toString(), likePattern, "\\"))
.filter(row -> likeFilter(row[0].toString(), likeOp))
.toArray(Object[][]::new);
}
boolean nonComments = isSchemaNonColumnComments(schema);
Expand Down Expand Up @@ -204,4 +161,21 @@ private DataType[] generateTableColumnsDataTypes(boolean nonComments) {
}
return result.toArray(new DataType[0]);
}

@Override
public String getOperationName() {
return "SHOW COLUMNS";
}

@Override
public String getColumnName() {
// Dummy implementation since the main logic is overridden in execute method
return null;
}

@Override
public Collection<String> retrieveDataForTableResult(Context ctx) {
// Dummy implementation since the main logic is overridden in execute method
return Collections.emptyList();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,7 @@
public class ShowDatabasesOperation extends AbstractOneColumnShowOperation {

public ShowDatabasesOperation(String catalogName, String preposition, LikeOp likeOp) {
super(catalogName, null, preposition, likeOp);
super(catalogName, preposition, likeOp);
}

public ShowDatabasesOperation(LikeOp likeOp) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,8 @@
import org.apache.flink.table.functions.FunctionIdentifier;
import org.apache.flink.table.operations.utils.LikeOp;

import javax.annotation.Nullable;

import java.util.Arrays;
import java.util.Collection;
import java.util.stream.Collectors;
Expand All @@ -48,6 +50,7 @@ public enum FunctionScope {
}

private final FunctionScope functionScope;
private final @Nullable String databaseName;

public ShowFunctionsOperation() {
// "SHOW FUNCTIONS" default is ALL scope
Expand All @@ -64,53 +67,40 @@ public ShowFunctionsOperation(
String catalogName,
String databaseName,
LikeOp likeOp) {
super(catalogName, databaseName, preposition, likeOp);
super(catalogName, preposition, likeOp);
this.functionScope = functionScope;
this.databaseName = databaseName;
}

@Override
public Collection<String> retrieveDataForTableResult(Context ctx) {
final String[] functionNames;
final String preposition = getPreposition();
final String catalogName = getCatalogName();
final String databaseName = getDatabaseName();
if (preposition == null) {
// it's to show current_catalog.current_database
switch (functionScope) {
case USER:
switch (functionScope) {
case USER:
if (preposition == null) {
return Arrays.asList(ctx.getFunctionCatalog().getUserDefinedFunctions());
case ALL:
}
return ctx.getFunctionCatalog().getUserDefinedFunctions(catalogName, databaseName)
.stream()
.map(FunctionIdentifier::getFunctionName)
.collect(Collectors.toList());
case ALL:
if (preposition == null) {
return Arrays.asList(ctx.getFunctionCatalog().getFunctions());
default:
throw new UnsupportedOperationException(
String.format(
"SHOW FUNCTIONS with %s scope is not supported.",
functionScope));
}
} else {
switch (functionScope) {
case USER:
return ctx.getFunctionCatalog()
.getUserDefinedFunctions(catalogName, databaseName).stream()
.map(FunctionIdentifier::getFunctionName)
.collect(Collectors.toList());
case ALL:
return Arrays.asList(
ctx.getFunctionCatalog().getFunctions(catalogName, databaseName));
default:
throw new UnsupportedOperationException(
String.format(
"SHOW FUNCTIONS with %s scope is not supported.",
functionScope));
}
}
return Arrays.asList(
ctx.getFunctionCatalog().getFunctions(catalogName, databaseName));
default:
throw new UnsupportedOperationException(
String.format(
"SHOW FUNCTIONS with %s scope is not supported.", functionScope));
}
}

@Override
public String getOperationName() {
return functionScope == FunctionScope.ALL
? "SHOW FUNCTIONS"
: String.format("SHOW %s FUNCTIONS", functionScope);
return functionScope == FunctionScope.ALL ? "SHOW FUNCTIONS" : "SHOW USER FUNCTIONS";
}

@Override
Expand All @@ -121,4 +111,12 @@ public String getColumnName() {
public FunctionScope getFunctionScope() {
return functionScope;
}

@Override
public String getPrepositionSummaryString() {
if (databaseName == null) {
return super.getPrepositionSummaryString();
}
return super.getPrepositionSummaryString() + "." + databaseName;
}
}
Loading

0 comments on commit 9c07fda

Please sign in to comment.