Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 7 additions & 0 deletions docs/src/main/sphinx/connector/faker.md
Original file line number Diff line number Diff line change
Expand Up @@ -126,6 +126,13 @@ See the Datafaker's documentation for more information about
[the expression](https://www.datafaker.net/documentation/expressions/) syntax
and [available providers](https://www.datafaker.net/documentation/providers/).

To test a generator expression, without having to recreate the table, use the
`random_string` function from the `default` schema:

```sql
SELECT default.random_string('#{Name.first_name}')
```

### Non-character types

Faker supports the following non-character types:
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,11 +25,13 @@
import io.trino.spi.connector.ConnectorSession;
import io.trino.spi.connector.ConnectorSplitManager;
import io.trino.spi.connector.ConnectorTransactionHandle;
import io.trino.spi.function.FunctionProvider;
import io.trino.spi.session.PropertyMetadata;
import io.trino.spi.transaction.IsolationLevel;
import jakarta.inject.Inject;

import java.util.List;
import java.util.Optional;
import java.util.Set;

import static io.trino.spi.StandardErrorCode.INVALID_COLUMN_PROPERTY;
Expand All @@ -48,18 +50,21 @@ public class FakerConnector
private final FakerSplitManager splitManager;
private final FakerPageSourceProvider pageSourceProvider;
private final FakerPageSinkProvider pageSinkProvider;
private final FakerFunctionProvider functionProvider;

@Inject
public FakerConnector(
FakerMetadata metadata,
FakerSplitManager splitManager,
FakerPageSourceProvider pageSourceProvider,
FakerPageSinkProvider pageSinkProvider)
FakerPageSinkProvider pageSinkProvider,
FakerFunctionProvider functionProvider)
{
this.metadata = requireNonNull(metadata, "metadata is null");
this.splitManager = requireNonNull(splitManager, "splitManager is null");
this.pageSourceProvider = requireNonNull(pageSourceProvider, "pageSourceProvider is null");
this.pageSinkProvider = requireNonNull(pageSinkProvider, "pageSinkProvider is null");
this.functionProvider = requireNonNull(functionProvider, "functionPovider is null");
}

@Override
Expand Down Expand Up @@ -165,4 +170,10 @@ private static void checkProperty(boolean expression, ErrorCodeSupplier errorCod
throw new TrinoException(errorCode, errorMessage);
}
}

@Override
public Optional<FunctionProvider> getFunctionProvider()
{
return Optional.of(functionProvider);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
/*
* Licensed 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 io.trino.plugin.faker;

import com.google.inject.Inject;
import io.trino.metadata.SqlScalarFunction;
import io.trino.operator.scalar.annotations.ScalarFromAnnotationsParser;
import io.trino.spi.function.BoundSignature;
import io.trino.spi.function.FunctionDependencies;
import io.trino.spi.function.FunctionId;
import io.trino.spi.function.FunctionMetadata;
import io.trino.spi.function.FunctionProvider;
import io.trino.spi.function.InvocationConvention;
import io.trino.spi.function.ScalarFunctionImplementation;

import java.util.List;

import static com.google.common.collect.ImmutableList.toImmutableList;

public class FakerFunctionProvider
implements FunctionProvider
{
private final List<SqlScalarFunction> functions;

@Inject
public FakerFunctionProvider()
{
functions = ScalarFromAnnotationsParser.parseFunctionDefinitions(FakerFunctions.class);
}

@Override
public ScalarFunctionImplementation getScalarFunctionImplementation(
FunctionId functionId,
BoundSignature boundSignature,
FunctionDependencies functionDependencies,
InvocationConvention invocationConvention)
{
return functions.stream()
.filter(function -> function.getFunctionMetadata().getFunctionId().equals(functionId))
.map(function -> function.specialize(boundSignature, functionDependencies))
.findFirst()
.orElseThrow(() -> new IllegalArgumentException("Unknown function " + functionId))
.getScalarFunctionImplementation(invocationConvention);
}

public List<FunctionMetadata> functionsMetadata()
{
return functions.stream()
.map(SqlScalarFunction::getFunctionMetadata)
.collect(toImmutableList());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,42 @@
/*
* Licensed 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 io.trino.plugin.faker;

import io.airlift.slice.Slice;
import io.trino.spi.function.Description;
import io.trino.spi.function.ScalarFunction;
import io.trino.spi.function.SqlType;
import net.datafaker.Faker;

import static io.airlift.slice.Slices.utf8Slice;
import static io.trino.spi.type.StandardTypes.VARCHAR;

public final class FakerFunctions
{
private final Faker faker;

public FakerFunctions()
{
faker = new Faker();
}

@ScalarFunction(deterministic = false)
@Description("Generate a random string based on the Faker expression")
@SqlType(VARCHAR)
public Slice randomString(@SqlType(VARCHAR) Slice fakerExpression)
{
return utf8Slice(faker.expression(fakerExpression.toStringUtf8()));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,11 @@
import io.trino.spi.connector.SchemaTableName;
import io.trino.spi.connector.SchemaTablePrefix;
import io.trino.spi.connector.TableColumnsMetadata;
import io.trino.spi.function.BoundSignature;
import io.trino.spi.function.FunctionDependencyDeclaration;
import io.trino.spi.function.FunctionId;
import io.trino.spi.function.FunctionMetadata;
import io.trino.spi.function.SchemaFunctionName;
import io.trino.spi.predicate.Domain;
import io.trino.spi.predicate.TupleDomain;
import io.trino.spi.security.TrinoPrincipal;
Expand Down Expand Up @@ -72,6 +77,7 @@ public class FakerMetadata
implements ConnectorMetadata
{
public static final String SCHEMA_NAME = "default";
public static final String RANDOM_STRING_FUNCTION = "random_string";

@GuardedBy("this")
private final List<SchemaInfo> schemas = new ArrayList<>();
Expand All @@ -81,12 +87,15 @@ public class FakerMetadata
@GuardedBy("this")
private final Map<SchemaTableName, TableInfo> tables = new HashMap<>();

private final FakerFunctionProvider functionsProvider;

@Inject
public FakerMetadata(FakerConfig config)
public FakerMetadata(FakerConfig config, FakerFunctionProvider functionProvider)
{
this.schemas.add(new SchemaInfo(SCHEMA_NAME, Map.of()));
this.nullProbability = config.getNullProbability();
this.defaultLimit = config.getDefaultLimit();
this.functionsProvider = requireNonNull(functionProvider, "functionProvider is null");
}

@Override
Expand Down Expand Up @@ -432,4 +441,33 @@ public Optional<ConstraintApplicationResult<ConnectorTableHandle>> applyFilter(
constraint.getExpression(),
true));
}

@Override
public Collection<FunctionMetadata> listFunctions(ConnectorSession session, String schemaName)
{
return functionsProvider.functionsMetadata();
}

@Override
public Collection<FunctionMetadata> getFunctions(ConnectorSession session, SchemaFunctionName name)
{
return functionsProvider.functionsMetadata().stream()
.filter(function -> function.getCanonicalName().equals(name.getFunctionName()))
.collect(toImmutableList());
}

@Override
public FunctionMetadata getFunctionMetadata(ConnectorSession session, FunctionId functionId)
{
return functionsProvider.functionsMetadata().stream()
.filter(function -> function.getFunctionId().equals(functionId))
.findFirst()
.orElseThrow(() -> new IllegalArgumentException("Unknown function " + functionId));
}

@Override
public FunctionDependencyDeclaration getFunctionDependencies(ConnectorSession session, FunctionId functionId, BoundSignature boundSignature)
{
return FunctionDependencyDeclaration.NO_DEPENDENCIES;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,6 +48,7 @@ public void configure(Binder binder)
binder.bind(FakerSplitManager.class).in(Scopes.SINGLETON);
binder.bind(FakerPageSourceProvider.class).in(Scopes.SINGLETON);
binder.bind(FakerPageSinkProvider.class).in(Scopes.SINGLETON);
binder.bind(FakerFunctionProvider.class).in(Scopes.SINGLETON);
configBinder(binder).bindConfig(FakerConfig.class);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -245,6 +245,14 @@ void testSelectGenerator()
assertUpdate("DROP TABLE faker.default.generators");
}

@Test
void testSelectFunctions()
{
@Language("SQL")
String testQuery = "SELECT faker.default.random_string('#{options.option ''a'', ''b''}') IN ('a', 'b')";
assertQuery(testQuery, "VALUES (true)");
}

@Test
void testSelectRange()
{
Expand Down