Skip to content
Open
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
6 changes: 6 additions & 0 deletions docs/content/docs/dev/table/sql-gateway/overview.md
Original file line number Diff line number Diff line change
Expand Up @@ -250,6 +250,12 @@ $ ./sql-gateway -Dkey=value
<td>Duration</td>
<td>Keepalive time for an idle worker thread. When the number of workers exceeds min workers, excessive threads are killed after this time interval.</td>
</tr>
<tr>
<td><h5>sql-gateway.read-only</h5></td>
<td style="word-wrap: break-word;">5 min</td>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You meant false as default value?

<td>Duration</td>
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
<td>Duration</td>
<td>Boolean</td>

<td>When enabled, the SQL Gateway operates in read-only mode and will reject all modify operations.</td>
</tr>
<tr>
<td><h5>sql-gateway.worker.threads.max</h5></td>
<td style="word-wrap: break-word;">500</td>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,4 +97,11 @@ public class SqlGatewayServiceConfigOptions {
.withDescription(
"Keepalive time for an idle worker thread. When the number of workers exceeds min workers, "
+ "excessive threads are killed after this time interval.");

public static final ConfigOption<Boolean> SQL_GATEWAY_READ_ONLY_MODE =
key("sql-gateway.read-only")
.booleanType()
.defaultValue(false)
.withDescription(
"When enabled, the SQL Gateway operates in read-only mode and will reject all modify operations.");
}
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@
import org.apache.flink.table.factories.PlannerFactoryUtil;
import org.apache.flink.table.functions.FunctionDefinition;
import org.apache.flink.table.functions.FunctionIdentifier;
import org.apache.flink.table.gateway.api.config.SqlGatewayServiceConfigOptions;
import org.apache.flink.table.gateway.api.operation.OperationHandle;
import org.apache.flink.table.gateway.api.results.FunctionInfo;
import org.apache.flink.table.gateway.api.results.TableInfo;
Expand Down Expand Up @@ -681,6 +682,17 @@ private ResultFetcher callModifyOperations(
TableEnvironmentInternal tableEnv,
OperationHandle handle,
List<ModifyOperation> modifyOperations) {
// Check if SQL Gateway is in read-only mode
Configuration configuration = sessionContext.getSessionConf().clone();
configuration.addAll(executionConfig);
boolean isReadOnlyMode =
configuration.get(SqlGatewayServiceConfigOptions.SQL_GATEWAY_READ_ONLY_MODE);

if (isReadOnlyMode) {
throw new SqlExecutionException(
"SQL Gateway is in read-only mode. Modify operations are not allowed.");
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I feel it's useful to show the modifyOperations in the error message in case it's unclear to users which are actually violating the read-only mode.

}

TableResultInternal result = tableEnv.executeInternal(modifyOperations);
// DeleteFromFilterOperation doesn't have a JobClient
if (modifyOperations.size() == 1
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -110,6 +110,7 @@
import static org.apache.flink.table.functions.FunctionKind.AGGREGATE;
import static org.apache.flink.table.functions.FunctionKind.OTHER;
import static org.apache.flink.table.functions.FunctionKind.SCALAR;
import static org.apache.flink.table.gateway.api.config.SqlGatewayServiceConfigOptions.SQL_GATEWAY_READ_ONLY_MODE;
import static org.apache.flink.table.gateway.api.results.ResultSet.ResultType.PAYLOAD;
import static org.apache.flink.table.gateway.service.utils.SqlGatewayServiceTestUtil.awaitOperationTermination;
import static org.apache.flink.table.gateway.service.utils.SqlGatewayServiceTestUtil.createInitializedSession;
Expand Down Expand Up @@ -1048,6 +1049,37 @@ void testGetOperationSchemaWhenOperationGetError() throws Exception {
.satisfies(anyCauseMatches(SqlGatewayException.class, msg)));
}

@Test
void testReadOnlyModeWhenOperationInsertError() {
Configuration config = new Configuration(MINI_CLUSTER.getClientConfiguration());
config.set(SQL_GATEWAY_READ_ONLY_MODE, true);

String pipelineName = "test-job";
config.set(PipelineOptions.NAME, pipelineName);

SessionHandle sessionHandle = service.openSession(defaultSessionEnvironment);
String sourceDdl = "CREATE TABLE source (a STRING) WITH ('connector'='datagen');";
String sinkDdl = "CREATE TABLE sink (a STRING) WITH ('connector'='blackhole');";
String selectSql = "SELECT * FROM source;";

service.executeStatement(sessionHandle, sourceDdl, -1, config);
service.executeStatement(sessionHandle, sinkDdl, -1, config);
service.executeStatement(sessionHandle, selectSql, -1, config);

OperationHandle operationHandle =
service.executeStatement(
sessionHandle,
String.format("INSERT INTO sink '%s';", selectSql),
-1,
config);

assertThatThrownBy(() -> fetchAllResults(service, sessionHandle, operationHandle))
.satisfies(
anyCauseMatches(
SqlExecutionException.class,
"SQL Gateway is in read-only mode. Modify operations are not allowed."));
}

// --------------------------------------------------------------------------------------------

private OperationHandle submitDefaultOperation(
Expand Down