未验证 提交 9d3dab10 编写于 作者: L Liang Zhang 提交者: GitHub

Rename ProxyContext (#7335)

* Rename ProxySchemaContexts.getAllSchemaNames

* Rename ProxyContext
上级 572be6ae
......@@ -39,6 +39,7 @@ public final class SingleLocalDataMergedResult implements MergedResult {
this.values = values.iterator();
}
@Override
public boolean next() {
if (values.hasNext()) {
......
......@@ -25,7 +25,7 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.Bac
import org.apache.shardingsphere.proxy.backend.communication.jdbc.execute.engine.jdbc.JDBCExecuteEngine;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.wrapper.PreparedStatementExecutorWrapper;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.wrapper.StatementExecutorWrapper;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
import java.util.List;
......@@ -56,7 +56,7 @@ public final class DatabaseCommunicationEngineFactory {
* @return instance of text protocol backend handler
*/
public DatabaseCommunicationEngine newTextProtocolInstance(final SQLStatement sqlStatement, final String sql, final BackendConnection backendConnection) {
SchemaContext schema = ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema());
SchemaContext schema = ProxyContext.getInstance().getSchema(backendConnection.getSchema());
return new JDBCDatabaseCommunicationEngine(sql, backendConnection, new JDBCExecuteEngine(backendConnection, new StatementExecutorWrapper(schema, sqlStatement)));
}
......@@ -70,7 +70,7 @@ public final class DatabaseCommunicationEngineFactory {
* @return instance of text protocol backend handler
*/
public DatabaseCommunicationEngine newBinaryProtocolInstance(final SQLStatement sqlStatement, final String sql, final List<Object> parameters, final BackendConnection backendConnection) {
SchemaContext schema = ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema());
SchemaContext schema = ProxyContext.getInstance().getSchema(backendConnection.getSchema());
return new JDBCDatabaseCommunicationEngine(sql,
backendConnection, new JDBCExecuteEngine(backendConnection, new PreparedStatementExecutorWrapper(schema, sqlStatement, parameters)));
}
......
......@@ -43,7 +43,7 @@ import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.sharding.route.engine.exception.TableExistsException;
import org.apache.shardingsphere.sql.parser.binder.metadata.table.TableMetaData;
import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
......@@ -79,7 +79,7 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
this.sql = sql;
connection = backendConnection;
executeEngine = sqlExecuteEngine;
schema = ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema());
schema = ProxyContext.getInstance().getSchema(backendConnection.getSchema());
}
@Override
......@@ -95,8 +95,8 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
}
private void logSQL(final ExecutionContext executionContext) {
if (ProxySchemaContexts.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)) {
SQLLogger.logSQL(sql, ProxySchemaContexts.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SIMPLE), executionContext);
if (ProxyContext.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SHOW)) {
SQLLogger.logSQL(sql, ProxyContext.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.SQL_SIMPLE), executionContext);
}
}
......@@ -131,7 +131,7 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
}
Optional<MetaDataRefreshStrategy> refreshStrategy = MetaDataRefreshStrategyFactory.newInstance(sqlStatementContext);
if (refreshStrategy.isPresent()) {
refreshStrategy.get().refreshMetaData(schema.getSchema().getMetaData(), ProxySchemaContexts.getInstance().getSchemaContexts().getDatabaseType(),
refreshStrategy.get().refreshMetaData(schema.getSchema().getMetaData(), ProxyContext.getInstance().getSchemaContexts().getDatabaseType(),
schema.getSchema().getDataSources(), sqlStatementContext, this::loadTableMetaData);
ShardingSphereEventBus.getInstance().post(new MetaDataPersistEvent(schema.getName(), schema.getSchema().getMetaData().getSchema()));
}
......@@ -139,8 +139,8 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
private Optional<TableMetaData> loadTableMetaData(final String tableName) throws SQLException {
RuleSchemaMetaDataLoader loader = new RuleSchemaMetaDataLoader(schema.getSchema().getRules());
return loader.load(ProxySchemaContexts.getInstance().getSchemaContexts().getDatabaseType(),
schema.getSchema().getDataSources(), tableName, ProxySchemaContexts.getInstance().getSchemaContexts().getProps());
return loader.load(ProxyContext.getInstance().getSchemaContexts().getDatabaseType(),
schema.getSchema().getDataSources(), tableName, ProxyContext.getInstance().getSchemaContexts().getProps());
}
private BackendResponse merge(final SQLStatementContext<?> sqlStatementContext) throws SQLException {
......@@ -164,8 +164,8 @@ public final class JDBCDatabaseCommunicationEngine implements DatabaseCommunicat
}
private MergedResult mergeQuery(final SQLStatementContext<?> sqlStatementContext, final List<QueryResult> queryResults) throws SQLException {
MergeEngine mergeEngine = new MergeEngine(ProxySchemaContexts.getInstance().getSchemaContexts().getDatabaseType(),
schema.getSchema().getMetaData().getSchema().getConfiguredSchemaMetaData(), ProxySchemaContexts.getInstance().getSchemaContexts().getProps(), schema.getSchema().getRules());
MergeEngine mergeEngine = new MergeEngine(ProxyContext.getInstance().getSchemaContexts().getDatabaseType(),
schema.getSchema().getMetaData().getSchema().getConfiguredSchemaMetaData(), ProxyContext.getInstance().getSchemaContexts().getProps(), schema.getSchema().getRules());
return mergeEngine.merge(queryResults, sqlStatementContext);
}
......
......@@ -33,7 +33,7 @@ import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
import org.apache.shardingsphere.infra.executor.sql.resourced.jdbc.connection.JDBCExecutionConnection;
import org.apache.shardingsphere.infra.executor.sql.resourced.jdbc.group.StatementOption;
import org.apache.shardingsphere.masterslave.route.engine.impl.MasterVisitedManager;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.transaction.core.TransactionType;
import java.sql.Connection;
......@@ -188,7 +188,7 @@ public final class BackendConnection implements JDBCExecutionConnection, AutoClo
}
private List<Connection> getConnectionFromUnderlying(final String dataSourceName, final int connectionSize, final ConnectionMode connectionMode) throws SQLException {
return ProxySchemaContexts.getInstance().getBackendDataSource().getConnections(schema, dataSourceName, connectionSize, connectionMode);
return ProxyContext.getInstance().getBackendDataSource().getConnections(schema, dataSourceName, connectionSize, connectionMode);
}
@Override
......@@ -220,7 +220,7 @@ public final class BackendConnection implements JDBCExecutionConnection, AutoClo
}
private void setFetchSize(final Statement statement) throws SQLException {
DatabaseType databaseType = ProxySchemaContexts.getInstance().getSchemaContexts().getDatabaseType();
DatabaseType databaseType = ProxyContext.getInstance().getSchemaContexts().getDatabaseType();
if (databaseType instanceof MySQLDatabaseType) {
statement.setFetchSize(MYSQL_MEMORY_FETCH_ONE_ROW_A_TIME);
} else if (databaseType instanceof PostgreSQLDatabaseType) {
......
......@@ -17,7 +17,7 @@
package org.apache.shardingsphere.proxy.backend.communication.jdbc.connection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
import org.apache.shardingsphere.transaction.core.TransactionType;
import org.apache.shardingsphere.transaction.spi.ShardingTransactionManager;
......@@ -41,7 +41,7 @@ public final class BackendTransactionManager implements TransactionManager {
connection = backendConnection;
transactionType = connection.getTransactionType();
localTransactionManager = new LocalTransactionManager(backendConnection);
ShardingTransactionManagerEngine engine = ProxySchemaContexts.getInstance().getTransactionContexts().getEngines().get(connection.getSchema());
ShardingTransactionManagerEngine engine = ProxyContext.getInstance().getTransactionContexts().getEngines().get(connection.getSchema());
shardingTransactionManager = null == engine ? null : engine.getTransactionManager(transactionType);
}
......
......@@ -43,7 +43,7 @@ import org.apache.shardingsphere.proxy.backend.executor.BackendExecutorContext;
import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.DeleteStatement;
import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.UpdateStatement;
......@@ -100,14 +100,14 @@ public final class JDBCExecuteEngine implements SQLExecuteEngine {
}
private Collection<ExecuteResult> execute(final ExecutionContext executionContext, final boolean isReturnGeneratedKeys, final boolean isExceptionThrown) throws SQLException {
int maxConnectionsSizePerQuery = ProxySchemaContexts.getInstance().getSchemaContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
int maxConnectionsSizePerQuery = ProxyContext.getInstance().getSchemaContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.MAX_CONNECTIONS_SIZE_PER_QUERY);
return ExecutorConstant.MANAGED_RESOURCE ? executeWithManagedResource(executionContext, maxConnectionsSizePerQuery, isReturnGeneratedKeys, isExceptionThrown)
: executeWithUnmanagedResource(executionContext, maxConnectionsSizePerQuery);
}
private Collection<ExecuteResult> executeWithManagedResource(final ExecutionContext executionContext,
final int maxConnectionsSizePerQuery, final boolean isReturnGeneratedKeys, final boolean isExceptionThrown) throws SQLException {
DatabaseType databaseType = ProxySchemaContexts.getInstance().getSchemaContexts().getDatabaseType();
DatabaseType databaseType = ProxyContext.getInstance().getSchemaContexts().getDatabaseType();
return sqlExecutor.execute(generateInputGroups(executionContext.getExecutionUnits(), maxConnectionsSizePerQuery, isReturnGeneratedKeys),
new ProxySQLExecutorCallback(databaseType, executionContext.getSqlStatementContext(), backendConnection, jdbcExecutorWrapper, isExceptionThrown, isReturnGeneratedKeys, true),
new ProxySQLExecutorCallback(databaseType, executionContext.getSqlStatementContext(), backendConnection, jdbcExecutorWrapper, isExceptionThrown, isReturnGeneratedKeys, false));
......@@ -121,7 +121,7 @@ public final class JDBCExecuteEngine implements SQLExecuteEngine {
}
private Collection<ExecuteResult> executeWithUnmanagedResource(final ExecutionContext executionContext, final int maxConnectionsSizePerQuery) throws SQLException {
Collection<ShardingSphereRule> rules = ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema()).getSchema().getRules();
Collection<ShardingSphereRule> rules = ProxyContext.getInstance().getSchema(backendConnection.getSchema()).getSchema().getRules();
Collection<InputGroup<RawSQLExecuteUnit>> inputGroups = new RawExecuteGroupEngine(maxConnectionsSizePerQuery, rules).generate(executionContext.getExecutionUnits());
// TODO handle query header
return rawExecutor.execute(inputGroups, new RawSQLExecutorCallback());
......
......@@ -30,7 +30,7 @@ import org.apache.shardingsphere.infra.executor.sql.resourced.jdbc.queryresult.S
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.wrapper.JDBCExecutorWrapper;
import org.apache.shardingsphere.proxy.backend.response.query.QueryHeaderBuilder;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.sql.parser.binder.segment.select.projection.ProjectionsContext;
import org.apache.shardingsphere.sql.parser.binder.statement.SQLStatementContext;
import org.apache.shardingsphere.sql.parser.binder.statement.dml.SelectStatementContext;
......@@ -99,7 +99,7 @@ public final class ProxySQLExecutorCallback extends DefaultSQLExecutorCallback<E
private List<QueryHeader> getQueryHeaders(final ProjectionsContext projectionsContext, final ResultSetMetaData resultSetMetaData) throws SQLException {
List<QueryHeader> result = new LinkedList<>();
for (int columnIndex = 1; columnIndex <= projectionsContext.getExpandProjections().size(); columnIndex++) {
result.add(QueryHeaderBuilder.build(projectionsContext, resultSetMetaData, ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema()), columnIndex));
result.add(QueryHeaderBuilder.build(projectionsContext, resultSetMetaData, ProxyContext.getInstance().getSchema(backendConnection.getSchema()), columnIndex));
}
return result;
}
......@@ -107,7 +107,7 @@ public final class ProxySQLExecutorCallback extends DefaultSQLExecutorCallback<E
private List<QueryHeader> getQueryHeaders(final ResultSetMetaData resultSetMetaData) throws SQLException {
List<QueryHeader> result = new LinkedList<>();
for (int columnIndex = 1; columnIndex <= resultSetMetaData.getColumnCount(); columnIndex++) {
result.add(QueryHeaderBuilder.build(resultSetMetaData, ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema()), columnIndex));
result.add(QueryHeaderBuilder.build(resultSetMetaData, ProxyContext.getInstance().getSchema(backendConnection.getSchema()), columnIndex));
}
return result;
}
......
......@@ -32,7 +32,7 @@ import org.apache.shardingsphere.infra.route.DataNodeRouter;
import org.apache.shardingsphere.infra.route.context.RouteContext;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.sql.parser.binder.statement.CommonSQLStatementContext;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
......@@ -49,7 +49,7 @@ import java.util.List;
@RequiredArgsConstructor
public final class PreparedStatementExecutorWrapper implements JDBCExecutorWrapper {
private static final ProxySchemaContexts PROXY_SCHEMA_CONTEXTS = ProxySchemaContexts.getInstance();
private static final ProxyContext PROXY_SCHEMA_CONTEXTS = ProxyContext.getInstance();
private final SchemaContext schema;
......
......@@ -31,7 +31,7 @@ import org.apache.shardingsphere.infra.route.DataNodeRouter;
import org.apache.shardingsphere.infra.route.context.RouteContext;
import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.sql.parser.binder.statement.CommonSQLStatementContext;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
......@@ -46,7 +46,7 @@ import java.util.Collections;
@RequiredArgsConstructor
public final class StatementExecutorWrapper implements JDBCExecutorWrapper {
private static final ProxySchemaContexts PROXY_SCHEMA_CONTEXTS = ProxySchemaContexts.getInstance();
private static final ProxyContext PROXY_SCHEMA_CONTEXTS = ProxyContext.getInstance();
private final SchemaContext schema;
......
......@@ -22,7 +22,7 @@ import lombok.Getter;
import lombok.NoArgsConstructor;
import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
import org.apache.shardingsphere.infra.executor.kernel.ExecutorKernel;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
/**
* Backend executor context.
......@@ -34,7 +34,7 @@ public final class BackendExecutorContext {
private static final BackendExecutorContext INSTANCE = new BackendExecutorContext();
private final ExecutorKernel executorKernel =
new ExecutorKernel(ProxySchemaContexts.getInstance().getSchemaContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.EXECUTOR_SIZE));
new ExecutorKernel(ProxyContext.getInstance().getSchemaContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.EXECUTOR_SIZE));
/**
* Get executor context instance.
......
......@@ -27,19 +27,19 @@ import org.apache.shardingsphere.transaction.context.TransactionContexts;
import org.apache.shardingsphere.transaction.context.impl.StandardTransactionContexts;
import javax.sql.DataSource;
import java.util.LinkedList;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
/**
* Proxy schema contexts.
* Proxy context.
*/
@Getter
public final class ProxySchemaContexts {
public final class ProxyContext {
private static final ProxySchemaContexts INSTANCE = new ProxySchemaContexts();
private static final ProxyContext INSTANCE = new ProxyContext();
private final JDBCBackendDataSource backendDataSource;
......@@ -47,7 +47,7 @@ public final class ProxySchemaContexts {
private TransactionContexts transactionContexts;
private ProxySchemaContexts() {
private ProxyContext() {
backendDataSource = new JDBCBackendDataSource();
schemaContexts = new StandardSchemaContexts();
transactionContexts = new StandardTransactionContexts();
......@@ -58,7 +58,7 @@ public final class ProxySchemaContexts {
*
* @return instance of ShardingSphere schemas.
*/
public static ProxySchemaContexts getInstance() {
public static ProxyContext getInstance() {
return INSTANCE;
}
......@@ -76,30 +76,30 @@ public final class ProxySchemaContexts {
/**
* Check schema exists.
*
* @param schema schema
* @param schemaName schema name
* @return schema exists or not
*/
public boolean schemaExists(final String schema) {
return null != schemaContexts && schemaContexts.getSchemaContexts().containsKey(schema);
public boolean schemaExists(final String schemaName) {
return schemaContexts.getSchemaContexts().containsKey(schemaName);
}
/**
* Get ShardingSphere schema.
* Get schema context.
*
* @param schemaName schema name
* @return ShardingSphere schema
* @return schema context
*/
public SchemaContext getSchema(final String schemaName) {
return Strings.isNullOrEmpty(schemaName) ? null : schemaContexts.getSchemaContexts().get(schemaName);
}
/**
* Get schema names.
* Get all schema names.
*
* @return schema names
* @return all schema names
*/
public List<String> getSchemaNames() {
return new LinkedList<>(schemaContexts.getSchemaContexts().keySet());
public List<String> getAllSchemaNames() {
return new ArrayList<>(schemaContexts.getSchemaContexts().keySet());
}
/**
......@@ -108,7 +108,7 @@ public final class ProxySchemaContexts {
* @return data source sample
*/
public Optional<DataSource> getDataSourceSample() {
List<String> schemaNames = getSchemaNames();
List<String> schemaNames = getAllSchemaNames();
if (schemaNames.isEmpty()) {
return Optional.empty();
}
......
......@@ -20,7 +20,7 @@ package org.apache.shardingsphere.proxy.backend.schema.datasource.impl;
import com.google.common.base.Preconditions;
import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
import org.apache.shardingsphere.proxy.backend.schema.datasource.BackendDataSource;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.transaction.core.TransactionType;
import org.apache.shardingsphere.transaction.spi.ShardingTransactionManager;
......@@ -76,7 +76,7 @@ public final class JDBCBackendDataSource implements BackendDataSource {
@SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
public List<Connection> getConnections(final String schemaName, final String dataSourceName,
final int connectionSize, final ConnectionMode connectionMode, final TransactionType transactionType) throws SQLException {
DataSource dataSource = ProxySchemaContexts.getInstance().getSchemaContexts().getSchemaContexts().get(schemaName).getSchema().getDataSources().get(dataSourceName);
DataSource dataSource = ProxyContext.getInstance().getSchemaContexts().getSchemaContexts().get(schemaName).getSchema().getDataSources().get(dataSourceName);
Preconditions.checkNotNull(dataSource, "Can not get connection from datasource %s.", dataSourceName);
if (1 == connectionSize) {
return Collections.singletonList(createConnection(schemaName, dataSourceName, dataSource, transactionType));
......@@ -106,7 +106,7 @@ public final class JDBCBackendDataSource implements BackendDataSource {
}
private Connection createConnection(final String schemaName, final String dataSourceName, final DataSource dataSource, final TransactionType transactionType) throws SQLException {
ShardingTransactionManager shardingTransactionManager = ProxySchemaContexts.getInstance().getTransactionContexts().getEngines().get(schemaName).getTransactionManager(transactionType);
ShardingTransactionManager shardingTransactionManager = ProxyContext.getInstance().getTransactionContexts().getEngines().get(schemaName).getTransactionManager(transactionType);
return isInShardingTransaction(shardingTransactionManager) ? shardingTransactionManager.getConnection(dataSourceName) : dataSource.getConnection();
}
......
......@@ -24,7 +24,7 @@ import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
......@@ -49,7 +49,7 @@ public final class BroadcastBackendHandler implements TextProtocolBackendHandler
public BackendResponse execute() {
Collection<BackendResponse> responses = new LinkedList<>();
String originalSchema = backendConnection.getSchema();
for (String each : ProxySchemaContexts.getInstance().getSchemaNames()) {
for (String each : ProxyContext.getInstance().getAllSchemaNames()) {
backendConnection.setCurrentSchema(each);
responses.add(databaseCommunicationEngineFactory.newTextProtocolInstance(sqlStatement, sql, backendConnection).execute());
}
......
......@@ -33,7 +33,7 @@ import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
import org.apache.shardingsphere.proxy.config.util.DataSourceParameterConverter;
import org.apache.shardingsphere.proxy.config.yaml.YamlDataSourceParameter;
......@@ -76,7 +76,7 @@ public final class RDLBackendHandler implements TextProtocolBackendHandler {
}
private BackendResponse execute(final CreateDatabaseStatementContext context) {
if (ProxySchemaContexts.getInstance().getSchemaNames().contains(context.getSqlStatement().getDatabaseName())) {
if (ProxyContext.getInstance().getAllSchemaNames().contains(context.getSqlStatement().getDatabaseName())) {
return new ErrorResponse(new DBCreateExistsException(context.getSqlStatement().getDatabaseName()));
}
// TODO Need to get the executed feedback from registry center for returning.
......@@ -87,7 +87,7 @@ public final class RDLBackendHandler implements TextProtocolBackendHandler {
}
private BackendResponse execute(final DropDatabaseStatementContext context) {
if (!ProxySchemaContexts.getInstance().getSchemaNames().contains(context.getSqlStatement().getDatabaseName())) {
if (!ProxyContext.getInstance().getAllSchemaNames().contains(context.getSqlStatement().getDatabaseName())) {
return new ErrorResponse(new DBCreateExistsException(context.getSqlStatement().getDatabaseName()));
}
// TODO Need to get the executed feedback from registry center for returning.
......@@ -119,7 +119,7 @@ public final class RDLBackendHandler implements TextProtocolBackendHandler {
}
private SQLStatementContext<?> getSQLStatementContext() {
DatabaseType databaseType = ProxySchemaContexts.getInstance().getSchemaContexts().getDatabaseType();
DatabaseType databaseType = ProxyContext.getInstance().getSchemaContexts().getDatabaseType();
if (sqlStatement instanceof CreateDataSourcesStatement) {
return new CreateDataSourcesStatementContext((CreateDataSourcesStatement) sqlStatement, databaseType);
}
......@@ -146,7 +146,7 @@ public final class RDLBackendHandler implements TextProtocolBackendHandler {
}
private boolean isRegistryCenterExisted() {
return !(ProxySchemaContexts.getInstance().getSchemaContexts() instanceof StandardSchemaContexts);
return !(ProxyContext.getInstance().getSchemaContexts() instanceof StandardSchemaContexts);
}
@Override
......
......@@ -24,7 +24,7 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.Bac
import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
import org.apache.shardingsphere.sharding.merge.dal.common.SingleLocalDataMergedResult;
......@@ -44,16 +44,15 @@ public final class ShowDatabasesBackendHandler implements TextProtocolBackendHan
private MergedResult mergedResult;
@SuppressWarnings("unchecked")
@Override
public BackendResponse execute() {
mergedResult = new SingleLocalDataMergedResult(getSchemaNames());
return new QueryResponse(Collections.singletonList(new QueryHeader("information_schema", "SCHEMATA", "Database", "SCHEMA_NAME", 100, Types.VARCHAR, 0, false, false, false, false)));
}
private Collection getSchemaNames() {
Collection<String> result = new LinkedList<>(ProxySchemaContexts.getInstance().getSchemaNames());
Collection<String> authorizedSchemas = ProxySchemaContexts.getInstance().getSchemaContexts().getAuthentication().getUsers().get(backendConnection.getUsername()).getAuthorizedSchemas();
private Collection<Object> getSchemaNames() {
Collection<Object> result = new LinkedList<>(ProxyContext.getInstance().getAllSchemaNames());
Collection<String> authorizedSchemas = ProxyContext.getInstance().getSchemaContexts().getAuthentication().getUsers().get(backendConnection.getUsername()).getAuthorizedSchemas();
if (!authorizedSchemas.isEmpty()) {
result.retainAll(authorizedSchemas);
}
......
......@@ -28,7 +28,7 @@ import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
......@@ -54,7 +54,7 @@ public final class ShowTablesBackendHandler implements TextProtocolBackendHandle
@Override
public BackendResponse execute() {
SchemaContext context = ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema());
SchemaContext context = ProxyContext.getInstance().getSchema(backendConnection.getSchema());
if (null == context) {
return new ErrorResponse(new NoDatabaseSelectedException());
}
......
......@@ -26,7 +26,7 @@ import org.apache.shardingsphere.proxy.backend.exception.NoDatabaseSelectedExcep
import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
......@@ -52,7 +52,7 @@ public final class UnicastBackendHandler implements TextProtocolBackendHandler {
@Override
public BackendResponse execute() {
if (null == backendConnection.getSchema()) {
Map<String, SchemaContext> schemaContexts = ProxySchemaContexts.getInstance().getSchemaContexts().getSchemaContexts();
Map<String, SchemaContext> schemaContexts = ProxyContext.getInstance().getSchemaContexts().getSchemaContexts();
if (schemaContexts.isEmpty()) {
return new ErrorResponse(new NoDatabaseSelectedException());
}
......
......@@ -24,7 +24,7 @@ import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLUseStatement;
import org.apache.shardingsphere.sql.parser.sql.common.util.SQLUtil;
......@@ -44,7 +44,7 @@ public final class UseDatabaseBackendHandler implements TextProtocolBackendHandl
@Override
public BackendResponse execute() {
String schema = SQLUtil.getExactlyValue(useStatement.getSchema());
if (ProxySchemaContexts.getInstance().schemaExists(schema) && isAuthorizedSchema(schema)) {
if (ProxyContext.getInstance().schemaExists(schema) && isAuthorizedSchema(schema)) {
backendConnection.setCurrentSchema(schema);
return new UpdateResponse();
}
......@@ -53,7 +53,7 @@ public final class UseDatabaseBackendHandler implements TextProtocolBackendHandl
}
private boolean isAuthorizedSchema(final String schema) {
Collection<String> authorizedSchemas = ProxySchemaContexts.getInstance().getSchemaContexts().getAuthentication().getUsers().get(backendConnection.getUsername()).getAuthorizedSchemas();
Collection<String> authorizedSchemas = ProxyContext.getInstance().getSchemaContexts().getAuthentication().getUsers().get(backendConnection.getUsername()).getAuthorizedSchemas();
return authorizedSchemas.isEmpty() || authorizedSchemas.contains(schema);
}
......
......@@ -28,7 +28,7 @@ import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
......@@ -54,7 +54,7 @@ public final class QueryBackendHandler implements TextProtocolBackendHandler {
@Override
public BackendResponse execute() {
SchemaContext context = ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema());
SchemaContext context = ProxyContext.getInstance().getSchema(backendConnection.getSchema());
if (null == context) {
return new ErrorResponse(new NoDatabaseSelectedException());
}
......
......@@ -27,7 +27,7 @@ import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
import org.apache.shardingsphere.proxy.backend.text.sctl.exception.InvalidShardingCTLFormatException;
......@@ -57,7 +57,7 @@ public final class ShardingCTLExplainBackendHandler implements TextProtocolBacke
if (!explainStatement.isPresent()) {
return new ErrorResponse(new InvalidShardingCTLFormatException(sql));
}
SchemaContext schema = ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema());
SchemaContext schema = ProxyContext.getInstance().getSchema(backendConnection.getSchema());
StatementExecutorWrapper statementExecutorWrapper =
new StatementExecutorWrapper(schema, schema.getRuntimeContext().getSqlParserEngine().parse(explainStatement.get().getSql(), false));
executionUnits = statementExecutorWrapper.generateExecutionContext(explainStatement.get().getSql()).getExecutionUnits().iterator();
......
......@@ -23,7 +23,7 @@ import org.apache.shardingsphere.infra.executor.sql.raw.execute.result.query.Que
import org.apache.shardingsphere.infra.hint.HintManager;
import org.apache.shardingsphere.infra.merge.result.MergedResult;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.sctl.hint.internal.command.HintShowTableStatusCommand;
import org.apache.shardingsphere.proxy.backend.text.sctl.hint.internal.result.HintShowTableStatusResult;
import org.apache.shardingsphere.sharding.merge.dal.common.MultipleLocalDataMergedResult;
......@@ -57,7 +57,7 @@ public final class HintShowTableStatusExecutor extends AbstractHintQueryExecutor
protected MergedResult createMergedResult() {
Map<String, HintShowTableStatusResult> results = new HashMap<>();
Collection<String> tableNames =
ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema()).getSchema().getMetaData().getSchema().getConfiguredSchemaMetaData().getAllTableNames();
ProxyContext.getInstance().getSchema(backendConnection.getSchema()).getSchema().getMetaData().getSchema().getConfiguredSchemaMetaData().getAllTableNames();
for (String each : tableNames) {
if (HintManager.isDatabaseShardingOnly()) {
fillShardingValues(results, each, HintManager.getDatabaseShardingValues(), Collections.emptyList());
......
......@@ -22,7 +22,7 @@ import org.apache.shardingsphere.infra.database.type.DatabaseType;
import org.apache.shardingsphere.infra.database.type.DatabaseTypes;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.ConnectionStateHandler;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandlerFactory;
import org.apache.shardingsphere.proxy.backend.text.admin.BroadcastBackendHandler;
......@@ -69,9 +69,9 @@ public final class TextProtocolBackendHandlerFactoryTest {
@SneakyThrows(ReflectiveOperationException.class)
private void setTransactionContexts() {
Field transactionContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("transactionContexts");
Field transactionContexts = ProxyContext.getInstance().getClass().getDeclaredField("transactionContexts");
transactionContexts.setAccessible(true);
transactionContexts.set(ProxySchemaContexts.getInstance(), createSchemaContext());
transactionContexts.set(ProxyContext.getInstance(), createSchemaContext());
}
private TransactionContexts createSchemaContext() {
......
......@@ -26,7 +26,7 @@ import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngine;
import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngineFactory;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
import org.junit.Before;
import org.junit.Test;
......@@ -50,9 +50,9 @@ public final class DatabaseCommunicationEngineFactoryTest {
@Before
@SneakyThrows(ReflectiveOperationException.class)
public void setUp() {
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
BackendConnection backendConnection = mock(BackendConnection.class);
when(backendConnection.getSchema()).thenReturn("schema");
......
......@@ -27,7 +27,7 @@ import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.infra.exception.ShardingSphereException;
import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.schema.datasource.impl.JDBCBackendDataSource;
import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
import org.apache.shardingsphere.transaction.context.TransactionContexts;
......@@ -86,9 +86,9 @@ public final class BackendConnectionTest {
@SneakyThrows(ReflectiveOperationException.class)
private void setSchemaContexts() {
Field field = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field field = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
field.setAccessible(true);
field.set(ProxySchemaContexts.getInstance(),
field.set(ProxyContext.getInstance(),
new StandardSchemaContexts(createSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
}
......@@ -105,9 +105,9 @@ public final class BackendConnectionTest {
@SneakyThrows(ReflectiveOperationException.class)
private void setTransactionContexts() {
Field field = ProxySchemaContexts.getInstance().getClass().getDeclaredField("transactionContexts");
Field field = ProxyContext.getInstance().getClass().getDeclaredField("transactionContexts");
field.setAccessible(true);
field.set(ProxySchemaContexts.getInstance(), createTransactionContexts());
field.set(ProxyContext.getInstance(), createTransactionContexts());
}
private TransactionContexts createTransactionContexts() {
......@@ -121,9 +121,9 @@ public final class BackendConnectionTest {
@SneakyThrows(ReflectiveOperationException.class)
private void setBackendDataSource() {
Field field = ProxySchemaContexts.getInstance().getClass().getDeclaredField("backendDataSource");
Field field = ProxyContext.getInstance().getClass().getDeclaredField("backendDataSource");
field.setAccessible(true);
field.set(ProxySchemaContexts.getInstance(), backendDataSource);
field.set(ProxyContext.getInstance(), backendDataSource);
}
@Test
......@@ -292,10 +292,10 @@ public final class BackendConnectionTest {
@SneakyThrows(ReflectiveOperationException.class)
@After
public void clean() {
Field field = ProxySchemaContexts.getInstance().getClass().getDeclaredField("backendDataSource");
Field field = ProxyContext.getInstance().getClass().getDeclaredField("backendDataSource");
field.setAccessible(true);
Class<?> clazz = field.getType();
Object datasource = clazz.getDeclaredConstructors()[0].newInstance();
field.set(ProxySchemaContexts.getInstance(), datasource);
field.set(ProxyContext.getInstance(), datasource);
}
}
......@@ -18,7 +18,7 @@
package org.apache.shardingsphere.proxy.backend.communication.jdbc.connection;
import lombok.SneakyThrows;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
import org.apache.shardingsphere.transaction.context.TransactionContexts;
import org.apache.shardingsphere.transaction.core.TransactionType;
......@@ -64,9 +64,9 @@ public final class BackendTransactionManagerTest {
@SneakyThrows(ReflectiveOperationException.class)
private void setTransactionContexts() {
Field transactionContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("transactionContexts");
Field transactionContexts = ProxyContext.getInstance().getClass().getDeclaredField("transactionContexts");
transactionContexts.setAccessible(true);
transactionContexts.set(ProxySchemaContexts.getInstance(), getTransactionContexts());
transactionContexts.set(ProxyContext.getInstance(), getTransactionContexts());
}
private TransactionContexts getTransactionContexts() {
......
......@@ -28,7 +28,7 @@ import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.infra.executor.sql.ConnectionMode;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.datasource.fixture.CallTimeRecordDataSource;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
import org.apache.shardingsphere.transaction.context.TransactionContexts;
import org.junit.Before;
......@@ -70,9 +70,9 @@ public final class JDBCBackendDataSourceTest {
@SneakyThrows(ReflectiveOperationException.class)
private void setSchemaContexts() {
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(createSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
}
......@@ -89,9 +89,9 @@ public final class JDBCBackendDataSourceTest {
@SneakyThrows(ReflectiveOperationException.class)
private void setTransactionContexts() {
Field transactionContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("transactionContexts");
Field transactionContexts = ProxyContext.getInstance().getClass().getDeclaredField("transactionContexts");
transactionContexts.setAccessible(true);
transactionContexts.set(ProxySchemaContexts.getInstance(), createTransactionContexts());
transactionContexts.set(ProxyContext.getInstance(), createTransactionContexts());
}
private TransactionContexts createTransactionContexts() {
......@@ -111,19 +111,19 @@ public final class JDBCBackendDataSourceTest {
@Test
public void assertGetConnectionFixedOne() throws SQLException {
Connection actual = ProxySchemaContexts.getInstance().getBackendDataSource().getConnection("schema", "ds_1");
Connection actual = ProxyContext.getInstance().getBackendDataSource().getConnection("schema", "ds_1");
assertThat(actual, instanceOf(Connection.class));
}
@Test
public void assertGetConnectionsSucceed() throws SQLException {
List<Connection> actual = ProxySchemaContexts.getInstance().getBackendDataSource().getConnections("schema", "ds_1", 5, ConnectionMode.MEMORY_STRICTLY);
List<Connection> actual = ProxyContext.getInstance().getBackendDataSource().getConnections("schema", "ds_1", 5, ConnectionMode.MEMORY_STRICTLY);
assertThat(actual.size(), is(5));
}
@Test(expected = SQLException.class)
public void assertGetConnectionsFailed() throws SQLException {
ProxySchemaContexts.getInstance().getBackendDataSource().getConnections("schema", "ds_1", 6, ConnectionMode.MEMORY_STRICTLY);
ProxyContext.getInstance().getBackendDataSource().getConnections("schema", "ds_1", 6, ConnectionMode.MEMORY_STRICTLY);
}
@Test
......@@ -156,7 +156,7 @@ public final class JDBCBackendDataSourceTest {
@Override
public List<Connection> call() throws SQLException {
return ProxySchemaContexts.getInstance().getBackendDataSource().getConnections("schema", datasourceName, connectionSize, connectionMode);
return ProxyContext.getInstance().getBackendDataSource().getConnections("schema", datasourceName, connectionSize, connectionMode);
}
}
}
......@@ -40,18 +40,18 @@ import static org.junit.Assert.assertThat;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public final class ProxySchemaContextsTest {
public final class ProxyContextTest {
@Test
public void assertGetDataSourceSample() throws NoSuchFieldException, IllegalAccessException {
Map<String, DataSource> mockDataSourceMap = new HashMap<>(2, 1);
mockDataSourceMap.put("ds_1", new MockedDataSource());
mockDataSourceMap.put("ds_2", new MockedDataSource());
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(mockDataSourceMap), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
Optional<DataSource> actual = ProxySchemaContexts.getInstance().getDataSourceSample();
Optional<DataSource> actual = ProxyContext.getInstance().getDataSourceSample();
assertThat(actual, is(Optional.of(mockDataSourceMap.get("ds_1"))));
}
......
......@@ -29,7 +29,7 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.Bac
import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
import org.junit.Before;
import org.junit.Test;
......@@ -68,9 +68,9 @@ public final class BroadcastBackendHandlerTest {
@Before
@SneakyThrows(ReflectiveOperationException.class)
public void setUp() {
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
when(backendConnection.getSchema()).thenReturn("schema_0");
}
......
......@@ -29,7 +29,7 @@ import org.apache.shardingsphere.proxy.backend.exception.DBCreateExistsException
import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.rdl.parser.statement.rdl.CreateDataSourcesStatement;
import org.apache.shardingsphere.rdl.parser.statement.rdl.CreateShardingRuleStatement;
import org.apache.shardingsphere.sql.parser.sql.common.statement.ddl.CreateDatabaseStatement;
......@@ -53,9 +53,9 @@ public final class RDLBackendHandlerTest {
@Before
@SneakyThrows(ReflectiveOperationException.class)
public void setUp() {
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
}
......@@ -127,14 +127,14 @@ public final class RDLBackendHandlerTest {
@SneakyThrows(ReflectiveOperationException.class)
private void setGovernanceSchemaContexts(final boolean isGovernance) {
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
if (isGovernance) {
SchemaContexts mockedSchemaContexts = mock(SchemaContexts.class);
when(mockedSchemaContexts.getSchemaContexts()).thenReturn(Collections.singletonMap("schema", mock(SchemaContext.class)));
schemaContexts.set(ProxySchemaContexts.getInstance(), mockedSchemaContexts);
schemaContexts.set(ProxyContext.getInstance(), mockedSchemaContexts);
} else {
schemaContexts.set(ProxySchemaContexts.getInstance(), new StandardSchemaContexts());
schemaContexts.set(ProxyContext.getInstance(), new StandardSchemaContexts());
}
}
......
......@@ -27,7 +27,7 @@ import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.junit.Before;
import org.junit.Test;
import org.junit.runner.RunWith;
......@@ -58,9 +58,9 @@ public final class ShowDatabasesBackendHandlerTest {
BackendConnection backendConnection = mock(BackendConnection.class);
when(backendConnection.getUsername()).thenReturn("root");
showDatabasesBackendHandler = new ShowDatabasesBackendHandler(backendConnection);
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(), getAuthentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
}
......
......@@ -27,7 +27,7 @@ import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
import org.junit.Before;
import org.junit.Test;
......@@ -58,9 +58,9 @@ public class ShowTablesBackendHandlerTest {
tablesBackendHandler = new ShowTablesBackendHandler("show tables", mock(SQLStatement.class), backendConnection);
Map<String, SchemaContext> schemaContextMap = getSchemaContextMap();
when(backendConnection.getSchema()).thenReturn("schema_0");
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(schemaContextMap, getAuthentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
}
......
......@@ -28,7 +28,7 @@ import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicati
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
import org.apache.shardingsphere.transaction.core.TransactionType;
import org.junit.Before;
......@@ -62,9 +62,9 @@ public final class UnicastBackendHandlerTest {
@Before
@SneakyThrows(ReflectiveOperationException.class)
public void setUp() {
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
setUnderlyingHandler(new UpdateResponse());
}
......
......@@ -28,7 +28,7 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.Bac
import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dal.MySQLUseStatement;
import org.junit.Before;
import org.junit.Test;
......@@ -59,9 +59,9 @@ public final class UseDatabaseBackendHandlerTest {
public void setUp() {
backendConnection = mock(BackendConnection.class);
when(backendConnection.getUsername()).thenReturn("root");
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(), getAuthentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
}
......
......@@ -9,7 +9,7 @@ import org.apache.shardingsphere.infra.context.runtime.RuntimeContext;
import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
import org.apache.shardingsphere.sql.parser.engine.StandardSQLParserEngine;
import org.junit.Before;
......@@ -54,9 +54,9 @@ public final class ShardingCTLExplainBackendHandlerTest {
BackendConnection connection = mock(BackendConnection.class);
when(connection.getSchema()).thenReturn("schema");
handler = new ShardingCTLExplainBackendHandler("sctl:explain select 1", connection);
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
}
......
......@@ -35,7 +35,7 @@ import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.sctl.exception.InvalidShardingCTLFormatException;
import org.apache.shardingsphere.proxy.backend.text.sctl.exception.UnsupportedShardingCTLTypeException;
import org.apache.shardingsphere.proxy.backend.text.sctl.hint.internal.HintManagerHolder;
......@@ -172,9 +172,9 @@ public final class ShardingCTLHintBackendHandlerTest {
public void assertShowTableStatus() throws SQLException {
clearThreadLocal();
when(backendConnection.getSchema()).thenReturn("schema");
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
String sql = "sctl:hint show table status";
ShardingCTLHintBackendHandler defaultShardingCTLHintBackendHandler = new ShardingCTLHintBackendHandler(sql, backendConnection);
......
......@@ -27,7 +27,7 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.Bac
import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.transaction.core.TransactionType;
import org.junit.Before;
import org.junit.Test;
......@@ -49,9 +49,9 @@ public final class ShardingCTLSetBackendHandlerTest {
@Before
@SneakyThrows(ReflectiveOperationException.class)
public void setUp() {
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
}
......
......@@ -24,7 +24,7 @@ import org.apache.shardingsphere.infra.context.SchemaContext;
import org.apache.shardingsphere.infra.context.SchemaContexts;
import org.apache.shardingsphere.infra.context.SchemaContextsBuilder;
import org.apache.shardingsphere.proxy.backend.schema.ProxyDataSourceContext;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.config.ProxyConfiguration;
import org.apache.shardingsphere.proxy.config.YamlProxyConfiguration;
import org.apache.shardingsphere.proxy.db.DatabaseServerInfo;
......@@ -53,7 +53,7 @@ public abstract class AbstractBootstrapInitializer implements BootstrapInitializ
ProxyConfiguration proxyConfig = getProxyConfiguration(yamlConfig);
SchemaContexts schemaContexts = decorateSchemaContexts(createSchemaContexts(proxyConfig));
TransactionContexts transactionContexts = decorateTransactionContexts(createTransactionContexts(schemaContexts));
ProxySchemaContexts.getInstance().init(schemaContexts, transactionContexts);
ProxyContext.getInstance().init(schemaContexts, transactionContexts);
initOpenTracing();
setDatabaseServerInfo();
new ShardingSphereProxy().start(port);
......@@ -77,13 +77,13 @@ public abstract class AbstractBootstrapInitializer implements BootstrapInitializ
}
private void initOpenTracing() {
if (ProxySchemaContexts.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_OPENTRACING_ENABLED)) {
if (ProxyContext.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_OPENTRACING_ENABLED)) {
OpenTracingTracer.init();
}
}
private void setDatabaseServerInfo() {
Optional<DataSource> dataSourceSample = ProxySchemaContexts.getInstance().getDataSourceSample();
Optional<DataSource> dataSourceSample = ProxyContext.getInstance().getDataSourceSample();
if (dataSourceSample.isPresent()) {
DatabaseServerInfo databaseServerInfo = new DatabaseServerInfo(dataSourceSample.get());
log.info(databaseServerInfo.toString());
......
......@@ -21,14 +21,14 @@ import com.google.common.util.concurrent.ListeningExecutorService;
import lombok.Getter;
import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
import org.apache.shardingsphere.infra.executor.kernel.impl.ShardingSphereExecutorService;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
/**
* User executor group.
*/
public final class UserExecutorGroup {
private static final ProxySchemaContexts PROXY_SCHEMA_CONTEXTS = ProxySchemaContexts.getInstance();
private static final ProxyContext PROXY_SCHEMA_CONTEXTS = ProxyContext.getInstance();
private static final String NAME_FORMAT = "Command-%d";
......
......@@ -24,7 +24,7 @@ import lombok.extern.slf4j.Slf4j;
import org.apache.shardingsphere.db.protocol.payload.PacketPayload;
import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.command.CommandExecutorTask;
import org.apache.shardingsphere.proxy.frontend.auth.AuthenticationResult;
import org.apache.shardingsphere.proxy.frontend.executor.ChannelThreadExecutorGroup;
......@@ -49,8 +49,8 @@ public final class FrontendChannelInboundHandler extends ChannelInboundHandlerAd
public FrontendChannelInboundHandler(final DatabaseProtocolFrontendEngine databaseProtocolFrontendEngine) {
this.databaseProtocolFrontendEngine = databaseProtocolFrontendEngine;
TransactionType transactionType = TransactionType.valueOf(ProxySchemaContexts.getInstance().getSchemaContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_TRANSACTION_TYPE));
boolean supportHint = ProxySchemaContexts.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED);
TransactionType transactionType = TransactionType.valueOf(ProxyContext.getInstance().getSchemaContexts().getProps().getValue(ConfigurationPropertyKey.PROXY_TRANSACTION_TYPE));
boolean supportHint = ProxyContext.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_HINT_ENABLED);
backendConnection = new BackendConnection(transactionType, supportHint);
}
......
......@@ -24,7 +24,7 @@ import lombok.RequiredArgsConstructor;
import org.apache.shardingsphere.db.protocol.codec.PacketCodec;
import org.apache.shardingsphere.infra.database.type.DatabaseType;
import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.protocol.DatabaseProtocolFrontendEngineFactory;
import org.apache.shardingsphere.proxy.frontend.spi.DatabaseProtocolFrontendEngine;
......@@ -37,8 +37,8 @@ public final class ServerHandlerInitializer extends ChannelInitializer<SocketCha
@Override
protected void initChannel(final SocketChannel socketChannel) {
// TODO Consider loading from configuration.
DatabaseType databaseType = ProxySchemaContexts.getInstance().getSchemaContexts().getSchemaContexts().isEmpty() ? new MySQLDatabaseType()
: ProxySchemaContexts.getInstance().getSchemaContexts().getDatabaseType();
DatabaseType databaseType = ProxyContext.getInstance().getSchemaContexts().getSchemaContexts().isEmpty() ? new MySQLDatabaseType()
: ProxyContext.getInstance().getSchemaContexts().getDatabaseType();
DatabaseProtocolFrontendEngine databaseProtocolFrontendEngine = DatabaseProtocolFrontendEngineFactory.newInstance(databaseType);
ChannelPipeline pipeline = socketChannel.pipeline();
pipeline.addLast(new PacketCodec(databaseProtocolFrontendEngine.getCodecEngine()));
......
......@@ -31,7 +31,7 @@ import org.apache.shardingsphere.db.protocol.mysql.packet.handshake.MySQLHandsha
import org.apache.shardingsphere.db.protocol.mysql.packet.handshake.MySQLHandshakeResponse41Packet;
import org.apache.shardingsphere.db.protocol.mysql.payload.MySQLPacketPayload;
import org.apache.shardingsphere.db.protocol.payload.PacketPayload;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.connection.ConnectionIdGenerator;
import org.apache.shardingsphere.proxy.frontend.auth.AuthenticationResultBuilder;
import org.apache.shardingsphere.proxy.frontend.auth.AuthenticationEngine;
......@@ -83,7 +83,7 @@ public final class MySQLAuthenticationEngine implements AuthenticationEngine {
MySQLHandshakeResponse41Packet packet = new MySQLHandshakeResponse41Packet((MySQLPacketPayload) payload);
authResponse = packet.getAuthResponse();
sequenceId = packet.getSequenceId();
if (!Strings.isNullOrEmpty(packet.getDatabase()) && !ProxySchemaContexts.getInstance().schemaExists(packet.getDatabase())) {
if (!Strings.isNullOrEmpty(packet.getDatabase()) && !ProxyContext.getInstance().schemaExists(packet.getDatabase())) {
context.writeAndFlush(new MySQLErrPacket(++sequenceId, MySQLServerErrorCode.ER_BAD_DB_ERROR, packet.getDatabase()));
return AuthenticationResultBuilder.continued();
}
......
......@@ -24,7 +24,7 @@ import org.apache.commons.collections4.CollectionUtils;
import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLServerErrorCode;
import org.apache.shardingsphere.db.protocol.mysql.packet.handshake.MySQLAuthPluginData;
import org.apache.shardingsphere.infra.auth.ProxyUser;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import java.util.Arrays;
import java.util.Collection;
......@@ -37,7 +37,7 @@ import java.util.Optional;
@Getter
public final class MySQLAuthenticationHandler {
private static final ProxySchemaContexts PROXY_SCHEMA_CONTEXTS = ProxySchemaContexts.getInstance();
private static final ProxyContext PROXY_SCHEMA_CONTEXTS = ProxyContext.getInstance();
private final MySQLAuthPluginData authPluginData = new MySQLAuthPluginData();
......
......@@ -31,7 +31,7 @@ import org.apache.shardingsphere.db.protocol.packet.DatabasePacket;
import org.apache.shardingsphere.db.protocol.payload.PacketPayload;
import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.command.executor.CommandExecutor;
import org.apache.shardingsphere.proxy.frontend.command.executor.QueryCommandExecutor;
import org.apache.shardingsphere.proxy.frontend.command.CommandExecuteEngine;
......@@ -78,7 +78,7 @@ public final class MySQLCommandExecuteEngine implements CommandExecuteEngine {
return;
}
int count = 0;
int flushThreshold = ProxySchemaContexts.getInstance().getSchemaContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD);
int flushThreshold = ProxyContext.getInstance().getSchemaContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD);
int currentSequenceId = 0;
while (queryCommandExecutor.next()) {
count++;
......
......@@ -24,7 +24,7 @@ import org.apache.shardingsphere.db.protocol.mysql.packet.generic.MySQLErrPacket
import org.apache.shardingsphere.db.protocol.mysql.packet.generic.MySQLOKPacket;
import org.apache.shardingsphere.db.protocol.packet.DatabasePacket;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.command.executor.CommandExecutor;
import org.apache.shardingsphere.sql.parser.sql.common.util.SQLUtil;
......@@ -44,7 +44,7 @@ public final class MySQLComInitDbExecutor implements CommandExecutor {
@Override
public Collection<DatabasePacket<?>> execute() {
String schema = SQLUtil.getExactlyValue(packet.getSchema());
if (ProxySchemaContexts.getInstance().schemaExists(schema) && isAuthorizedSchema(schema)) {
if (ProxyContext.getInstance().schemaExists(schema) && isAuthorizedSchema(schema)) {
backendConnection.setCurrentSchema(packet.getSchema());
return Collections.singletonList(new MySQLOKPacket(1));
}
......@@ -52,7 +52,7 @@ public final class MySQLComInitDbExecutor implements CommandExecutor {
}
private boolean isAuthorizedSchema(final String schema) {
Collection<String> authorizedSchemas = ProxySchemaContexts.getInstance().getSchemaContexts().getAuthentication().getUsers().get(backendConnection.getUsername()).getAuthorizedSchemas();
Collection<String> authorizedSchemas = ProxyContext.getInstance().getSchemaContexts().getAuthentication().getUsers().get(backendConnection.getUsername()).getAuthorizedSchemas();
return authorizedSchemas.isEmpty() || authorizedSchemas.contains(schema);
}
}
......@@ -38,7 +38,7 @@ import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.command.executor.QueryCommandExecutor;
import org.apache.shardingsphere.proxy.frontend.mysql.MySQLErrPacketFactory;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
......@@ -70,14 +70,14 @@ public final class MySQLComStmtExecuteExecutor implements QueryCommandExecutor {
public MySQLComStmtExecuteExecutor(final MySQLComStmtExecutePacket comStmtExecutePacket, final BackendConnection backendConnection) {
SQLStatement sqlStatement =
ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema()).getRuntimeContext().getSqlParserEngine().parse(comStmtExecutePacket.getSql(), true);
ProxyContext.getInstance().getSchema(backendConnection.getSchema()).getRuntimeContext().getSqlParserEngine().parse(comStmtExecutePacket.getSql(), true);
databaseCommunicationEngine = DatabaseCommunicationEngineFactory.getInstance().newBinaryProtocolInstance(sqlStatement,
comStmtExecutePacket.getSql(), comStmtExecutePacket.getParameters(), backendConnection);
}
@Override
public Collection<DatabasePacket<?>> execute() {
if (ProxySchemaContexts.getInstance().getSchemaContexts().isCircuitBreak()) {
if (ProxyContext.getInstance().getSchemaContexts().isCircuitBreak()) {
return Collections.singletonList(new MySQLErrPacket(1, CommonErrorCode.CIRCUIT_BREAK_MODE));
}
BackendResponse backendResponse = getBackendResponse();
......
......@@ -28,7 +28,7 @@ import org.apache.shardingsphere.db.protocol.mysql.packet.generic.MySQLErrPacket
import org.apache.shardingsphere.db.protocol.packet.DatabasePacket;
import org.apache.shardingsphere.infra.context.SchemaContext;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.command.executor.CommandExecutor;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
......@@ -49,7 +49,7 @@ public final class MySQLComStmtPrepareExecutor implements CommandExecutor {
public MySQLComStmtPrepareExecutor(final MySQLComStmtPreparePacket packet, final BackendConnection backendConnection) {
this.packet = packet;
schema = ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema());
schema = ProxyContext.getInstance().getSchema(backendConnection.getSchema());
}
private int getColumnsCount(final SQLStatement sqlStatement) {
......
......@@ -27,7 +27,7 @@ import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicati
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.command.executor.CommandExecutor;
import org.apache.shardingsphere.proxy.frontend.mysql.MySQLErrPacketFactory;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
......@@ -55,7 +55,7 @@ public final class MySQLComFieldListPacketExecutor implements CommandExecutor {
schemaName = backendConnection.getSchema();
String sql = getShowColumnsSQL();
SQLStatement sqlStatement =
ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema()).getRuntimeContext().getSqlParserEngine().parse(sql, false);
ProxyContext.getInstance().getSchema(backendConnection.getSchema()).getRuntimeContext().getSqlParserEngine().parse(sql, false);
databaseCommunicationEngine = DatabaseCommunicationEngineFactory.getInstance().newTextProtocolInstance(sqlStatement, sql, backendConnection);
}
......
......@@ -37,7 +37,7 @@ import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandlerFactory;
import org.apache.shardingsphere.proxy.frontend.command.executor.QueryCommandExecutor;
......@@ -73,7 +73,7 @@ public final class MySQLComQueryPacketExecutor implements QueryCommandExecutor {
@Override
public Collection<DatabasePacket<?>> execute() {
if (ProxySchemaContexts.getInstance().getSchemaContexts().isCircuitBreak()) {
if (ProxyContext.getInstance().getSchemaContexts().isCircuitBreak()) {
return Collections.singletonList(new MySQLErrPacket(1, CommonErrorCode.CIRCUIT_BREAK_MODE));
}
BackendResponse backendResponse = getBackendResponse();
......
......@@ -34,7 +34,7 @@ import org.apache.shardingsphere.infra.context.impl.StandardSchemaContexts;
import org.apache.shardingsphere.infra.context.runtime.RuntimeContext;
import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.connection.ConnectionIdGenerator;
import org.apache.shardingsphere.proxy.frontend.auth.AuthenticationResult;
import org.apache.shardingsphere.proxy.frontend.mysql.auth.MySQLAuthenticationEngine;
......@@ -147,7 +147,7 @@ public final class MySQLFrontendEngineTest {
private void setAuthentication(final ProxyUser proxyUser) {
Authentication authentication = new Authentication();
authentication.getUsers().put("root", proxyUser);
initProxySchemaContexts(authentication);
initProxyContext(authentication);
}
@SneakyThrows(ReflectiveOperationException.class)
......@@ -158,10 +158,10 @@ public final class MySQLFrontendEngineTest {
}
@SneakyThrows(ReflectiveOperationException.class)
private void initProxySchemaContexts(final Authentication authentication) {
Field field = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
private void initProxyContext(final Authentication authentication) {
Field field = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
field.setAccessible(true);
field.set(ProxySchemaContexts.getInstance(), getSchemaContexts(authentication));
field.set(ProxyContext.getInstance(), getSchemaContexts(authentication));
}
private SchemaContexts getSchemaContexts(final Authentication authentication) {
......
......@@ -32,7 +32,7 @@ import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties
import org.apache.shardingsphere.infra.context.SchemaContext;
import org.apache.shardingsphere.infra.context.impl.StandardSchemaContexts;
import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.auth.AuthenticationResultBuilder;
import org.junit.Before;
import org.junit.Test;
......@@ -137,9 +137,9 @@ public final class MySQLAuthenticationEngineTest {
}
private void setSchemas() throws NoSuchFieldException, IllegalAccessException {
Field field = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field field = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
field.setAccessible(true);
field.set(ProxySchemaContexts.getInstance(),
field.set(ProxyContext.getInstance(),
new StandardSchemaContexts(Collections.singletonMap("sharding_db", mock(SchemaContext.class)),
new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
}
......
......@@ -30,7 +30,7 @@ import org.apache.shardingsphere.infra.context.impl.StandardSchemaContexts;
import org.apache.shardingsphere.infra.context.runtime.RuntimeContext;
import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.junit.Before;
import org.junit.Test;
......@@ -110,14 +110,14 @@ public final class MySQLAuthenticationHandlerTest {
private void setAuthentication(final ProxyUser proxyUser) {
Authentication authentication = new Authentication();
authentication.getUsers().put("root", proxyUser);
initProxySchemaContexts(authentication);
initProxyContext(authentication);
}
@SneakyThrows
private void initProxySchemaContexts(final Authentication authentication) {
Field field = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
private void initProxyContext(final Authentication authentication) {
Field field = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
field.setAccessible(true);
field.set(ProxySchemaContexts.getInstance(), getSchemaContexts(authentication));
field.set(ProxyContext.getInstance(), getSchemaContexts(authentication));
}
private SchemaContexts getSchemaContexts(final Authentication authentication) {
......
......@@ -35,7 +35,7 @@ import org.apache.shardingsphere.infra.context.runtime.RuntimeContext;
import org.apache.shardingsphere.infra.context.schema.ShardingSphereSchema;
import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.mysql.command.admin.initdb.MySQLComInitDbExecutor;
import org.apache.shardingsphere.proxy.frontend.mysql.command.admin.ping.MySQLComPingExecutor;
import org.apache.shardingsphere.proxy.frontend.mysql.command.admin.quit.MySQLComQuitExecutor;
......@@ -66,9 +66,9 @@ public final class MySQLCommandExecutorFactoryTest {
@Test
@SneakyThrows(ReflectiveOperationException.class)
public void assertNewInstance() {
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
BackendConnection backendConnection = mock(BackendConnection.class);
when(backendConnection.getSchema()).thenReturn("schema");
......
......@@ -30,7 +30,7 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.Bac
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.rdl.parser.engine.ShardingSphereSQLParserEngine;
import org.junit.Before;
import org.junit.Test;
......@@ -61,9 +61,9 @@ public final class MySQLComStmtExecuteExecutorTest {
@Before
public void setUp() throws ReflectiveOperationException {
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
}
......
......@@ -32,7 +32,7 @@ import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.Postgre
import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.PostgreSQLRandomGenerator;
import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.PostgreSQLSSLNegativePacket;
import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.connection.ConnectionIdGenerator;
import org.apache.shardingsphere.proxy.frontend.auth.AuthenticationResultBuilder;
import org.apache.shardingsphere.proxy.frontend.auth.AuthenticationEngine;
......@@ -84,7 +84,7 @@ public final class PostgreSQLAuthenticationEngine implements AuthenticationEngin
PostgreSQLComStartupPacket comStartupPacket = new PostgreSQLComStartupPacket(payload);
startupMessageReceived.set(true);
String databaseName = comStartupPacket.getParametersMap().get(DATABASE_NAME_KEYWORD);
if (!Strings.isNullOrEmpty(databaseName) && !ProxySchemaContexts.getInstance().schemaExists(databaseName)) {
if (!Strings.isNullOrEmpty(databaseName) && !ProxyContext.getInstance().schemaExists(databaseName)) {
PostgreSQLErrorResponsePacket responsePacket = createErrorPacket(PostgreSQLErrorCode.INVALID_CATALOG_NAME, String.format("database \"%s\" does not exist", databaseName));
context.writeAndFlush(responsePacket);
context.close();
......
......@@ -18,13 +18,15 @@
package org.apache.shardingsphere.proxy.frontend.postgresql.auth;
import com.google.common.base.Strings;
import lombok.AccessLevel;
import lombok.NoArgsConstructor;
import org.apache.commons.codec.binary.Hex;
import org.apache.commons.codec.digest.DigestUtils;
import org.apache.commons.collections4.CollectionUtils;
import org.apache.shardingsphere.db.protocol.postgresql.constant.PostgreSQLErrorCode;
import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.PostgreSQLPasswordMessagePacket;
import org.apache.shardingsphere.infra.auth.ProxyUser;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import java.security.MessageDigest;
import java.util.Collection;
......@@ -33,7 +35,8 @@ import java.util.Map;
/**
* Authentication handler for PostgreSQL.
*/
public class PostgreSQLAuthenticationHandler {
@NoArgsConstructor(access = AccessLevel.PRIVATE)
public final class PostgreSQLAuthenticationHandler {
/**
* Login.
......@@ -46,7 +49,7 @@ public class PostgreSQLAuthenticationHandler {
*/
public static PostgreSQLLoginResult loginWithMd5Password(final String username, final String databaseName, final byte[] md5Salt, final PostgreSQLPasswordMessagePacket passwordMessagePacket) {
ProxyUser proxyUser = null;
for (Map.Entry<String, ProxyUser> entry : ProxySchemaContexts.getInstance().getSchemaContexts().getAuthentication().getUsers().entrySet()) {
for (Map.Entry<String, ProxyUser> entry : ProxyContext.getInstance().getSchemaContexts().getAuthentication().getUsers().entrySet()) {
if (entry.getKey().equals(username)) {
proxyUser = entry.getValue();
break;
......
......@@ -33,7 +33,7 @@ import org.apache.shardingsphere.db.protocol.postgresql.packet.generic.PostgreSQ
import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.command.executor.CommandExecutor;
import org.apache.shardingsphere.proxy.frontend.command.executor.QueryCommandExecutor;
import org.apache.shardingsphere.proxy.frontend.command.CommandExecuteEngine;
......@@ -87,7 +87,7 @@ public final class PostgreSQLCommandExecuteEngine implements CommandExecuteEngin
return;
}
int count = 0;
int proxyFrontendFlushThreshold = ProxySchemaContexts.getInstance().getSchemaContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD);
int proxyFrontendFlushThreshold = ProxyContext.getInstance().getSchemaContexts().getProps().<Integer>getValue(ConfigurationPropertyKey.PROXY_FRONTEND_FLUSH_THRESHOLD);
while (queryCommandExecutor.next()) {
count++;
while (!context.channel().isWritable() && context.channel().isActive()) {
......
......@@ -40,7 +40,7 @@ import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryData;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.command.executor.QueryCommandExecutor;
import org.apache.shardingsphere.proxy.frontend.postgresql.PostgreSQLErrPacketFactory;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
......@@ -74,7 +74,7 @@ public final class PostgreSQLComBindExecutor implements QueryCommandExecutor {
public PostgreSQLComBindExecutor(final PostgreSQLComBindPacket packet, final BackendConnection backendConnection) {
this.packet = packet;
SchemaContext schemaContext = ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema());
SchemaContext schemaContext = ProxyContext.getInstance().getSchema(backendConnection.getSchema());
if (null != packet.getSql() && null != schemaContext) {
SQLStatement sqlStatement = schemaContext.getRuntimeContext().getSqlParserEngine().parse(packet.getSql(), true);
databaseCommunicationEngine =
......@@ -86,7 +86,7 @@ public final class PostgreSQLComBindExecutor implements QueryCommandExecutor {
@Override
public Collection<DatabasePacket<?>> execute() {
if (ProxySchemaContexts.getInstance().getSchemaContexts().isCircuitBreak()) {
if (ProxyContext.getInstance().getSchemaContexts().isCircuitBreak()) {
return Collections.singletonList(new PostgreSQLErrorResponsePacket());
}
List<DatabasePacket<?>> result = new LinkedList<>();
......
......@@ -24,7 +24,7 @@ import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.bin
import org.apache.shardingsphere.db.protocol.postgresql.packet.command.query.binary.parse.PostgreSQLParseCompletePacket;
import org.apache.shardingsphere.infra.context.SchemaContext;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.command.executor.CommandExecutor;
import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
......@@ -44,7 +44,7 @@ public final class PostgreSQLComParseExecutor implements CommandExecutor {
public PostgreSQLComParseExecutor(final PostgreSQLComParsePacket packet, final BackendConnection backendConnection) {
this.packet = packet;
schema = ProxySchemaContexts.getInstance().getSchema(backendConnection.getSchema());
schema = ProxyContext.getInstance().getSchema(backendConnection.getSchema());
binaryStatementRegistry = BinaryStatementRegistry.getInstance().get(backendConnection.getConnectionId());
}
......
......@@ -34,7 +34,7 @@ import org.apache.shardingsphere.proxy.backend.response.BackendResponse;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.response.query.QueryResponse;
import org.apache.shardingsphere.proxy.backend.response.update.UpdateResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandler;
import org.apache.shardingsphere.proxy.backend.text.TextProtocolBackendHandlerFactory;
import org.apache.shardingsphere.proxy.frontend.command.executor.QueryCommandExecutor;
......@@ -70,7 +70,7 @@ public final class PostgreSQLComQueryExecutor implements QueryCommandExecutor {
@Override
public Collection<DatabasePacket<?>> execute() {
if (ProxySchemaContexts.getInstance().getSchemaContexts().isCircuitBreak()) {
if (ProxyContext.getInstance().getSchemaContexts().isCircuitBreak()) {
return Collections.singletonList(new PostgreSQLErrorResponsePacket());
}
BackendResponse backendResponse = getBackendResponse();
......
......@@ -28,7 +28,7 @@ import org.apache.shardingsphere.db.protocol.postgresql.packet.handshake.Postgre
import org.apache.shardingsphere.db.protocol.postgresql.payload.PostgreSQLPacketPayload;
import org.apache.shardingsphere.infra.auth.ProxyUser;
import org.apache.shardingsphere.infra.context.impl.StandardSchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.apache.shardingsphere.proxy.frontend.auth.AuthenticationResult;
import org.apache.shardingsphere.transaction.context.TransactionContexts;
import org.junit.Test;
......@@ -123,10 +123,10 @@ public final class PostgreSQLAuthenticationEngineTest {
payload.writeInt4(4 + md5Digest.length() + 1);
payload.writeStringNul(md5Digest);
ProxySchemaContexts proxySchemaContexts = ProxySchemaContexts.getInstance();
ProxyContext proxyContext = ProxyContext.getInstance();
StandardSchemaContexts standardSchemaContexts = new StandardSchemaContexts();
standardSchemaContexts.getAuthentication().getUsers().put(username, new ProxyUser(password, null));
proxySchemaContexts.init(standardSchemaContexts, mock(TransactionContexts.class));
proxyContext.init(standardSchemaContexts, mock(TransactionContexts.class));
actual = engine.auth(channelHandlerContext, payload);
assertThat(actual.isFinished(), is(password.equals(inputPassword)));
}
......
......@@ -28,7 +28,7 @@ import org.apache.shardingsphere.infra.database.type.dialect.MySQLDatabaseType;
import org.apache.shardingsphere.proxy.backend.communication.DatabaseCommunicationEngine;
import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
import org.apache.shardingsphere.proxy.backend.response.error.ErrorResponse;
import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
import org.apache.shardingsphere.proxy.backend.schema.ProxyContext;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.mockito.Mock;
......@@ -58,9 +58,9 @@ public final class PostgreSQLComBindExecutorTest {
@Test
@SneakyThrows
public void assertExecuteHasError() {
Field schemaContexts = ProxySchemaContexts.getInstance().getClass().getDeclaredField("schemaContexts");
Field schemaContexts = ProxyContext.getInstance().getClass().getDeclaredField("schemaContexts");
schemaContexts.setAccessible(true);
schemaContexts.set(ProxySchemaContexts.getInstance(),
schemaContexts.set(ProxyContext.getInstance(),
new StandardSchemaContexts(getSchemaContextMap(), new Authentication(), new ConfigurationProperties(new Properties()), new MySQLDatabaseType()));
BackendConnection connection = mock(BackendConnection.class);
when(connection.getSchema()).thenReturn("schema");
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册