├── .gitignore ├── LICENSE.txt ├── NOTICE.txt ├── README.md ├── README.txt ├── bin ├── fservers.sh ├── master-backup.sh ├── start-wasp.sh ├── stop-wasp.sh ├── wasp ├── wasp-config.sh ├── wasp-daemon.sh ├── wasp-daemons.sh ├── wirb.rb └── zookeepers.sh ├── conf ├── fservers ├── hadoop-metrics.properties ├── log4j.properties ├── wasp-env.sh └── wasp-site.xml ├── eclipse_formatter_apache.xml ├── image ├── wasp1.jpg ├── wasp2.jpg ├── wasp3.jpg ├── wasp4.jpg ├── wasp5.jpg └── wasp6.jpg ├── lib └── ruby │ ├── irb │ └── hirb.rb │ ├── shell.rb │ ├── shell │ ├── commands.rb │ ├── commands │ │ ├── assign.rb │ │ ├── balance_switch.rb │ │ ├── balancer.rb │ │ ├── close_entitygroup.rb │ │ ├── describe_index.rb │ │ ├── describe_table.rb │ │ ├── disable.rb │ │ ├── disable_all.rb │ │ ├── drop.rb │ │ ├── enable.rb │ │ ├── enable_all.rb │ │ ├── exists.rb │ │ ├── is_disabled.rb │ │ ├── is_enabled.rb │ │ ├── move.rb │ │ ├── query.rb │ │ ├── set_table_state.rb │ │ ├── show_indexes.rb │ │ ├── show_tables.rb │ │ ├── split.rb │ │ ├── sql.rb │ │ ├── status.rb │ │ ├── truncate.rb │ │ ├── unassign.rb │ │ ├── unlock_table.rb │ │ ├── version.rb │ │ └── zk_dump.rb │ └── formatter.rb │ ├── wasp.rb │ └── wasp │ ├── admin.rb │ └── wasp.rb ├── pom.xml └── src ├── assembly ├── bin.xml └── src.xml ├── main ├── jamon │ └── com │ │ └── alibaba │ │ └── wasp │ │ └── tmpl │ │ ├── common │ │ └── TaskMonitorTmpl.jamon │ │ ├── fserver │ │ ├── EntityGroupListTmpl.jamon │ │ ├── FSStatusTmpl.jamon │ │ └── ServerMetricsTmpl.jamon │ │ └── master │ │ ├── AssignmentManagerStatusTmpl.jamon │ │ ├── BackupMasterListTmpl.jamon │ │ ├── FServerListTmpl.jamon │ │ └── MasterStatusTmpl.jamon ├── java │ └── com │ │ └── alibaba │ │ └── wasp │ │ ├── ChildrenExistException.java │ │ ├── ClientConcernedException.java │ │ ├── ClockOutOfSyncException.java │ │ ├── ClusterId.java │ │ ├── ClusterStatus.java │ │ ├── DaemonThreadFactory.java │ │ ├── DataType.java │ │ ├── DeserializationException.java │ │ ├── EnityGroupOfflineException.java │ │ ├── EntityGroupException.java │ │ ├── EntityGroupInfo.java │ │ ├── EntityGroupLoad.java │ │ ├── EntityGroupLocation.java │ │ ├── EntityGroupMovedException.java │ │ ├── EntityGroupOfflineException.java │ │ ├── EntityGroupTransaction.java │ │ ├── FConstants.java │ │ ├── FieldKeyWord.java │ │ ├── GlobalEntityGroupInfo.java │ │ ├── IncorrectParameterException.java │ │ ├── InvalidColumnOperationException.java │ │ ├── LocalWaspCluster.java │ │ ├── MasterNotRunningException.java │ │ ├── MetaException.java │ │ ├── NoServerForEntityGroupException.java │ │ ├── NotAllChildTableDisableException.java │ │ ├── NotMatchPrimaryKeyException.java │ │ ├── NotServingEntityGroupException.java │ │ ├── PleaseHoldException.java │ │ ├── PrimaryKeyAlreadyExistsException.java │ │ ├── PrimaryKeyNotExistException.java │ │ ├── QueryConditionException.java │ │ ├── ReadModel.java │ │ ├── RemoteExceptionHandler.java │ │ ├── SQLErrorCode.java │ │ ├── SelectForUpdateLockNotFoundException.java │ │ ├── Server.java │ │ ├── ServerLoad.java │ │ ├── ServerName.java │ │ ├── TableExistsException.java │ │ ├── TableLockedException.java │ │ ├── TableNotDisabledException.java │ │ ├── TableNotEnabledException.java │ │ ├── TableNotFoundException.java │ │ ├── TransactionParseException.java │ │ ├── UnknownEntityGroupException.java │ │ ├── UnknownProtocolException.java │ │ ├── UnknownScannerException.java │ │ ├── UnknownSessionException.java │ │ ├── VersionAnnotation.java │ │ ├── WaspException.java │ │ ├── WaspIOException.java │ │ ├── YouAreDeadException.java │ │ ├── ZNodeClearer.java │ │ ├── ZooKeeperConnectionException.java │ │ ├── client │ │ ├── ClientProtocol.java │ │ ├── ConnectionUtils.java │ │ ├── ExecuteResult.java │ │ ├── FClient.java │ │ ├── FClientCallable.java │ │ ├── FClientFServerTracker.java │ │ ├── FClientInterface.java │ │ ├── FConnection.java │ │ ├── FConnectionManager.java │ │ ├── MasterAdminKeepAliveConnection.java │ │ ├── MasterMonitorKeepAliveConnection.java │ │ ├── MetaServerKeepAliveConnection.java │ │ ├── QueryResult.java │ │ ├── RetriesExhaustedException.java │ │ ├── ServerCallable.java │ │ ├── WaspAdmin.java │ │ ├── WriteResult.java │ │ └── ZooKeeperKeepAliveConnection.java │ │ ├── conf │ │ └── WaspConfiguration.java │ │ ├── coprocessor │ │ ├── DoubleColumnInterpreter.java │ │ ├── FloatColumnInterpreter.java │ │ ├── IntegerColumnInterpreter.java │ │ ├── WaspAggregateImplementation.java │ │ ├── WaspAggregateProtocol.java │ │ └── WaspAggregationClient.java │ │ ├── executor │ │ ├── EventHandler.java │ │ └── ExecutorService.java │ │ ├── fserver │ │ ├── AdminProtocol.java │ │ ├── DefaultEntityGroupSplitPolicy.java │ │ ├── EntityGroup.java │ │ ├── EntityGroupAlreadyInTransitionException.java │ │ ├── EntityGroupOpeningState.java │ │ ├── EntityGroupScanner.java │ │ ├── EntityGroupServices.java │ │ ├── EntityGroupSplitPolicy.java │ │ ├── FSDumpServlet.java │ │ ├── FSStatusServlet.java │ │ ├── FServer.java │ │ ├── FServerCommandLine.java │ │ ├── FServerRunningException.java │ │ ├── FServerServices.java │ │ ├── FServerStoppedException.java │ │ ├── GlobalEntityGroup.java │ │ ├── InternalScanner.java │ │ ├── LeaseException.java │ │ ├── LeaseListener.java │ │ ├── Leases.java │ │ ├── MXBean.java │ │ ├── MXBeanImpl.java │ │ ├── MetricsEntityGroupAggregateSourceImpl.java │ │ ├── MetricsEntityGroupSourceImpl.java │ │ ├── MetricsEntityGroupWrapperImpl.java │ │ ├── MetricsFServerSourceImpl.java │ │ ├── MetricsFServerWrapperImpl.java │ │ ├── MultiVersionConsistencyControl.java │ │ ├── OnlineEntityGroups.java │ │ ├── OperationStatus.java │ │ ├── SplitRequest.java │ │ ├── SplitThread.java │ │ ├── SplitTransaction.java │ │ ├── TwoPhaseCommit.java │ │ ├── TwoPhaseCommitProtocol.java │ │ ├── WrongEntityGroupException.java │ │ ├── handler │ │ │ ├── CloseEntityGroupHandler.java │ │ │ └── OpenEntityGroupHandler.java │ │ ├── metrics │ │ │ ├── FServerMetrics.java │ │ │ ├── MetricsEntityGroup.java │ │ │ ├── MetricsEntityGroupAggregateSource.java │ │ │ ├── MetricsEntityGroupSource.java │ │ │ ├── MetricsEntityGroupWrapper.java │ │ │ ├── MetricsFServer.java │ │ │ ├── MetricsFServerSource.java │ │ │ ├── MetricsFServerSourceFactory.java │ │ │ └── MetricsFServerWrapper.java │ │ └── redo │ │ │ ├── AlreadyCommitTransactionException.java │ │ │ ├── AlreadyExsitsTransactionIdException.java │ │ │ ├── NotInitlizedRedoException.java │ │ │ ├── Redo.java │ │ │ ├── RedoLog.java │ │ │ ├── RedoLogNotServingException.java │ │ │ ├── Transaction.java │ │ │ └── WALEdit.java │ │ ├── ipc │ │ ├── Call.java │ │ ├── CallFuture.java │ │ ├── Callback.java │ │ ├── ClientCache.java │ │ ├── NettyServer.java │ │ ├── NettyTransceiver.java │ │ ├── NettyTransportCodec.java │ │ ├── ProtobufRpcEngine.java │ │ ├── ProtocolSignature.java │ │ ├── RpcEngine.java │ │ ├── RpcServer.java │ │ ├── ServerNotRunningYetException.java │ │ ├── ServerRpcController.java │ │ ├── Transceiver.java │ │ ├── VersionedProtocol.java │ │ ├── WaspRPC.java │ │ └── WaspRPCErrorHandler.java │ │ ├── jdbc │ │ ├── CloseWatcher.java │ │ ├── ConnectionInfo.java │ │ ├── Driver.java │ │ ├── JdbcBatchUpdateException.java │ │ ├── JdbcConnection.java │ │ ├── JdbcException.java │ │ ├── JdbcPreparedStatement.java │ │ ├── JdbcResultFormatter.java │ │ ├── JdbcSQLException.java │ │ ├── JdbcStatement.java │ │ ├── Logger.java │ │ ├── SetTypes.java │ │ ├── command │ │ │ ├── CommandInterface.java │ │ │ └── CommandRemote.java │ │ ├── expression │ │ │ └── ParameterInterface.java │ │ ├── result │ │ │ ├── JdbcDatabaseMetaData.java │ │ │ ├── JdbcResultSet.java │ │ │ ├── JdbcResultSetMetaData.java │ │ │ ├── ResultColumn.java │ │ │ ├── ResultInterface.java │ │ │ ├── ResultRemote.java │ │ │ └── ResultTransfer.java │ │ └── value │ │ │ ├── CompareMode.java │ │ │ ├── CompareModeDefault.java │ │ │ ├── Value.java │ │ │ ├── ValueArray.java │ │ │ ├── ValueBoolean.java │ │ │ ├── ValueByte.java │ │ │ ├── ValueBytes.java │ │ │ ├── ValueDate.java │ │ │ ├── ValueDecimal.java │ │ │ ├── ValueDouble.java │ │ │ ├── ValueFloat.java │ │ │ ├── ValueInt.java │ │ │ ├── ValueLob.java │ │ │ ├── ValueLobDb.java │ │ │ ├── ValueLong.java │ │ │ ├── ValueNull.java │ │ │ ├── ValueProtobufObject.java │ │ │ ├── ValueShort.java │ │ │ ├── ValueString.java │ │ │ ├── ValueStringFixed.java │ │ │ ├── ValueStringIgnoreCase.java │ │ │ ├── ValueTime.java │ │ │ └── ValueTimestamp.java │ │ ├── jdbcx │ │ ├── JdbcConnectionPool.java │ │ ├── JdbcDataSource.java │ │ ├── JdbcDataSourceFactory.java │ │ ├── JdbcXAConnection.java │ │ └── JdbcXid.java │ │ ├── master │ │ ├── ActiveMasterManager.java │ │ ├── AssignCallable.java │ │ ├── AssignmentManager.java │ │ ├── BulkAssigner.java │ │ ├── BulkReOpen.java │ │ ├── CatalogJanitor.java │ │ ├── DeadServer.java │ │ ├── EntityGroupPlan.java │ │ ├── EntityGroupState.java │ │ ├── EntityGroupStates.java │ │ ├── FMaster.java │ │ ├── FMasterAdminProtocol.java │ │ ├── FMasterCommandLine.java │ │ ├── FMasterDumpServlet.java │ │ ├── FMasterMonitorProtocol.java │ │ ├── FMasterServices.java │ │ ├── FMasterStatusServlet.java │ │ ├── FMetaServerProtocol.java │ │ ├── FServerManager.java │ │ ├── FServerStatusProtocol.java │ │ ├── GeneralBulkAssigner.java │ │ ├── LoadBalancer.java │ │ ├── MasterProtocol.java │ │ ├── MetricsMasterSourceImpl.java │ │ ├── MetricsMasterWrapperImpl.java │ │ ├── OfflineCallback.java │ │ ├── TableLockManager.java │ │ ├── UnAssignCallable.java │ │ ├── balancer │ │ │ ├── BalancerChore.java │ │ │ ├── BaseLoadBalancer.java │ │ │ ├── ClusterLoadState.java │ │ │ ├── ClusterStatusChore.java │ │ │ ├── DefaultLoadBalancer.java │ │ │ ├── EntityGroupInfoComparator.java │ │ │ ├── LoadBalancerFactory.java │ │ │ └── ServerAndLoad.java │ │ ├── handler │ │ │ ├── ClosedEntityGroupHandler.java │ │ │ ├── CreateIndexHandler.java │ │ │ ├── CreateTableHandler.java │ │ │ ├── DeleteIndexHandler.java │ │ │ ├── DeleteTableHandler.java │ │ │ ├── DisableTableHandler.java │ │ │ ├── EnableTableHandler.java │ │ │ ├── ModifyTableHandler.java │ │ │ ├── OpenedEntityGroupHandler.java │ │ │ ├── ServerShutdownHandler.java │ │ │ ├── SplitEntityGroupHandler.java │ │ │ ├── TableEventHandler.java │ │ │ └── TruncateTableHandler.java │ │ └── metrics │ │ │ ├── MetricsMaster.java │ │ │ ├── MetricsMasterSource.java │ │ │ ├── MetricsMasterSourceFactory.java │ │ │ └── MetricsMasterWrapper.java │ │ ├── messagequeue │ │ ├── Broker.java │ │ ├── Message.java │ │ ├── MessageBroker.java │ │ ├── MessageID.java │ │ ├── MessageQueue.java │ │ ├── MessageQueueClient.java │ │ ├── Publisher.java │ │ ├── PublisherBuilder.java │ │ ├── Subscriber.java │ │ └── SubscriberBuilder.java │ │ ├── meta │ │ ├── AbstractMetaService.java │ │ ├── FMetaEditor.java │ │ ├── FMetaReader.java │ │ ├── FMetaScanner.java │ │ ├── FMetaServices.java │ │ ├── FMetaServicesImplWithoutRetry.java │ │ ├── FMetaStore.java │ │ ├── FMetaUtil.java │ │ ├── FMetaVisitor.java │ │ ├── FTable.java │ │ ├── Field.java │ │ ├── Index.java │ │ ├── IndexField.java │ │ ├── RowBuilder.java │ │ ├── StorageCleanChore.java │ │ ├── StorageTableNameBuilder.java │ │ └── TableSchemaCacheReader.java │ │ ├── metrics │ │ ├── BaseSource.java │ │ ├── BaseSourceImpl.java │ │ ├── DynamicMetricsRegistry.java │ │ ├── JmxCacheBuster.java │ │ ├── MetricsExecutor.java │ │ └── lib │ │ │ ├── MetricHistogram.java │ │ │ ├── MetricMutableHistogram.java │ │ │ ├── MetricMutableQuantiles.java │ │ │ ├── MetricQuantile.java │ │ │ └── MetricSampleQuantiles.java │ │ ├── monitoring │ │ ├── MonitoredTask.java │ │ ├── MonitoredTaskImpl.java │ │ ├── StateDumpServlet.java │ │ ├── TaskMonitor.java │ │ └── ThreadMonitoring.java │ │ ├── plan │ │ ├── AggregateQueryPlan.java │ │ ├── AlterTablePlan.java │ │ ├── BaseDriver.java │ │ ├── CreateIndexPlan.java │ │ ├── CreateTablePlan.java │ │ ├── DDLPlan.java │ │ ├── DMLPlan.java │ │ ├── DMLTransactionPlan.java │ │ ├── DQLPlan.java │ │ ├── DeletePlan.java │ │ ├── DescTablePlan.java │ │ ├── DropIndexPlan.java │ │ ├── DropTablePlan.java │ │ ├── GlobalQueryPlan.java │ │ ├── InsertPlan.java │ │ ├── LocalQueryPlan.java │ │ ├── NotingTodoPlan.java │ │ ├── Plan.java │ │ ├── ShowIndexesPlan.java │ │ ├── ShowTablesPlan.java │ │ ├── TruncateTablePlan.java │ │ ├── UpdatePlan.java │ │ ├── action │ │ │ ├── Action.java │ │ │ ├── ColumnAction.java │ │ │ ├── ColumnStruct.java │ │ │ ├── DMLAction.java │ │ │ ├── DeleteAction.java │ │ │ ├── GetAction.java │ │ │ ├── InsertAction.java │ │ │ ├── NoColumnPrimaryAction.java │ │ │ ├── NoPrimaryReadAction.java │ │ │ ├── Primary.java │ │ │ ├── PrimaryAction.java │ │ │ ├── Read.java │ │ │ ├── ReadAction.java │ │ │ ├── ScanAction.java │ │ │ ├── TransactionAction.java │ │ │ └── UpdateAction.java │ │ ├── execute │ │ │ ├── Execution.java │ │ │ └── ExecutionEngine.java │ │ ├── operator │ │ │ ├── AvgOperator.java │ │ │ ├── CountOperator.java │ │ │ ├── FetchOperator.java │ │ │ ├── FilterOperator.java │ │ │ ├── GroupByOperator.java │ │ │ ├── InsertOperator.java │ │ │ ├── JoinOperator.java │ │ │ ├── MaxOperator.java │ │ │ ├── MinOperator.java │ │ │ ├── Operator.java │ │ │ ├── PageOperator.java │ │ │ ├── SelectOperator.java │ │ │ ├── SumOperator.java │ │ │ ├── TableScanOperator.java │ │ │ └── UpdateOperator.java │ │ └── parser │ │ │ ├── AggregateInfo.java │ │ │ ├── Condition.java │ │ │ ├── ParseContext.java │ │ │ ├── Parser.java │ │ │ ├── QueryInfo.java │ │ │ ├── UnsupportedException.java │ │ │ ├── WaspParser.java │ │ │ └── druid │ │ │ ├── DruidDDLParser.java │ │ │ ├── DruidDMLParser.java │ │ │ ├── DruidDQLParser.java │ │ │ ├── DruidParser.java │ │ │ ├── FMetaEventOperation.java │ │ │ ├── MetaEventOperation.java │ │ │ └── dialect │ │ │ ├── Limit.java │ │ │ ├── WaspSqlAlterTableAddColumn.java │ │ │ ├── WaspSqlAlterTableAddIndex.java │ │ │ ├── WaspSqlAlterTableAddUnique.java │ │ │ ├── WaspSqlAlterTableChangeColumn.java │ │ │ ├── WaspSqlAlterTableCharacter.java │ │ │ ├── WaspSqlAlterTableStatement.java │ │ │ ├── WaspSqlColumnDefinition.java │ │ │ ├── WaspSqlCreateIndexStatement.java │ │ │ ├── WaspSqlCreateTableParser.java │ │ │ ├── WaspSqlCreateTableStatement.java │ │ │ ├── WaspSqlCreateUserStatement.java │ │ │ ├── WaspSqlDeleteStatement.java │ │ │ ├── WaspSqlDescribeStatement.java │ │ │ ├── WaspSqlDropTableStatement.java │ │ │ ├── WaspSqlDropUser.java │ │ │ ├── WaspSqlExecuteStatement.java │ │ │ ├── WaspSqlExprParser.java │ │ │ ├── WaspSqlHelpStatement.java │ │ │ ├── WaspSqlInsertStatement.java │ │ │ ├── WaspSqlKillStatement.java │ │ │ ├── WaspSqlLoadDataInFileStatement.java │ │ │ ├── WaspSqlLoadXmlStatement.java │ │ │ ├── WaspSqlLockTableStatement.java │ │ │ ├── WaspSqlParserUtils.java │ │ │ ├── WaspSqlPartitionByKey.java │ │ │ ├── WaspSqlPrepareStatement.java │ │ │ ├── WaspSqlRenameTableStatement.java │ │ │ ├── WaspSqlSelectParser.java │ │ │ ├── WaspSqlShowCharacterSetStatement.java │ │ │ ├── WaspSqlShowColumnsStatement.java │ │ │ ├── WaspSqlShowCreateDatabaseStatement.java │ │ │ ├── WaspSqlShowCreateTableStatement.java │ │ │ ├── WaspSqlShowDatabasesStatement.java │ │ │ ├── WaspSqlShowErrorsStatement.java │ │ │ ├── WaspSqlShowGrantsStatement.java │ │ │ ├── WaspSqlShowIndexesStatement.java │ │ │ ├── WaspSqlShowOpenTablesStatement.java │ │ │ ├── WaspSqlShowProcessListStatement.java │ │ │ ├── WaspSqlShowProfileStatement.java │ │ │ ├── WaspSqlShowProfilesStatement.java │ │ │ ├── WaspSqlShowStatusStatement.java │ │ │ ├── WaspSqlShowTableStatusStatement.java │ │ │ ├── WaspSqlShowTablesStatement.java │ │ │ ├── WaspSqlShowVariantsStatement.java │ │ │ ├── WaspSqlShowWarningsStatement.java │ │ │ ├── WaspSqlStartTransactionStatement.java │ │ │ ├── WaspSqlStatementParser.java │ │ │ ├── WaspSqlUnlockTablesStatement.java │ │ │ └── WaspSqlUpdateStatement.java │ │ ├── protobuf │ │ ├── ProtobufUtil.java │ │ ├── RequestConverter.java │ │ ├── ResponseConverter.java │ │ └── generated │ │ │ ├── AdminProtos.java │ │ │ ├── ClientProtos.java │ │ │ ├── ClusterIdProtos.java │ │ │ ├── ClusterStatusProtos.java │ │ │ ├── ComparatorProtos.java │ │ │ ├── FSProtos.java │ │ │ ├── FServerAdminProtos.java │ │ │ ├── FServerStatusProtos.java │ │ │ ├── LoadBalancerProtos.java │ │ │ ├── MasterAdminProtos.java │ │ │ ├── MasterMonitorProtos.java │ │ │ ├── MasterProtos.java │ │ │ ├── MetaProtos.java │ │ │ ├── RPCProtos.java │ │ │ ├── Tracing.java │ │ │ ├── WaspProtos.java │ │ │ └── ZooKeeperProtos.java │ │ ├── security │ │ └── User.java │ │ ├── session │ │ ├── ConnectionSession.java │ │ ├── ExecuteSession.java │ │ ├── ExecutionEngineSession.java │ │ ├── SessionFactory.java │ │ ├── SessionInterface.java │ │ └── SessionWithState.java │ │ ├── storage │ │ ├── StorageActionManager.java │ │ ├── StorageServices.java │ │ ├── StorageServicesImpl.java │ │ └── StorageTableNotFoundException.java │ │ ├── tmpl │ │ ├── common │ │ │ ├── TaskMonitorTmpl.java │ │ │ └── TaskMonitorTmplImpl.java │ │ ├── fserver │ │ │ ├── EntityGroupListTmpl.java │ │ │ ├── EntityGroupListTmplImpl.java │ │ │ ├── FSStatusTmpl.java │ │ │ ├── FSStatusTmplImpl.java │ │ │ ├── ServerMetricsTmpl.java │ │ │ └── ServerMetricsTmplImpl.java │ │ └── master │ │ │ ├── AssignmentManagerStatusTmpl.java │ │ │ ├── AssignmentManagerStatusTmplImpl.java │ │ │ ├── BackupMasterListTmpl.java │ │ │ ├── BackupMasterListTmplImpl.java │ │ │ ├── FServerListTmpl.java │ │ │ ├── FServerListTmplImpl.java │ │ │ ├── MasterStatusTmpl.java │ │ │ └── MasterStatusTmplImpl.java │ │ ├── util │ │ ├── ByteBufferInputStream.java │ │ ├── ByteBufferOutputStream.java │ │ ├── DateTimeUtils.java │ │ ├── IOUtils.java │ │ ├── InfoServer.java │ │ ├── InjectionEvent.java │ │ ├── InjectionHandler.java │ │ ├── Int32.java │ │ ├── Int64.java │ │ ├── JVMClusterUtil.java │ │ ├── JdbcUtils.java │ │ ├── KeyLocker.java │ │ ├── MathUtils.java │ │ ├── MutateList.java │ │ ├── New.java │ │ ├── Node.java │ │ ├── ParserUtils.java │ │ ├── ServerCommandLine.java │ │ ├── SmallLRUCache.java │ │ ├── StatementBuilder.java │ │ ├── StringUtils.java │ │ ├── Utils.java │ │ ├── VersionInfo.java │ │ ├── WaspConfTool.java │ │ └── WaspFsck.java │ │ └── zookeeper │ │ ├── ClusterStatusTracker.java │ │ ├── DrainingServerTracker.java │ │ ├── FQuorumPeer.java │ │ ├── FServerTracker.java │ │ ├── LoadBalancerTracker.java │ │ ├── MasterAddressTracker.java │ │ ├── RecoverableZooKeeper.java │ │ ├── ZKAssign.java │ │ ├── ZKClusterId.java │ │ ├── ZKConfig.java │ │ ├── ZKServerTool.java │ │ ├── ZKTable.java │ │ ├── ZKTableReadOnly.java │ │ ├── ZKUtil.java │ │ ├── ZooKeeperListener.java │ │ ├── ZooKeeperMainServerArg.java │ │ ├── ZooKeeperNodeTracker.java │ │ └── ZooKeeperWatcher.java ├── protobuf │ ├── Admin.proto │ ├── Client.proto │ ├── ClusterID.proto │ ├── ClusterStatus.proto │ ├── Comparator.proto │ ├── FS.proto │ ├── FServerAdmin.proto │ ├── FServerStatus.proto │ ├── LoadBalancer.proto │ ├── Master.proto │ ├── MasterAdmin.proto │ ├── MasterMonitor.proto │ ├── Meta.proto │ ├── RPC.proto │ ├── Tracing.proto │ ├── Wasp.proto │ └── ZooKeeper.proto └── resources │ ├── example │ └── sql │ ├── jdbc │ └── _messages_en.prop │ ├── wasp-default.xml │ └── wasp-webapps │ ├── fserver │ ├── fserver.jsp │ └── index.html │ ├── master │ ├── index.html │ ├── master.jsp │ ├── table.jsp │ ├── tablesDetailed.jsp │ └── zk.jsp │ └── static │ ├── css │ ├── bootstrap-responsive.css │ ├── bootstrap-responsive.min.css │ ├── bootstrap.css │ ├── bootstrap.min.css │ └── wasp.css │ ├── hbase_logo.png │ ├── hbase_logo_med.gif │ ├── hbase_logo_small.png │ ├── js │ ├── bootstrap.js │ ├── bootstrap.min.js │ ├── html5shiv.js │ ├── jquery.min.js │ └── tab.js │ ├── wasp_logo.jpg │ ├── wasp_logo_med.jpg │ └── wasp_logo_small.jpg ├── saveVersion.sh └── test ├── java └── com │ └── alibaba │ └── wasp │ ├── MiniWaspCluster.java │ ├── TestStartingCluster.java │ ├── WaspCluster.java │ ├── WaspTestingUtility.java │ ├── client │ ├── TestAdmin.java │ ├── TestDataCorrectness.java │ ├── TestFClientLoadBalance.java │ ├── TestFConnectionUtility.java │ └── TestFromClientSide.java │ ├── executor │ └── TestExecutorService.java │ ├── fserver │ ├── TestEntityGroup.java │ ├── TestEntityGroupAlone.java │ ├── TestEntityGroupInfo.java │ ├── TestEntityGroupOnCluster.java │ ├── TestSplitTransaction.java │ ├── TestSplitTransactionOnCluster.java │ ├── TestTwoPhaseCommit.java │ ├── handler │ │ ├── TestCloseEntityGroupHandler.java │ │ └── TestOpenEntityGroupHandler.java │ ├── metrics │ │ ├── MetricsAssertHelper.java │ │ ├── MetricsAssertHelperImpl.java │ │ ├── MetricsEntityGroupWrapperStub.java │ │ ├── MetricsFServerWrapperStub.java │ │ ├── TestMetricsEntityGroup.java │ │ ├── TestMetricsEntityGroupSourceImpl.java │ │ └── TestMetricsFServer.java │ └── redo │ │ ├── MemRedoLog.java │ │ ├── TestRedoLog.java │ │ └── TestTransaction.java │ ├── ipc │ ├── TestProtoBufRpc.java │ └── protobuf │ │ └── generated │ │ ├── TestProtos.java │ │ └── TestRpcServiceProtos.java │ ├── jdbc │ ├── TestAtomicOperation.java │ ├── TestDriver.java │ ├── TestJdbcBase.java │ ├── TestJdbcConnectionPool.java │ ├── TestJdbcResultFormatter.java │ ├── TestJdbcResultSet.java │ ├── TestJdbcStatement.java │ ├── TestPreparedStatement.java │ └── TestResultVerifier.java │ ├── master │ ├── Mocking.java │ ├── TestActiveMasterManager.java │ ├── TestAssignmentManager.java │ ├── TestAssignmentManagerOnCluster.java │ ├── TestDeadServer.java │ ├── TestMaster.java │ ├── TestMasterShutdown.java │ ├── TestOpenedEntityGroupHandler.java │ ├── TestRestartCluster.java │ ├── TestWaspRPCException.java │ └── balancer │ │ ├── TestBalancerBase.java │ │ └── TestDefaultLoadBalancer.java │ ├── messagequeue │ ├── TestPublisher.java │ └── TestSubscriber.java │ ├── meta │ ├── FMetaTestUtil.java │ ├── MemFMetaStore.java │ ├── RowBuilderTestUtil.java │ ├── TestFMetaStore.java │ ├── TestFTable.java │ ├── TestRowBuilder.java │ ├── TestStorageCleanChore.java │ └── TestTableSchemaCacheReader.java │ ├── metrics │ └── TestBaseSourceImpl.java │ ├── plan │ ├── action │ │ ├── ActionTestUtil.java │ │ ├── TestColumnStruct.java │ │ ├── TestInsertAction.java │ │ └── TestTransactionAction.java │ ├── execute │ │ └── TestExecutionEngine.java │ └── parser │ │ └── druid │ │ ├── DruidParserTestUtil.java │ │ ├── TestDruidDDLFailParser.java │ │ ├── TestDruidDDLParser.java │ │ ├── TestDruidDMLFailParser.java │ │ ├── TestDruidDMLParser.java │ │ ├── TestDruidDQLFailParser.java │ │ ├── TestDruidDQLParser.java │ │ ├── TestDruidParser.java │ │ └── TestFMetaEventOperation.java │ ├── protobuf │ └── TestProtobufUtil.java │ ├── storage │ └── NullStorageServices.java │ └── util │ ├── MockFServerServices.java │ ├── MockServer.java │ └── ResultInHBasePrinter.java └── protobuf ├── test.proto └── test_rpc_service.proto /.gitignore: -------------------------------------------------------------------------------- 1 | # Libraries 2 | # Libraries 3 | /lib/*.jar 4 | 5 | # Eclipse files 6 | .classpath 7 | .project 8 | .settings/ 9 | 10 | # Maven files 11 | target/ 12 | 13 | # Intellij IDEA files 14 | .iml 15 | .ipr 16 | .iws 17 | -------------------------------------------------------------------------------- /NOTICE.txt: -------------------------------------------------------------------------------- 1 | This product includes software developed by The Apache Software 2 | Foundation (http://www.apache.org/). 3 | 4 | In addition, this product includes software developed by: 5 | 6 | JUnit (http://www.junit.org/) included under the Common Public License v1.0. See 7 | the full text here: http://junit.sourceforge.net/cpl-v10.html 8 | 9 | JRuby (http://jruby.org) is tri-licensed. We include it under terms of the 10 | Common Public License v1.0. 11 | 12 | JRuby itself includes libraries variously licensed. See its COPYING document 13 | for details: https://github.com/jruby/jruby/blob/master/COPYING -------------------------------------------------------------------------------- /conf/fservers: -------------------------------------------------------------------------------- 1 | localhost -------------------------------------------------------------------------------- /conf/wasp-site.xml: -------------------------------------------------------------------------------- 1 | 2 | 3 | 15 | 16 | 17 | -------------------------------------------------------------------------------- /image/wasp1.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/alibaba/wasp/b2593d8e4b31ca6da0cd2f3e18356338d9b6dace/image/wasp1.jpg -------------------------------------------------------------------------------- /image/wasp2.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/alibaba/wasp/b2593d8e4b31ca6da0cd2f3e18356338d9b6dace/image/wasp2.jpg -------------------------------------------------------------------------------- /image/wasp3.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/alibaba/wasp/b2593d8e4b31ca6da0cd2f3e18356338d9b6dace/image/wasp3.jpg -------------------------------------------------------------------------------- /image/wasp4.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/alibaba/wasp/b2593d8e4b31ca6da0cd2f3e18356338d9b6dace/image/wasp4.jpg -------------------------------------------------------------------------------- /image/wasp5.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/alibaba/wasp/b2593d8e4b31ca6da0cd2f3e18356338d9b6dace/image/wasp5.jpg -------------------------------------------------------------------------------- /image/wasp6.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/alibaba/wasp/b2593d8e4b31ca6da0cd2f3e18356338d9b6dace/image/wasp6.jpg -------------------------------------------------------------------------------- /lib/ruby/shell/commands/assign.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class Assign < Command 24 | def help 25 | return <<-EOF 26 | Assign a entityGroup.Use with caution.If entityGroup already assigned, 27 | this command will just go ahead and reassign 28 | the entityGroup anyways. For experts only. 29 | EOF 30 | end 31 | 32 | def command(entityGroup_name) 33 | format_simple_command do 34 | admin.assign(entityGroup_name) 35 | end 36 | end 37 | end 38 | end 39 | end 40 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/balance_switch.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class BalanceSwitch < Command 24 | def help 25 | return <<-EOF 26 | Enable/Disable balancer. Returns previous balancer state. 27 | Examples: 28 | 29 | hbase> balance_switch true 30 | hbase> balance_switch false 31 | EOF 32 | end 33 | 34 | def command(enableDisable) 35 | format_simple_command do 36 | formatter.row([ 37 | admin.balance_switch(enableDisable)? "true" : "false" 38 | ]) 39 | end 40 | end 41 | end 42 | end 43 | end 44 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/balancer.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class Balancer < Command 24 | def help 25 | return <<-EOF 26 | Trigger the cluster balancer. Returns true if balancer ran and was able to 27 | tell the fservers to unassign all the regions to balance (the re-assignment itself is async). 28 | Otherwise false (Will not run if entityGroups in transition). 29 | EOF 30 | end 31 | 32 | def command() 33 | format_simple_command do 34 | formatter.row([ 35 | admin.balancer()? "true": "false" 36 | ]) 37 | end 38 | end 39 | end 40 | end 41 | end 42 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/describe_index.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class DescribeIndex < Command 24 | def help 25 | return <<-EOF 26 | Describe the named table. For example: 27 | wasp> describeIndex 't1','index1' 28 | EOF 29 | end 30 | 31 | def command(table, index) 32 | now = Time.now 33 | 34 | desc = admin.describe_index(table, index) 35 | 36 | #formatter.header([ "DESCRIPTION", "ENABLED" ], [ 64 ]) 37 | #formatter.row([ desc, admin.enabled?(table).to_s ], true, [ 64 ]) 38 | puts desc 39 | formatter.footer(now) 40 | end 41 | end 42 | end 43 | end 44 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/describe_table.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class DescribeTable < Command 24 | def help 25 | return <<-EOF 26 | Describe the named table. For example: 27 | wasp> describeTable 't1' 28 | EOF 29 | end 30 | 31 | def command(table) 32 | now = Time.now 33 | 34 | desc = admin.describe_table(table) 35 | 36 | #formatter.header([ "DESCRIPTION", "ENABLED" ], [ 64 ]) 37 | #formatter.row([ desc, admin.enabled?(table).to_s ], true, [ 64 ]) 38 | puts desc 39 | formatter.footer(now) 40 | end 41 | end 42 | end 43 | end 44 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/disable.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class Disable < Command 24 | def help 25 | return <<-EOF 26 | Start disable of named table: e.g. "hbase> disable 't1'" 27 | EOF 28 | end 29 | 30 | def command(table) 31 | format_simple_command do 32 | admin.disable(table) 33 | end 34 | end 35 | end 36 | end 37 | end 38 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/drop.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class Drop < Command 24 | def help 25 | return <<-EOF 26 | Drop the named table. Table must first be disabled. 27 | 28 | wasp> drop 'table name' 29 | EOF 30 | end 31 | 32 | def command(table) 33 | format_simple_command do 34 | admin.drop(table) 35 | end 36 | end 37 | end 38 | end 39 | end 40 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/enable.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class Enable < Command 24 | def help 25 | return <<-EOF 26 | Start enable of named table: e.g. "wasp> enable 't1'" 27 | EOF 28 | end 29 | 30 | def command(table) 31 | format_simple_command do 32 | admin.enable(table) 33 | end 34 | end 35 | end 36 | end 37 | end 38 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/exists.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class Exists < Command 24 | def help 25 | return <<-EOF 26 | Does the named table exist? e.g. "hbase> exists 't1'" 27 | EOF 28 | end 29 | 30 | def command(table) 31 | format_simple_command do 32 | formatter.row([ 33 | "Table #{table} " + (admin.exists?(table.to_s) ? "does exist" : "does not exist") 34 | ]) 35 | end 36 | end 37 | end 38 | end 39 | end 40 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/is_disabled.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class IsDisabled < Command 24 | def help 25 | return <<-EOF 26 | Is named table disabled?: e.g. "hbase> is_disabled 't1'" 27 | EOF 28 | end 29 | 30 | def command(table) 31 | format_simple_command do 32 | formatter.row([ 33 | admin.disabled?(table)? "true" : "false" 34 | ]) 35 | end 36 | end 37 | end 38 | end 39 | end 40 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/is_enabled.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class IsEnabled < Command 24 | def help 25 | return <<-EOF 26 | Is named table enabled?: e.g. "hbase> is_enabled 't1'" 27 | EOF 28 | end 29 | 30 | def command(table) 31 | format_simple_command do 32 | formatter.row([ 33 | admin.enabled?(table)? "true" : "false" 34 | ]) 35 | end 36 | end 37 | end 38 | end 39 | end 40 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/query.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class Query < Command 24 | def help 25 | return <<-EOF 26 | Execute a query sentence 27 | 28 | Examples: 29 | 30 | wasp> query "select * from table1 where col1='a'" 31 | wasp> query "select * from table1 where col1='a'",{MODEL=>'CURRENT'} 32 | wasp> query "select * from table1 where col1='a'",{MODEL=>'SNAPSHOT'} 33 | wasp> query "select * from table1 where col1='a'",{MODEL=>'INCONSISTENT'} 34 | EOF 35 | end 36 | 37 | def command(sql_sentence,args = {}) 38 | admin.executeQuery(sql_sentence,args) 39 | end 40 | end 41 | end 42 | end 43 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/set_table_state.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class SetTableState < Command 24 | def help 25 | return <<-EOF 26 | Set table state. State are DISABLED,DISABLING,ENABLING,ENABLED. For example: 27 | wasp> set_table_state 't1', 'DISABLED' 28 | EOF 29 | end 30 | def command(table_name, state) 31 | now = Time.now 32 | 33 | desc = admin.set_table_state(table_name, state) 34 | 35 | puts desc 36 | formatter.footer(now) 37 | end 38 | end 39 | end 40 | end 41 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/show_indexes.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class ShowIndexes < Command 24 | def help 25 | return <<-EOF 26 | List all tables in wasp. 27 | 28 | wasp> show_indexes 't1' 29 | EOF 30 | end 31 | 32 | def command(table) 33 | now = Time.now 34 | formatter.header([ "INDEX" ]) 35 | 36 | list = admin.show_indexes(table) 37 | list.each do |index| 38 | formatter.row([ index ]) 39 | end 40 | 41 | formatter.footer(now, list.size) 42 | end 43 | end 44 | end 45 | end 46 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/show_tables.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class ShowTables < Command 24 | def help 25 | return <<-EOF 26 | List all tables in wasp. 27 | 28 | wasp> show tables 29 | EOF 30 | end 31 | 32 | def command(regex = ".*") 33 | now = Time.now 34 | formatter.header([ "TABLE" ]) 35 | 36 | regex = /#{regex}/ unless regex.is_a?(Regexp) 37 | list = admin.show_tables.grep(regex) 38 | list.each do |table| 39 | formatter.row([ table ]) 40 | end 41 | 42 | formatter.footer(now, list.size) 43 | end 44 | end 45 | end 46 | end 47 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/sql.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class Sql < Command 24 | def help 25 | return <<-EOF 26 | Execute a sql sentence 27 | if a select sql please use ( query 'select * from t1' ) 28 | 29 | Examples: 30 | 31 | wasp> sql "create table tabname(col1 type1 [not null] [primary key],col2 type2 [not null],..)" 32 | wasp> sql "insert into table1(field1,field2) values(value1,value2)" 33 | EOF 34 | end 35 | 36 | def command(sql_sentence,args = {}) 37 | admin.executeSQL(sql_sentence,args) 38 | end 39 | end 40 | end 41 | end 42 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/status.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class Status < Command 24 | def help 25 | return <<-EOF 26 | Show cluster status. Can be 'summary', 'simple', or 'detailed'. The 27 | default is 'summary'. Examples: 28 | 29 | wasp> status 30 | wasp> status 'simple' 31 | wasp> status 'summary' 32 | wasp> status 'detailed' 33 | EOF 34 | end 35 | 36 | def command(format = 'summary') 37 | admin.status(format) 38 | end 39 | end 40 | end 41 | end 42 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/truncate.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class Truncate < Command 24 | def help 25 | return <<-EOF 26 | Disables, drops and recreates the specified table. 27 | EOF 28 | end 29 | 30 | def command(table) 31 | format_simple_command do 32 | puts "Truncating '#{table}' table (it may take a while):" 33 | admin.truncate(table) { |log| puts " - #{log}" } 34 | end 35 | end 36 | 37 | end 38 | end 39 | end 40 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/unlock_table.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class UnlockTable < Command 24 | def help 25 | return <<-EOF 26 | Unlock the named table. For example: 27 | wasp> unlock_table 't1' 28 | EOF 29 | end 30 | 31 | def command(table) 32 | now = Time.now 33 | 34 | desc = admin.unlock_table(table) 35 | 36 | #formatter.header([ "DESCRIPTION", "ENABLED" ], [ 64 ]) 37 | #formatter.row([ desc, admin.enabled?(table).to_s ], true, [ 64 ]) 38 | puts desc 39 | formatter.footer(now) 40 | end 41 | end 42 | end 43 | end 44 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/version.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class Version < Command 24 | def help 25 | return <<-EOF 26 | Output this Wasp version 27 | EOF 28 | end 29 | 30 | def command 31 | # Output version. 32 | puts "#{com.alibaba.wasp.util.VersionInfo.getVersion()}, " + 33 | "r#{com.alibaba.wasp.util.VersionInfo.getRevision()}, " + 34 | "#{com.alibaba.wasp.util.VersionInfo.getDate()}" 35 | end 36 | end 37 | end 38 | end 39 | -------------------------------------------------------------------------------- /lib/ruby/shell/commands/zk_dump.rb: -------------------------------------------------------------------------------- 1 | # 2 | # Copyright 2010 The Apache Software Foundation 3 | # 4 | # Licensed to the Apache Software Foundation (ASF) under one 5 | # or more contributor license agreements. See the NOTICE file 6 | # distributed with this work for additional information 7 | # regarding copyright ownership. The ASF licenses this file 8 | # to you under the Apache License, Version 2.0 (the 9 | # "License"); you may not use this file except in compliance 10 | # with the License. You may obtain a copy of the License at 11 | # 12 | # http://www.apache.org/licenses/LICENSE-2.0 13 | # 14 | # Unless required by applicable law or agreed to in writing, software 15 | # distributed under the License is distributed on an "AS IS" BASIS, 16 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | # See the License for the specific language governing permissions and 18 | # limitations under the License. 19 | # 20 | 21 | module Shell 22 | module Commands 23 | class ZkDump < Command 24 | def help 25 | return <<-EOF 26 | Dump status of HBase cluster as seen by ZooKeeper. 27 | EOF 28 | end 29 | 30 | def command 31 | puts admin.zk_dump 32 | end 33 | end 34 | end 35 | end 36 | -------------------------------------------------------------------------------- /src/assembly/src.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | src 6 | 7 | tar.gz 8 | 9 | 10 | 11 | 12 | ${basedir}/*.txt 13 | 14 | 15 | 16 | 17 | pom.xml 18 | 19 | 20 | 21 | src 22 | 23 | 24 | conf 25 | 26 | 27 | bin 28 | 755 29 | 30 | 31 | 32 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/ClockOutOfSyncException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | package com.alibaba.wasp; 20 | 21 | import java.io.IOException; 22 | 23 | /** 24 | * This exception is thrown by the master when a fserver clock skew is too 25 | * high. 26 | */ 27 | @SuppressWarnings("serial") 28 | public class ClockOutOfSyncException extends IOException { 29 | public ClockOutOfSyncException(String message) { 30 | super(message); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/DeserializationException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp; 19 | 20 | /** 21 | * Failed deserialization. 22 | */ 23 | @SuppressWarnings("serial") 24 | public class DeserializationException extends WaspException { 25 | public DeserializationException() { 26 | super(); 27 | } 28 | 29 | public DeserializationException(final String message) { 30 | super(message); 31 | } 32 | 33 | public DeserializationException(final String message, final Throwable t) { 34 | super(message, t); 35 | } 36 | 37 | public DeserializationException(final Throwable t) { 38 | super(t); 39 | } 40 | } 41 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/EnityGroupOfflineException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp; 19 | 20 | public class EnityGroupOfflineException extends EntityGroupException { 21 | 22 | private static final long serialVersionUID = 9019653481489057103L; 23 | 24 | public EnityGroupOfflineException() { 25 | } 26 | 27 | /** 28 | * @param s 29 | */ 30 | public EnityGroupOfflineException(String s) { 31 | super(s); 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/EntityGroupException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | package com.alibaba.wasp; 20 | 21 | import java.io.IOException; 22 | 23 | /** 24 | * Thrown when something happens related to entityGroup handling. Subclasses have to 25 | * be more specific. 26 | */ 27 | public class EntityGroupException extends IOException { 28 | private static final long serialVersionUID = 1473510258071111371L; 29 | 30 | /** default constructor */ 31 | public EntityGroupException() { 32 | super(); 33 | } 34 | 35 | /** 36 | * Constructor 37 | * 38 | * @param s 39 | * message 40 | */ 41 | public EntityGroupException(String s) { 42 | super(s); 43 | } 44 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/EntityGroupOfflineException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp; 21 | 22 | 23 | /** Thrown when a table can not be located */ 24 | public class EntityGroupOfflineException extends EntityGroupException { 25 | private static final long serialVersionUID = 466008402L; 26 | /** default constructor */ 27 | public EntityGroupOfflineException() { 28 | super(); 29 | } 30 | 31 | /** @param s message */ 32 | public EntityGroupOfflineException(String s) { 33 | super(s); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/GlobalEntityGroupInfo.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp; 19 | 20 | public class GlobalEntityGroupInfo extends EntityGroupInfo { 21 | 22 | public GlobalEntityGroupInfo() { 23 | super("Global"); 24 | } 25 | 26 | /** 27 | * @see com.alibaba.wasp.EntityGroupInfo#getEntityGroupNameAsString() 28 | */ 29 | @Override 30 | public String getEntityGroupNameAsString() { 31 | return FConstants.GLOBAL_ENTITYGROUP; 32 | } 33 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/IncorrectParameterException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp; 19 | 20 | import org.apache.hadoop.hbase.DoNotRetryIOException; 21 | 22 | public class IncorrectParameterException extends DoNotRetryIOException { 23 | 24 | private static final long serialVersionUID = 8997699186656752983L; 25 | 26 | public IncorrectParameterException() { 27 | } 28 | 29 | /** 30 | * @param arg0 31 | */ 32 | public IncorrectParameterException(String arg0) { 33 | super(arg0); 34 | } 35 | 36 | /** 37 | * @param arg0 38 | * @param arg1 39 | */ 40 | public IncorrectParameterException(String arg0, Throwable arg1) { 41 | super(arg0, arg1); 42 | } 43 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/InvalidColumnOperationException.java: -------------------------------------------------------------------------------- 1 | package com.alibaba.wasp; 2 | 3 | import org.apache.hadoop.hbase.DoNotRetryIOException; 4 | 5 | /** 6 | * Thrown if a request is table schema modification is requested but made for an 7 | * invalid column name. 8 | */ 9 | public class InvalidColumnOperationException extends DoNotRetryIOException { 10 | private static final long serialVersionUID = -872511639311444777L; 11 | 12 | /** default constructor */ 13 | public InvalidColumnOperationException() { 14 | super(); 15 | } 16 | 17 | /** 18 | * Constructor 19 | * @param s message 20 | */ 21 | public InvalidColumnOperationException(String s) { 22 | super(s); 23 | } 24 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/NoServerForEntityGroupException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp; 21 | 22 | 23 | /** 24 | * Thrown when no FServer can be found for a entityGroup 25 | */ 26 | public class NoServerForEntityGroupException extends EntityGroupException { 27 | private static final long serialVersionUID = 1L << 11 - 1L; 28 | 29 | /** default constructor */ 30 | public NoServerForEntityGroupException() { 31 | super(); 32 | } 33 | 34 | /** 35 | * Constructor 36 | * @param s message 37 | */ 38 | public NoServerForEntityGroupException(String s) { 39 | super(s); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/NotMatchPrimaryKeyException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp; 19 | 20 | public class NotMatchPrimaryKeyException extends ClientConcernedException { 21 | 22 | /** 23 | * 24 | */ 25 | private static final long serialVersionUID = -710818159811010988L; 26 | 27 | /** 28 | * 29 | */ 30 | public NotMatchPrimaryKeyException() { 31 | } 32 | 33 | /** 34 | * @param message 35 | */ 36 | public NotMatchPrimaryKeyException(String message) { 37 | super(message); 38 | } 39 | 40 | /** 41 | * @see com.alibaba.wasp.ClientConcernedException#getErrorCode() 42 | */ 43 | @Override 44 | public int getErrorCode() { 45 | return SQLErrorCode.PRIMARY_KEY_NOT_MATCH; 46 | } 47 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/PleaseHoldException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | package com.alibaba.wasp; 20 | 21 | import java.io.IOException; 22 | 23 | /** 24 | * This exception is thrown by the master when a entityGroup server was shut down and 25 | * restarted so fast that the master still hasn't processed the server shutdown 26 | * of the first instance, or when master is initializing and client call admin 27 | * operations 28 | */ 29 | @SuppressWarnings("serial") 30 | public class PleaseHoldException extends IOException { 31 | public PleaseHoldException(String message) { 32 | super(message); 33 | } 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/PrimaryKeyAlreadyExistsException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp; 21 | 22 | public class PrimaryKeyAlreadyExistsException extends ClientConcernedException { 23 | 24 | /** 25 | * 26 | */ 27 | private static final long serialVersionUID = -1202722426751237587L; 28 | 29 | public PrimaryKeyAlreadyExistsException() { 30 | } 31 | 32 | /** 33 | * @param message 34 | */ 35 | public PrimaryKeyAlreadyExistsException(String message) { 36 | super(message); 37 | } 38 | 39 | @Override 40 | public int getErrorCode() { 41 | return SQLErrorCode.PRIMARY_KEY_EXIST; 42 | } 43 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/PrimaryKeyNotExistException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp; 19 | 20 | 21 | public class PrimaryKeyNotExistException extends ClientConcernedException { 22 | 23 | /** 24 | * 25 | */ 26 | private static final long serialVersionUID = -1202722426751237587L; 27 | 28 | public PrimaryKeyNotExistException(){ 29 | } 30 | 31 | /** 32 | * @param message 33 | */ 34 | public PrimaryKeyNotExistException(String message) { 35 | super(message); 36 | } 37 | 38 | @Override 39 | public int getErrorCode() { 40 | return SQLErrorCode.PRIMARY_KEY_NOT_EXIST; 41 | } 42 | 43 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/QueryConditionException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp; 19 | 20 | public class QueryConditionException extends ClientConcernedException { 21 | 22 | /** 23 | * 24 | */ 25 | private static final long serialVersionUID = -710818159811010988L; 26 | 27 | /** 28 | * 29 | */ 30 | public QueryConditionException() { 31 | } 32 | 33 | /** 34 | * @param message 35 | */ 36 | public QueryConditionException(String message) { 37 | super(message); 38 | } 39 | 40 | /** 41 | * @see ClientConcernedException#getErrorCode() 42 | */ 43 | @Override 44 | public int getErrorCode() { 45 | return SQLErrorCode.QUERY_CONDITION_ERROR; 46 | } 47 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/ReadModel.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp; 21 | 22 | public enum ReadModel { 23 | CURRENT, 24 | 25 | SNAPSHOT, 26 | 27 | INCONSISTENT; 28 | 29 | } 30 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/TableExistsException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, software 10 | * distributed under the License is distributed on an "AS IS" BASIS, 11 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | * See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | package com.alibaba.wasp; 16 | 17 | 18 | /** 19 | * Thrown when a table exists but should not 20 | */ 21 | public class TableExistsException extends ClientConcernedException { 22 | private static final long serialVersionUID = 1L << 7 - 1L; 23 | 24 | /** default constructor */ 25 | public TableExistsException() { 26 | super(); 27 | } 28 | 29 | /** 30 | * Constructor 31 | * 32 | * @param s 33 | * message 34 | */ 35 | public TableExistsException(String s) { 36 | super(s); 37 | } 38 | 39 | /** 40 | * @see com.alibaba.wasp.ClientConcernedException#getErrorCode() 41 | */ 42 | @Override 43 | public int getErrorCode() { 44 | return SQLErrorCode.GENERAL_ERROR_1; 45 | } 46 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/TableNotFoundException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | package com.alibaba.wasp; 20 | 21 | 22 | /** Thrown when a table can not be located */ 23 | 24 | public class TableNotFoundException extends ClientConcernedException { 25 | private static final long serialVersionUID = 993179627856392526L; 26 | 27 | /** default constructor */ 28 | public TableNotFoundException() { 29 | super(); 30 | } 31 | 32 | /** 33 | * @param s 34 | * message 35 | */ 36 | public TableNotFoundException(String s) { 37 | super(s); 38 | } 39 | 40 | /** 41 | * @see ClientConcernedException#getErrorCode() 42 | */ 43 | @Override 44 | public int getErrorCode() { 45 | return SQLErrorCode.GENERAL_ERROR_1; 46 | } 47 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/TransactionParseException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed under the Apache License, Version 2.0 (the "License"); 4 | * you may not use this file except in compliance with the License. 5 | * You may obtain a copy of the License at 6 | * 7 | * http://www.apache.org/licenses/LICENSE-2.0 8 | * 9 | * Unless required by applicable law or agreed to in writing, software 10 | * distributed under the License is distributed on an "AS IS" BASIS, 11 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 12 | * See the License for the specific language governing permissions and 13 | * limitations under the License. 14 | */ 15 | package com.alibaba.wasp; 16 | 17 | 18 | /** 19 | * Thrown when parse transaction get anything not allow. 20 | */ 21 | public class TransactionParseException extends ClientConcernedException { 22 | private static final long serialVersionUID = 1L << 7 - 1L; 23 | 24 | /** default constructor */ 25 | public TransactionParseException() { 26 | super(); 27 | } 28 | 29 | /** 30 | * Constructor 31 | * 32 | * @param s 33 | * message 34 | */ 35 | public TransactionParseException(String s) { 36 | super(s); 37 | } 38 | 39 | /** 40 | * @see ClientConcernedException#getErrorCode() 41 | */ 42 | @Override 43 | public int getErrorCode() { 44 | return SQLErrorCode.PARSE_TRANSACTION_ERROR; 45 | } 46 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/UnknownEntityGroupException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | package com.alibaba.wasp; 20 | 21 | /** 22 | * Thrown when we are asked to operate on a entityGroup we know nothing about. 23 | */ 24 | public class UnknownEntityGroupException extends EntityGroupException { 25 | private static final long serialVersionUID = 1968858760475205392L; 26 | 27 | public UnknownEntityGroupException(String entityGroupName) { 28 | super(entityGroupName); 29 | } 30 | } 31 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/UnknownScannerException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp; 19 | 20 | import org.apache.hadoop.hbase.DoNotRetryIOException; 21 | 22 | public class UnknownScannerException extends DoNotRetryIOException { 23 | 24 | private static final long serialVersionUID = 4240700719149477391L; 25 | 26 | public UnknownScannerException() { 27 | 28 | } 29 | 30 | /** 31 | * @param arg0 32 | */ 33 | public UnknownScannerException(String arg0) { 34 | super(arg0); 35 | } 36 | 37 | /** 38 | * @param arg0 39 | * @param arg1 40 | */ 41 | public UnknownScannerException(String arg0, Throwable arg1) { 42 | super(arg0, arg1); 43 | 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/UnknownSessionException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp; 19 | 20 | import org.apache.hadoop.hbase.DoNotRetryIOException; 21 | 22 | public class UnknownSessionException extends DoNotRetryIOException { 23 | 24 | private static final long serialVersionUID = 6073369246219011709L; 25 | 26 | public UnknownSessionException() { 27 | 28 | } 29 | 30 | /** 31 | * @param arg0 32 | */ 33 | public UnknownSessionException(String arg0) { 34 | super(arg0); 35 | 36 | } 37 | 38 | /** 39 | * @param arg0 40 | * @param arg1 41 | */ 42 | public UnknownSessionException(String arg0, Throwable arg1) { 43 | super(arg0, arg1); 44 | 45 | } 46 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/WaspException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp; 19 | 20 | /** 21 | * Base checked exception in Wasp. 22 | * 23 | */ 24 | @SuppressWarnings("serial") 25 | public class WaspException extends Exception { 26 | public WaspException() { 27 | super(); 28 | } 29 | 30 | public WaspException(final String message) { 31 | super(message); 32 | } 33 | 34 | public WaspException(final String message, final Throwable t) { 35 | super(message, t); 36 | } 37 | 38 | public WaspException(final Throwable t) { 39 | super(t); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/YouAreDeadException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | package com.alibaba.wasp; 20 | 21 | import java.io.IOException; 22 | 23 | /** 24 | * This exception is thrown by the master when a entityGroup server reports and is 25 | * already being processed as dead. This can happen when a entityGroup server loses 26 | * its session but didn't figure it yet. 27 | */ 28 | @SuppressWarnings("serial") 29 | public class YouAreDeadException extends IOException { 30 | public YouAreDeadException(String message) { 31 | super(message); 32 | } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/client/MetaServerKeepAliveConnection.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.client; 19 | 20 | import com.alibaba.wasp.master.FMetaServerProtocol; 21 | 22 | import java.io.Closeable; 23 | 24 | public interface MetaServerKeepAliveConnection extends FMetaServerProtocol,Closeable { 25 | 26 | @Override 27 | public void close(); 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/fserver/DefaultEntityGroupSplitPolicy.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.fserver; 19 | 20 | public class DefaultEntityGroupSplitPolicy extends EntityGroupSplitPolicy { 21 | 22 | public DefaultEntityGroupSplitPolicy() { 23 | super(); 24 | } 25 | 26 | /** 27 | * @see com.alibaba.wasp.fserver.EntityGroupSplitPolicy#getSplitPoint(byte[]) 28 | */ 29 | @Override 30 | protected byte[] getSplitPoint(byte[] splitPoint) { 31 | return splitPoint; 32 | } 33 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/fserver/EntityGroupAlreadyInTransitionException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.fserver; 21 | 22 | import java.io.IOException; 23 | 24 | /** 25 | * This exception is thrown when a FServer is asked to open or close 26 | * a entityGroup but it's already processing it 27 | */ 28 | public class EntityGroupAlreadyInTransitionException extends IOException { 29 | 30 | private static final long serialVersionUID = 443073202297226375L; 31 | 32 | public EntityGroupAlreadyInTransitionException(String s) { 33 | super(s); 34 | } 35 | 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/fserver/EntityGroupOpeningState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2011 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.fserver; 21 | 22 | public enum EntityGroupOpeningState { 23 | 24 | OPENED, 25 | 26 | ALREADY_OPENED, 27 | 28 | FAILED_OPENING; 29 | } 30 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/fserver/EntityGroupServices.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.fserver; 19 | 20 | import com.alibaba.wasp.EntityGroupInfo; 21 | import org.apache.hadoop.conf.Configuration; 22 | 23 | public interface EntityGroupServices { 24 | 25 | /** 26 | * return Configuration. 27 | * 28 | * @return 29 | */ 30 | public Configuration getConf(); 31 | 32 | /** 33 | * return FServerServices. 34 | * 35 | * @return 36 | */ 37 | public FServerServices getFServerServices(); 38 | 39 | /** 40 | * return EntityGroupInfo. 41 | * 42 | * @return 43 | */ 44 | public EntityGroupInfo getEntityGroupInfo(); 45 | 46 | } 47 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/fserver/FServerStoppedException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.fserver; 21 | 22 | import java.io.IOException; 23 | 24 | /** 25 | * Thrown by the entityGroup server when it is in shutting down state. 26 | */ 27 | @SuppressWarnings("serial") 28 | public class FServerStoppedException extends IOException { 29 | 30 | public FServerStoppedException(String s) { 31 | super(s); 32 | } 33 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/fserver/LeaseException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | package com.alibaba.wasp.fserver; 20 | 21 | import org.apache.hadoop.hbase.DoNotRetryIOException; 22 | 23 | /** 24 | * Reports a problem with a lease 25 | */ 26 | public class LeaseException extends DoNotRetryIOException { 27 | 28 | private static final long serialVersionUID = 8179703995292418650L; 29 | 30 | /** default constructor */ 31 | public LeaseException() { 32 | super(); 33 | } 34 | 35 | /** 36 | * @param message 37 | */ 38 | public LeaseException(String message) { 39 | super(message); 40 | } 41 | } 42 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/fserver/LeaseListener.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | package com.alibaba.wasp.fserver; 20 | 21 | 22 | 23 | /** 24 | * LeaseListener is an interface meant to be implemented by users of the Leases 25 | * class. 26 | * 27 | * It receives events from the Leases class about the status of its accompanying 28 | * lease. Users of the Leases class can use a LeaseListener subclass to, for 29 | * example, clean up resources after a lease has expired. 30 | */ 31 | public interface LeaseListener { 32 | 33 | /** When a lease expires, this method is called. */ 34 | public void leaseExpired(); 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/fserver/MXBean.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.alibaba.wasp.fserver; 20 | 21 | /** 22 | * This is the JMX management interface for HBase Region Server information 23 | */ 24 | public interface MXBean { 25 | 26 | /** 27 | * Return FServer's ServerName 28 | * 29 | * @return ServerName 30 | */ 31 | public String getServerName(); 32 | 33 | /** 34 | * Get Zookeeper Quorum 35 | * 36 | * @return Comma-separated list of Zookeeper Quorum servers 37 | */ 38 | public String getZookeeperQuorum(); 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/fserver/WrongEntityGroupException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.fserver; 21 | 22 | import java.io.IOException; 23 | 24 | /** 25 | * Thrown when a request contains a key which is not part of this entityGroup 26 | */ 27 | public class WrongEntityGroupException extends IOException { 28 | private static final long serialVersionUID = 993179627856392526L; 29 | 30 | /** constructor */ 31 | public WrongEntityGroupException() { 32 | super(); 33 | } 34 | 35 | /** 36 | * Constructor 37 | * @param s message 38 | */ 39 | public WrongEntityGroupException(String s) { 40 | super(s); 41 | } 42 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/ipc/Callback.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | package com.alibaba.wasp.ipc; 20 | 21 | /** 22 | * Interface for receiving asynchronous callbacks. 23 | * For each request with an asynchronous callback, 24 | * either {@link #handleResult(Object)} or {@link #handleError(Exception)} 25 | * will be invoked. 26 | */ 27 | public interface Callback { 28 | /** 29 | * Receives a callback result. 30 | * @param result the result returned in the callback. 31 | */ 32 | void handleResult(T result); 33 | 34 | /** 35 | * Receives an error. 36 | * @param error the error returned in the callback. 37 | */ 38 | void handleError(Throwable error); 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/ipc/ServerNotRunningYetException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package com.alibaba.wasp.ipc; 22 | 23 | import java.io.IOException; 24 | 25 | public class ServerNotRunningYetException extends IOException { 26 | 27 | private static final long serialVersionUID = -1037907015006723006L; 28 | 29 | public ServerNotRunningYetException(String s) { 30 | super(s); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/ipc/WaspRPCErrorHandler.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package com.alibaba.wasp.ipc; 22 | 23 | /** 24 | * An interface for calling out of RPC for error conditions. 25 | */ 26 | public interface WaspRPCErrorHandler { 27 | /** 28 | * Take actions on the event of an OutOfMemoryError. 29 | * @param e the throwable 30 | * @return if the server should be shut down 31 | */ 32 | public boolean checkOOME(final Throwable e) ; 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/messagequeue/Broker.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.messagequeue; 19 | 20 | import com.alibaba.wasp.EntityGroupInfo; 21 | 22 | import java.io.IOException; 23 | 24 | public interface Broker { 25 | /** 26 | * 27 | * @param subscriber 28 | */ 29 | public void register(Subscriber subscriber) throws IOException; 30 | 31 | /** 32 | * 33 | * @param subscriber 34 | */ 35 | public void remove(Subscriber subscriber) throws IOException; 36 | 37 | /** 38 | * 39 | * @param entityGroupInfo 40 | */ 41 | public void remove(EntityGroupInfo entityGroupInfo) throws IOException; 42 | } 43 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/messagequeue/Message.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.messagequeue; 19 | 20 | public interface Message { 21 | 22 | /** 23 | * 24 | * @return 25 | */ 26 | public MessageID getMessageID(); 27 | 28 | /** 29 | * @return the isCommited 30 | */ 31 | public boolean isCommited(); 32 | 33 | /** 34 | * @param isCommited 35 | * the isCommited to set 36 | */ 37 | public void setCommited(boolean isCommited); 38 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/messagequeue/MessageQueue.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.messagequeue; 19 | 20 | import java.io.IOException; 21 | 22 | /** 23 | * Base class; 24 | * 25 | */ 26 | public interface MessageQueue { 27 | 28 | /** 29 | * receive message and call entityGroup's doAsynchronous 30 | * 31 | * @return 32 | */ 33 | public abstract T doAsynchronous(Message message) throws IOException; 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/meta/AbstractMetaService.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.meta; 19 | 20 | import com.alibaba.wasp.MetaException; 21 | import org.apache.hadoop.conf.Configuration; 22 | 23 | public abstract class AbstractMetaService { 24 | 25 | public static FMetaServices globalFMetaservice; 26 | 27 | public static FMetaServices getService(Configuration configuration) 28 | throws MetaException { 29 | if (globalFMetaservice == null) { 30 | globalFMetaservice = new FMetaStore(configuration); 31 | } 32 | return globalFMetaservice; 33 | } 34 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/meta/FMetaVisitor.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with this 6 | * work for additional information regarding copyright ownership. The ASF 7 | * licenses this file to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT 15 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the 16 | * License for the specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package com.alibaba.wasp.meta; 20 | 21 | import org.apache.hadoop.hbase.client.Result; 22 | 23 | import java.io.IOException; 24 | 25 | /** 26 | * Implementations 'visit' a catalog table row. 27 | */ 28 | public interface FMetaVisitor { 29 | /** 30 | * Visit the catalog table row. 31 | * 32 | * @param r A row from catalog table 33 | * @return True if we are to proceed scanning the table, else false if we are 34 | * to stop now. 35 | */ 36 | public boolean visit(final Result r) throws IOException; 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/CreateIndexPlan.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan; 19 | 20 | import com.alibaba.wasp.meta.Index; 21 | 22 | /** 23 | * The plan to create index. 24 | * 25 | */ 26 | public class CreateIndexPlan extends DDLPlan { 27 | 28 | private Index index; 29 | 30 | public CreateIndexPlan(Index index) { 31 | this.index = index; 32 | } 33 | 34 | public Index getIndex() { 35 | return index; 36 | } 37 | 38 | public void setIndex(Index index) { 39 | this.index = index; 40 | } 41 | 42 | /** 43 | * @see Object#toString() 44 | */ 45 | @Override 46 | public String toString() { 47 | return "CreateIndexPlan [index=" + index + "]"; 48 | } 49 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/DDLPlan.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan; 19 | 20 | /** 21 | * DDL's basic plan used to distinguish between DML. 22 | */ 23 | public abstract class DDLPlan extends Plan { 24 | 25 | /** 26 | * Default constructor. 27 | */ 28 | public DDLPlan() { 29 | } 30 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/DMLPlan.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan; 19 | 20 | import com.alibaba.wasp.plan.action.DMLAction; 21 | 22 | import java.util.List; 23 | 24 | /** 25 | * DML's basic plan used to distinguish between DDL. 26 | */ 27 | public abstract class DMLPlan extends Plan { 28 | 29 | /** 30 | * Default constructor. 31 | */ 32 | public DMLPlan() { 33 | } 34 | 35 | public abstract List getActions(); 36 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/NotingTodoPlan.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan; 19 | 20 | /** 21 | * noting to do plan 22 | * eg. "CREATE TABLE if not exists test. if test table exits, nothing to do 23 | * 24 | */ 25 | public class NotingTodoPlan extends DDLPlan { 26 | 27 | /** 28 | * @see Object#toString() 29 | */ 30 | @Override 31 | public String toString() { 32 | return "NotingTodoPlan"; 33 | } 34 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/Plan.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan; 19 | 20 | public abstract class Plan { 21 | 22 | public Plan() { 23 | } 24 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/action/DMLAction.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.action; 19 | 20 | import com.alibaba.wasp.EntityGroupLocation; 21 | 22 | public interface DMLAction { 23 | 24 | public String getTableName(); 25 | 26 | public EntityGroupLocation getEntityGroupLocation(); 27 | 28 | } 29 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/action/NoColumnPrimaryAction.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.action; 19 | 20 | public abstract class NoColumnPrimaryAction extends Action implements Primary { 21 | /** combined Primary Key's value **/ 22 | protected byte[] combinedPrimaryKey; 23 | 24 | /** 25 | * @return the primayKey 26 | */ 27 | public byte[] getCombinedPrimaryKey() { 28 | return combinedPrimaryKey; 29 | } 30 | 31 | /** 32 | * @param primayKey 33 | * the primayKey to set 34 | */ 35 | public void setCombinedPrimaryKey(byte[] primayKey) { 36 | this.combinedPrimaryKey = primayKey; 37 | } 38 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/action/NoPrimaryReadAction.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.action; 19 | 20 | import com.alibaba.wasp.ReadModel; 21 | 22 | public abstract class NoPrimaryReadAction extends ColumnAction implements Read { 23 | 24 | /** current,snapshot,inconsistent **/ 25 | protected ReadModel readerMode; 26 | 27 | /** 28 | * @return the readerMode 29 | */ 30 | public ReadModel getReaderMode() { 31 | return readerMode; 32 | } 33 | 34 | /** 35 | * @param readerMode 36 | * the readerMode to set 37 | */ 38 | public void setReaderMode(ReadModel readerMode) { 39 | this.readerMode = readerMode; 40 | } 41 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/action/Primary.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.action; 19 | 20 | public interface Primary { 21 | /** 22 | * @return the primayKey 23 | */ 24 | public byte[] getCombinedPrimaryKey(); 25 | 26 | /** 27 | * @param primayKey 28 | * the primayKey to set 29 | */ 30 | public void setCombinedPrimaryKey(byte[] primayKey); 31 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/action/PrimaryAction.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.action; 19 | 20 | public abstract class PrimaryAction extends ColumnAction implements Primary { 21 | 22 | /** combined Primary Key's value **/ 23 | protected byte[] combinedPrimaryKey; 24 | 25 | /** 26 | * @return the primayKey 27 | */ 28 | public byte[] getCombinedPrimaryKey() { 29 | return combinedPrimaryKey; 30 | } 31 | 32 | /** 33 | * @param primayKey 34 | * the primayKey to set 35 | */ 36 | public void setCombinedPrimaryKey(byte[] primayKey) { 37 | this.combinedPrimaryKey = primayKey; 38 | } 39 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/action/Read.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.action; 19 | 20 | import com.alibaba.wasp.ReadModel; 21 | 22 | public interface Read { 23 | /** 24 | * @return the readerMode 25 | */ 26 | public ReadModel getReaderMode(); 27 | 28 | /** 29 | * @param readerMode 30 | * the readerMode to set 31 | */ 32 | public void setReaderMode(ReadModel readerMode); 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/AvgOperator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.plan.operator; 21 | 22 | 23 | 24 | public class AvgOperator extends Operator { 25 | 26 | /* (non-Javadoc) 27 | * @see com.alibaba.wasp.util.Node#getName() 28 | */ 29 | @Override 30 | public String getName() { 31 | // TODO Auto-generated method stub 32 | return null; 33 | } 34 | 35 | @Override 36 | public Object forward(Object row) { 37 | // TODO Auto-generated method stub 38 | return null; 39 | } 40 | 41 | } 42 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/CountOperator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.plan.operator; 21 | 22 | 23 | public class CountOperator extends Operator { 24 | 25 | 26 | @Override 27 | public String getName() { 28 | // TODO Auto-generated method stub 29 | return null; 30 | } 31 | 32 | @Override 33 | public Object forward(Object row) { 34 | // TODO Auto-generated method stub 35 | return null; 36 | } 37 | 38 | } 39 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/FetchOperator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.plan.operator; 21 | 22 | 23 | public class FetchOperator extends Operator { 24 | 25 | @Override 26 | public String getName() { 27 | // TODO Auto-generated method stub 28 | return null; 29 | } 30 | 31 | @Override 32 | public Object forward(Object row) { 33 | // TODO Auto-generated method stub 34 | return null; 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/FilterOperator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.plan.operator; 21 | 22 | 23 | public class FilterOperator extends Operator { 24 | 25 | @Override 26 | public String getName() { 27 | // TODO Auto-generated method stub 28 | return null; 29 | } 30 | 31 | @Override 32 | public Object forward(Object row) { 33 | // TODO Auto-generated method stub 34 | return null; 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/GroupByOperator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.plan.operator; 21 | 22 | 23 | public class GroupByOperator extends Operator { 24 | 25 | 26 | @Override 27 | public String getName() { 28 | // TODO Auto-generated method stub 29 | return null; 30 | } 31 | 32 | @Override 33 | public Object forward(Object row) { 34 | // TODO Auto-generated method stub 35 | return null; 36 | } 37 | 38 | } 39 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/InsertOperator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.plan.operator; 21 | 22 | 23 | public class InsertOperator extends Operator { 24 | 25 | @Override 26 | public String getName() { 27 | // TODO Auto-generated method stub 28 | return null; 29 | } 30 | 31 | @Override 32 | public Object forward(Object row) { 33 | // TODO Auto-generated method stub 34 | return null; 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/JoinOperator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.plan.operator; 21 | 22 | 23 | public class JoinOperator extends Operator { 24 | 25 | @Override 26 | public String getName() { 27 | // TODO Auto-generated method stub 28 | return null; 29 | } 30 | 31 | @Override 32 | public Object forward(Object row) { 33 | // TODO Auto-generated method stub 34 | return null; 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/MaxOperator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | package com.alibaba.wasp.plan.operator; 20 | 21 | 22 | public class MaxOperator extends Operator { 23 | 24 | /* (non-Javadoc) 25 | * @see com.alibaba.wasp.util.Node#getName() 26 | */ 27 | @Override 28 | public String getName() { 29 | // TODO Auto-generated method stub 30 | return null; 31 | } 32 | 33 | @Override 34 | public Object forward(Object row) { 35 | // TODO Auto-generated method stub 36 | return null; 37 | } 38 | 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/MinOperator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.plan.operator; 21 | 22 | 23 | public class MinOperator extends Operator { 24 | 25 | @Override 26 | public String getName() { 27 | // TODO Auto-generated method stub 28 | return null; 29 | } 30 | 31 | @Override 32 | public Object forward(Object row) { 33 | // TODO Auto-generated method stub 34 | return null; 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/PageOperator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.plan.operator; 21 | 22 | 23 | public class PageOperator extends Operator { 24 | 25 | @Override 26 | public String getName() { 27 | // TODO Auto-generated method stub 28 | return null; 29 | } 30 | 31 | @Override 32 | public Object forward(Object row) { 33 | // TODO Auto-generated method stub 34 | return null; 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/SelectOperator.java: -------------------------------------------------------------------------------- 1 | package com.alibaba.wasp.plan.operator; 2 | 3 | public class SelectOperator extends Operator { 4 | 5 | @Override 6 | public String getName() { 7 | // TODO Auto-generated method stub 8 | return null; 9 | } 10 | 11 | @Override 12 | public Object forward(Object row) { 13 | // TODO Auto-generated method stub 14 | return null; 15 | } 16 | 17 | } 18 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/SumOperator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.plan.operator; 21 | 22 | 23 | public class SumOperator extends Operator { 24 | 25 | @Override 26 | public String getName() { 27 | // TODO Auto-generated method stub 28 | return null; 29 | } 30 | 31 | @Override 32 | public Object forward(Object row) { 33 | // TODO Auto-generated method stub 34 | return null; 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/operator/UpdateOperator.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.plan.operator; 21 | 22 | 23 | public class UpdateOperator extends Operator { 24 | 25 | @Override 26 | public String getName() { 27 | // TODO Auto-generated method stub 28 | return null; 29 | } 30 | 31 | @Override 32 | public Object forward(Object row) { 33 | // TODO Auto-generated method stub 34 | return null; 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/Limit.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | /** 21 | * Limit is the same with 22 | * com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlSelectQueryBlock.Limit 23 | * logic. 24 | * 25 | */ 26 | public class Limit 27 | extends 28 | com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlSelectQueryBlock.Limit { 29 | 30 | private static final long serialVersionUID = 8046011709891746659L; 31 | 32 | /** 33 | * 34 | */ 35 | public Limit() { 36 | } 37 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlAlterTableAddColumn.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlAlterTableAddColumn; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlAlterTableAddColumn extends MySqlAlterTableAddColumn { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlAlterTableAddColumn() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlAlterTableAddIndex.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlAlterTableAddIndex; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlAlterTableAddIndex extends MySqlAlterTableAddIndex { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlAlterTableAddIndex() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlAlterTableAddUnique.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlAlterTableAddUnique; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlAlterTableAddUnique extends MySqlAlterTableAddUnique { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlAlterTableAddUnique() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlAlterTableChangeColumn.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlAlterTableChangeColumn; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlAlterTableChangeColumn extends MySqlAlterTableChangeColumn { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlAlterTableChangeColumn() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlAlterTableCharacter.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlAlterTableCharacter; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlAlterTableCharacter extends MySqlAlterTableCharacter { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlAlterTableCharacter() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlAlterTableStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlAlterTableStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlAlterTableStatement extends MySqlAlterTableStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlAlterTableStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlCreateUserStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlCreateUserStatement; 21 | 22 | /** 23 | * WaspSqlCreateUserStatement is the same with MySqlCreateUserStatement logic. 24 | * 25 | */ 26 | public class WaspSqlCreateUserStatement extends MySqlCreateUserStatement { 27 | 28 | private static final long serialVersionUID = 2544962032727959307L; 29 | 30 | /** 31 | * 32 | */ 33 | public WaspSqlCreateUserStatement() { 34 | // TODO Auto-generated constructor stub 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlDeleteStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlDeleteStatement; 21 | 22 | /** 23 | * WaspSqlDeleteStatement is the same with MySqlDeleteStatement logic. 24 | * 25 | */ 26 | public class WaspSqlDeleteStatement extends MySqlDeleteStatement { 27 | 28 | private static final long serialVersionUID = 4467381250692217029L; 29 | 30 | /** 31 | * 32 | */ 33 | public WaspSqlDeleteStatement() { 34 | // TODO Auto-generated constructor stub 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlDescribeStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlDescribeStatement; 21 | 22 | /** 23 | * WaspSqlDescribeStatement is the same with MySqlDescribeStatement logic. 24 | * 25 | */ 26 | public class WaspSqlDescribeStatement extends MySqlDescribeStatement { 27 | 28 | private static final long serialVersionUID = -4239223656932655918L; 29 | 30 | /** 31 | * 32 | */ 33 | public WaspSqlDescribeStatement() { 34 | // TODO Auto-generated constructor stub 35 | } 36 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlDropTableStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlDropTableStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlDropTableStatement extends MySqlDropTableStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlDropTableStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlDropUser.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlDropUser; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlDropUser extends MySqlDropUser { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlDropUser() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlExecuteStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlExecuteStatement; 21 | 22 | /** 23 | * WaspSqlExecuteStatement is the same with WaspSqlExecuteStatement logic. 24 | * 25 | */ 26 | public class WaspSqlExecuteStatement extends MySqlExecuteStatement { 27 | 28 | private static final long serialVersionUID = 3730203435846480731L; 29 | 30 | /** 31 | * 32 | */ 33 | public WaspSqlExecuteStatement() { 34 | // TODO Auto-generated constructor stub 35 | } 36 | 37 | } 38 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlHelpStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlHelpStatement; 21 | 22 | /** 23 | * WaspSqlHelpStatement is the same with MySqlHelpStatement logic. 24 | * 25 | */ 26 | public class WaspSqlHelpStatement extends MySqlHelpStatement { 27 | 28 | private static final long serialVersionUID = 3429056627522601824L; 29 | 30 | /** 31 | * 32 | */ 33 | public WaspSqlHelpStatement() { 34 | // TODO Auto-generated constructor stub 35 | } 36 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlInsertStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlInsertStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlInsertStatement extends MySqlInsertStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlInsertStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlKillStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlKillStatement; 21 | 22 | /** 23 | * WaspSqlKillStatement is the same with MySqlKillStatement logic. 24 | * 25 | */ 26 | public class WaspSqlKillStatement extends MySqlKillStatement { 27 | 28 | private static final long serialVersionUID = 4576829343468665952L; 29 | 30 | /** 31 | * 32 | */ 33 | public WaspSqlKillStatement() { 34 | // TODO Auto-generated constructor stub 35 | } 36 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlLoadDataInFileStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlLoadDataInFileStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlLoadDataInFileStatement extends 26 | MySqlLoadDataInFileStatement { 27 | 28 | /** 29 | * 30 | */ 31 | public WaspSqlLoadDataInFileStatement() { 32 | // TODO Auto-generated constructor stub 33 | } 34 | 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlLoadXmlStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlLoadXmlStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlLoadXmlStatement extends MySqlLoadXmlStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlLoadXmlStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlSelectParser.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.parser.MySqlSelectParser; 21 | import com.alibaba.druid.sql.parser.SQLExprParser; 22 | 23 | /** 24 | * 25 | * WaspSqlSelectParser is the same with MySqlSelectParser's logic. 26 | * 27 | */ 28 | public class WaspSqlSelectParser extends MySqlSelectParser { 29 | 30 | /** 31 | * @param exprParser 32 | */ 33 | public WaspSqlSelectParser(SQLExprParser exprParser) { 34 | super(exprParser); 35 | } 36 | 37 | /** 38 | * @param sql 39 | */ 40 | public WaspSqlSelectParser(String sql) { 41 | super(sql); 42 | } 43 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowCharacterSetStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowCharacterSetStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowCharacterSetStatement extends 26 | MySqlShowCharacterSetStatement { 27 | 28 | /** 29 | * 30 | */ 31 | public WaspSqlShowCharacterSetStatement() { 32 | // TODO Auto-generated constructor stub 33 | } 34 | 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowColumnsStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowColumnsStatement; 21 | 22 | public class WaspSqlShowColumnsStatement extends MySqlShowColumnsStatement { 23 | 24 | /** 25 | * 26 | */ 27 | public WaspSqlShowColumnsStatement() { 28 | // TODO Auto-generated constructor stub 29 | } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowCreateDatabaseStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowCreateDatabaseStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowCreateDatabaseStatement extends 26 | MySqlShowCreateDatabaseStatement { 27 | 28 | /** 29 | * 30 | */ 31 | public WaspSqlShowCreateDatabaseStatement() { 32 | // TODO Auto-generated constructor stub 33 | } 34 | 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowCreateTableStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowCreateTableStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowCreateTableStatement extends 26 | MySqlShowCreateTableStatement { 27 | 28 | /** 29 | * 30 | */ 31 | public WaspSqlShowCreateTableStatement() { 32 | // TODO Auto-generated constructor stub 33 | } 34 | 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowDatabasesStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowDatabasesStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowDatabasesStatement extends MySqlShowDatabasesStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlShowDatabasesStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowErrorsStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowErrorsStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowErrorsStatement extends MySqlShowErrorsStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlShowErrorsStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowGrantsStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowGrantsStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowGrantsStatement extends MySqlShowGrantsStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlShowGrantsStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowIndexesStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowIndexesStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowIndexesStatement extends MySqlShowIndexesStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlShowIndexesStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowOpenTablesStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowOpenTablesStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowOpenTablesStatement extends 26 | MySqlShowOpenTablesStatement { 27 | 28 | /** 29 | * 30 | */ 31 | public WaspSqlShowOpenTablesStatement() { 32 | // TODO Auto-generated constructor stub 33 | } 34 | 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowProcessListStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowProcessListStatement; 21 | 22 | 23 | public class WaspSqlShowProcessListStatement extends 24 | MySqlShowProcessListStatement { 25 | 26 | /** 27 | * 28 | */ 29 | public WaspSqlShowProcessListStatement() { 30 | // TODO Auto-generated constructor stub 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowProfilesStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowProfilesStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowProfilesStatement extends MySqlShowProfilesStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlShowProfilesStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowStatusStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowStatusStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowStatusStatement extends MySqlShowStatusStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlShowStatusStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowTableStatusStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowTableStatusStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowTableStatusStatement extends 26 | MySqlShowTableStatusStatement { 27 | 28 | /** 29 | * 30 | */ 31 | public WaspSqlShowTableStatusStatement() { 32 | // TODO Auto-generated constructor stub 33 | } 34 | 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowTablesStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowTablesStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowTablesStatement extends MySqlShowTablesStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlShowTablesStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowVariantsStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowVariantsStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowVariantsStatement extends MySqlShowVariantsStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlShowVariantsStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlShowWarningsStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlShowWarningsStatement; 21 | 22 | /** 23 | * 24 | */ 25 | public class WaspSqlShowWarningsStatement extends MySqlShowWarningsStatement { 26 | 27 | /** 28 | * 29 | */ 30 | public WaspSqlShowWarningsStatement() { 31 | // TODO Auto-generated constructor stub 32 | } 33 | 34 | } 35 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlStartTransactionStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlStartTransactionStatement; 21 | 22 | /** 23 | * 24 | * WaspSqlStartTransactionStatement is the same with 25 | * MySqlStartTransactionStatement's logic. 26 | * 27 | */ 28 | public class WaspSqlStartTransactionStatement extends 29 | MySqlStartTransactionStatement { 30 | 31 | private static final long serialVersionUID = 6840070823702000873L; 32 | 33 | /** 34 | * 35 | */ 36 | public WaspSqlStartTransactionStatement() { 37 | // TODO Auto-generated constructor stub 38 | } 39 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlUnlockTablesStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlUnlockTablesStatement; 21 | 22 | /** 23 | * WaspSqlUnlockTablesStatement is the same with MySqlUnlockTablesStatement 24 | * logic. 25 | */ 26 | public class WaspSqlUnlockTablesStatement extends MySqlUnlockTablesStatement { 27 | 28 | private static final long serialVersionUID = 5373237256543549645L; 29 | 30 | /** 31 | * 32 | */ 33 | public WaspSqlUnlockTablesStatement() { 34 | // TODO Auto-generated constructor stub 35 | } 36 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/plan/parser/druid/dialect/WaspSqlUpdateStatement.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.plan.parser.druid.dialect; 19 | 20 | import com.alibaba.druid.sql.dialect.mysql.ast.statement.MySqlUpdateStatement; 21 | 22 | /** 23 | * WaspSqlUpdateStatement is the same with MySqlUpdateStatement's logic. 24 | */ 25 | public class WaspSqlUpdateStatement extends MySqlUpdateStatement { 26 | 27 | private static final long serialVersionUID = -811185497960548884L; 28 | 29 | /** 30 | * 31 | */ 32 | public WaspSqlUpdateStatement() { 33 | // TODO Auto-generated constructor stub 34 | } 35 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/security/User.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | package com.alibaba.wasp.security; 21 | 22 | /** 23 | * No-implements. 24 | * 25 | */ 26 | public class User { 27 | private String name; 28 | 29 | public String getName() { 30 | return name; 31 | } 32 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/session/SessionWithState.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with this 6 | * work for additional information regarding copyright ownership. The ASF 7 | * licenses this file to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT 15 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the 16 | * License for the specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package com.alibaba.wasp.session; 20 | 21 | 22 | /** 23 | * The base class for both remote and embedded sessions. 24 | */ 25 | public abstract class SessionWithState implements SessionInterface { 26 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/storage/StorageTableNotFoundException.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.storage; 19 | 20 | import org.apache.hadoop.hbase.DoNotRetryIOException; 21 | 22 | public class StorageTableNotFoundException extends DoNotRetryIOException { 23 | 24 | private static final long serialVersionUID = 1L; 25 | 26 | public StorageTableNotFoundException() { 27 | } 28 | 29 | /** 30 | * @param message 31 | */ 32 | public StorageTableNotFoundException(String message) { 33 | super(message); 34 | } 35 | 36 | /** 37 | * @param message 38 | * @param cause 39 | */ 40 | public StorageTableNotFoundException(String message, Throwable cause) { 41 | super(message, cause); 42 | } 43 | } -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/util/Int32.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | package com.alibaba.wasp.util; 20 | 21 | /** 22 | * Represent int32 type 23 | */ 24 | public class Int32 implements Comparable { 25 | private int value; 26 | 27 | public Int32() { 28 | } 29 | 30 | public Int32(int value) { 31 | set(value); 32 | } 33 | 34 | public void set(int value) { 35 | this.value = value; 36 | } 37 | 38 | public int get() { 39 | return value; 40 | } 41 | 42 | public int compareTo(Int32 o) { 43 | int thisValue = this.value; 44 | int thatValue = o.value; 45 | return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1)); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/util/Int64.java: -------------------------------------------------------------------------------- 1 | /** 2 | * 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | package com.alibaba.wasp.util; 20 | 21 | /** 22 | * Represent int64 type 23 | */ 24 | public class Int64 implements Comparable { 25 | private long value; 26 | 27 | public Int64() { 28 | } 29 | 30 | public Int64(int value) { 31 | set(value); 32 | } 33 | 34 | public void set(int value) { 35 | this.value = value; 36 | } 37 | 38 | public long get() { 39 | return value; 40 | } 41 | 42 | public int compareTo(Int64 o) { 43 | long thisValue = this.value; 44 | long thatValue = o.value; 45 | return (thisValue < thatValue ? -1 : (thisValue == thatValue ? 0 : 1)); 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/util/Node.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.util; 19 | 20 | import java.util.List; 21 | 22 | /** 23 | * This interface defines the functions needed by the walkers and dispatchers. 24 | * These are implemented by the node of the graph that needs to be walked. 25 | */ 26 | public interface Node { 27 | 28 | /** 29 | * Gets the vector of children nodes. This is used in the graph walker 30 | * algorithms. 31 | * 32 | * @return List 33 | */ 34 | List getChildren(); 35 | 36 | /** 37 | * Gets the name of the node. This is used in the rule dispatchers. 38 | * 39 | * @return String 40 | */ 41 | String getName(); 42 | } 43 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/util/WaspConfTool.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with this 6 | * work for additional information regarding copyright ownership. The ASF 7 | * licenses this file to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT 15 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the 16 | * License for the specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package com.alibaba.wasp.util; 20 | 21 | import com.alibaba.wasp.conf.WaspConfiguration; 22 | import org.apache.hadoop.conf.Configuration; 23 | 24 | public class WaspConfTool { 25 | public static void main(String args[]) { 26 | if (args.length < 1) { 27 | System.err.println("Usage: WaspConfTool "); 28 | System.exit(1); 29 | return; 30 | } 31 | 32 | Configuration conf = WaspConfiguration.create(); 33 | System.out.println(conf.get(args[0])); 34 | } 35 | } 36 | -------------------------------------------------------------------------------- /src/main/java/com/alibaba/wasp/util/WaspFsck.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one or more 5 | * contributor license agreements. See the NOTICE file distributed with this 6 | * work for additional information regarding copyright ownership. The ASF 7 | * licenses this file to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance with the License. 9 | * You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT 15 | * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the 16 | * License for the specific language governing permissions and limitations 17 | * under the License. 18 | */ 19 | package com.alibaba.wasp.util; 20 | 21 | public class WaspFsck { 22 | 23 | } 24 | -------------------------------------------------------------------------------- /src/main/protobuf/Admin.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | // This file contains protocol buffers that are used for Admin service. 20 | 21 | option java_package = "com.alibaba.wasp.protobuf.generated"; 22 | option java_outer_classname = "AdminProtos"; 23 | option java_generic_services = true; 24 | option java_generate_equals_and_hash = true; 25 | option optimize_for = SPEED; 26 | 27 | import "Wasp.proto"; 28 | 29 | message UUID { 30 | required uint64 leastSigBits = 1; 31 | required uint64 mostSigBits = 2; 32 | } 33 | 34 | service AdminService { 35 | 36 | } 37 | -------------------------------------------------------------------------------- /src/main/protobuf/ClusterID.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | // This file contains protocol buffers that are shared throughout wasp 20 | 21 | option java_package = "com.alibaba.wasp.protobuf.generated"; 22 | option java_outer_classname = "ClusterIdProtos"; 23 | option java_generate_equals_and_hash = true; 24 | option optimize_for = SPEED; 25 | 26 | /** 27 | * Content of the '/wasp/waspid', cluster id, znode. 28 | * Also cluster of the ${WASP_ROOTDIR}/wasp.id file. 29 | */ 30 | message ClusterId { 31 | // This is the cluster id, a uuid as a String 32 | required string clusterId = 1; 33 | } 34 | -------------------------------------------------------------------------------- /src/main/protobuf/FS.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | // This file contains protocol buffers that are written into the filesystem 20 | 21 | option java_package = "com.alibaba.wasp.protobuf.generated"; 22 | option java_outer_classname = "FSProtos"; 23 | option java_generate_equals_and_hash = true; 24 | option optimize_for = SPEED; 25 | 26 | /** 27 | * Reference file content used when we split an hfile under a entityGroup. 28 | */ 29 | message Reference { 30 | required bytes splitkey = 1; 31 | enum Range { 32 | TOP = 0; 33 | BOTTOM = 1; 34 | } 35 | required Range range = 2; 36 | } 37 | 38 | -------------------------------------------------------------------------------- /src/main/protobuf/LoadBalancer.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | // This file contains protocol buffers to represent the state of the load balancer. 20 | 21 | option java_package = "com.alibaba.wasp.protobuf.generated"; 22 | option java_outer_classname = "LoadBalancerProtos"; 23 | option java_generate_equals_and_hash = true; 24 | option optimize_for = SPEED; 25 | 26 | message LoadBalancerState { 27 | optional bool balancerOn = 1; 28 | } 29 | -------------------------------------------------------------------------------- /src/main/protobuf/Master.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | // This file contains protocol buffers that are used for protocols implemented by the master. 20 | 21 | option java_package = "com.alibaba.wasp.protobuf.generated"; 22 | option java_outer_classname = "MasterProtos"; 23 | option java_generic_services = true; 24 | option java_generate_equals_and_hash = true; 25 | option optimize_for = SPEED; 26 | 27 | message IsMasterRunningRequest { 28 | } 29 | 30 | message IsMasterRunningResponse { 31 | required bool isMasterRunning = 1; 32 | } 33 | 34 | service MasterService { 35 | /** return true if master is available */ 36 | rpc isMasterRunning(IsMasterRunningRequest) 37 | returns(IsMasterRunningResponse); 38 | } 39 | -------------------------------------------------------------------------------- /src/main/protobuf/Tracing.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | option java_package = "com.alibaba.wasp.protobuf.generated"; 19 | option java_outer_classname = "Tracing"; 20 | option java_generate_equals_and_hash = true; 21 | option optimize_for = SPEED; 22 | 23 | //Used to pass through the information necessary to continue 24 | //a trace after an RPC is made. All we need is the traceid 25 | //(so we know the overarching trace this message is a part of), and 26 | //the id of the current span when this message was sent, so we know 27 | //what span caused the new span we will create when this message is received. 28 | message RPCTInfo { 29 | optional int64 traceId = 1; 30 | optional int64 parentId = 2; 31 | } 32 | -------------------------------------------------------------------------------- /src/main/resources/wasp-webapps/fserver/fserver.jsp: -------------------------------------------------------------------------------- 1 | <%-- 2 | /** 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | --%> 20 | 21 | -------------------------------------------------------------------------------- /src/main/resources/wasp-webapps/fserver/index.html: -------------------------------------------------------------------------------- 1 | 20 | 21 | -------------------------------------------------------------------------------- /src/main/resources/wasp-webapps/master/index.html: -------------------------------------------------------------------------------- 1 | 20 | 21 | -------------------------------------------------------------------------------- /src/main/resources/wasp-webapps/master/master.jsp: -------------------------------------------------------------------------------- 1 | <%-- 2 | /** 3 | * Licensed to the Apache Software Foundation (ASF) under one 4 | * or more contributor license agreements. See the NOTICE file 5 | * distributed with this work for additional information 6 | * regarding copyright ownership. The ASF licenses this file 7 | * to you under the Apache License, Version 2.0 (the 8 | * "License"); you may not use this file except in compliance 9 | * with the License. You may obtain a copy of the License at 10 | * 11 | * http://www.apache.org/licenses/LICENSE-2.0 12 | * 13 | * Unless required by applicable law or agreed to in writing, software 14 | * distributed under the License is distributed on an "AS IS" BASIS, 15 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 16 | * See the License for the specific language governing permissions and 17 | * limitations under the License. 18 | */ 19 | --%> 20 | 21 | -------------------------------------------------------------------------------- /src/main/resources/wasp-webapps/static/css/wasp.css: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | body { padding-top: 60px; } 19 | .logo img { float: right; } 20 | .inner_header { margin-bottom: 1em;} 21 | section { margin-bottom: 3em; } 22 | -------------------------------------------------------------------------------- /src/main/resources/wasp-webapps/static/hbase_logo.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/alibaba/wasp/b2593d8e4b31ca6da0cd2f3e18356338d9b6dace/src/main/resources/wasp-webapps/static/hbase_logo.png -------------------------------------------------------------------------------- /src/main/resources/wasp-webapps/static/hbase_logo_med.gif: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/alibaba/wasp/b2593d8e4b31ca6da0cd2f3e18356338d9b6dace/src/main/resources/wasp-webapps/static/hbase_logo_med.gif -------------------------------------------------------------------------------- /src/main/resources/wasp-webapps/static/hbase_logo_small.png: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/alibaba/wasp/b2593d8e4b31ca6da0cd2f3e18356338d9b6dace/src/main/resources/wasp-webapps/static/hbase_logo_small.png -------------------------------------------------------------------------------- /src/main/resources/wasp-webapps/static/js/tab.js: -------------------------------------------------------------------------------- 1 | /* 2 | * Copyright The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | $(document).ready( 22 | function(){ 23 | var prefix = "tab_"; 24 | $('.tabbable .nav-pills a').click(function (e) { 25 | e.preventDefault(); 26 | location.hash = $(e.target).attr('href').substr(1).replace(prefix, ""); 27 | $(this).tab('show'); 28 | }); 29 | 30 | if (location.hash !== '') { 31 | var tabItem = $('a[href="' + location.hash.replace("#", "#"+prefix) + '"]'); 32 | tabItem.tab('show'); 33 | $(document).scrollTop(0); 34 | return false; 35 | } 36 | return true; 37 | } 38 | ); -------------------------------------------------------------------------------- /src/main/resources/wasp-webapps/static/wasp_logo.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/alibaba/wasp/b2593d8e4b31ca6da0cd2f3e18356338d9b6dace/src/main/resources/wasp-webapps/static/wasp_logo.jpg -------------------------------------------------------------------------------- /src/main/resources/wasp-webapps/static/wasp_logo_med.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/alibaba/wasp/b2593d8e4b31ca6da0cd2f3e18356338d9b6dace/src/main/resources/wasp-webapps/static/wasp_logo_med.jpg -------------------------------------------------------------------------------- /src/main/resources/wasp-webapps/static/wasp_logo_small.jpg: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/alibaba/wasp/b2593d8e4b31ca6da0cd2f3e18356338d9b6dace/src/main/resources/wasp-webapps/static/wasp_logo_small.jpg -------------------------------------------------------------------------------- /src/test/java/com/alibaba/wasp/fserver/redo/TestTransaction.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.fserver.redo; 19 | 20 | import com.alibaba.wasp.protobuf.generated.MetaProtos; 21 | import junit.framework.Assert; 22 | import org.junit.Test; 23 | 24 | public class TestTransaction { 25 | 26 | @Test 27 | public void testTransaction() { 28 | Transaction t = new Transaction(); 29 | MetaProtos.TransactionProto tProto = Transaction.conver(t); 30 | Transaction t2 = Transaction.convert(tProto.toByteArray()); 31 | Assert.assertEquals(t.getKeyStr(), t2.getKeyStr()); 32 | } 33 | } -------------------------------------------------------------------------------- /src/test/java/com/alibaba/wasp/jdbc/TestResultVerifier.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.jdbc; 19 | 20 | import java.lang.reflect.Method; 21 | 22 | /** 23 | * This handler is called after a method returned. 24 | */ 25 | public interface TestResultVerifier { 26 | 27 | /** 28 | * Verify the result or exception. 29 | * 30 | * @param returnValue the returned value or null 31 | * @param t the exception / error or null if the method returned normally 32 | * @param m the method or null if unknown 33 | * @param args the arguments or null if unknown 34 | * @return true if the method should be called again 35 | */ 36 | boolean verify(Object returnValue, Throwable t, Method m, Object... args); 37 | 38 | } 39 | -------------------------------------------------------------------------------- /src/test/java/com/alibaba/wasp/meta/TestFTable.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | package com.alibaba.wasp.meta; 19 | 20 | import com.alibaba.wasp.protobuf.RequestConverter; 21 | import com.alibaba.wasp.protobuf.generated.MasterAdminProtos; 22 | import com.google.protobuf.ServiceException; 23 | import org.junit.Assert; 24 | import org.junit.Test; 25 | 26 | public class TestFTable { 27 | 28 | @Test 29 | public void testConvert() throws ServiceException { 30 | FTable desc = FMetaTestUtil.makeTable("test"); 31 | MasterAdminProtos.CreateTableRequest request = RequestConverter.buildCreateTableRequest( 32 | desc, null); 33 | Assert.assertNotNull(request.getTableSchema()); 34 | FTable.convert(request.getTableSchema()); 35 | } 36 | } -------------------------------------------------------------------------------- /src/test/java/com/alibaba/wasp/plan/action/TestColumnStruct.java: -------------------------------------------------------------------------------- 1 | /** 2 | * Copyright 2010 The Apache Software Foundation 3 | * 4 | * Licensed to the Apache Software Foundation (ASF) under one 5 | * or more contributor license agreements. See the NOTICE file 6 | * distributed with this work for additional information 7 | * regarding copyright ownership. The ASF licenses this file 8 | * to you under the Apache License, Version 2.0 (the 9 | * "License"); you may not use this file except in compliance 10 | * with the License. You may obtain a copy of the License at 11 | * 12 | * http://www.apache.org/licenses/LICENSE-2.0 13 | * 14 | * Unless required by applicable law or agreed to in writing, software 15 | * distributed under the License is distributed on an "AS IS" BASIS, 16 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 17 | * See the License for the specific language governing permissions and 18 | * limitations under the License. 19 | */ 20 | 21 | package com.alibaba.wasp.plan.action; 22 | 23 | import com.alibaba.wasp.protobuf.generated.MetaProtos; 24 | import org.junit.Assert; 25 | import org.junit.Test; 26 | 27 | public class TestColumnStruct { 28 | 29 | @Test 30 | public void testColumnAction() { 31 | ColumnStruct columnAction = ActionTestUtil.makeTestColumnAction(); 32 | MetaProtos.ColumnStructProto pb = ColumnStruct.convert(columnAction); 33 | 34 | ColumnStruct derColumnAction = ColumnStruct.convert(pb); 35 | Assert.assertTrue(columnAction.equals(derColumnAction)); 36 | } 37 | } -------------------------------------------------------------------------------- /src/test/protobuf/test.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | 19 | option java_package = "com.alibaba.wasp.ipc.protobuf.generated"; 20 | option java_outer_classname = "TestProtos"; 21 | option java_generate_equals_and_hash = true; 22 | 23 | message EmptyRequestProto { 24 | } 25 | 26 | message EmptyResponseProto { 27 | } 28 | 29 | message EchoRequestProto { 30 | required string message = 1; 31 | } 32 | 33 | message EchoResponseProto { 34 | required string message = 1; 35 | } -------------------------------------------------------------------------------- /src/test/protobuf/test_rpc_service.proto: -------------------------------------------------------------------------------- 1 | /** 2 | * Licensed to the Apache Software Foundation (ASF) under one 3 | * or more contributor license agreements. See the NOTICE file 4 | * distributed with this work for additional information 5 | * regarding copyright ownership. The ASF licenses this file 6 | * to you under the Apache License, Version 2.0 (the 7 | * "License"); you may not use this file except in compliance 8 | * with the License. You may obtain a copy of the License at 9 | * 10 | * http://www.apache.org/licenses/LICENSE-2.0 11 | * 12 | * Unless required by applicable law or agreed to in writing, software 13 | * distributed under the License is distributed on an "AS IS" BASIS, 14 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 15 | * See the License for the specific language governing permissions and 16 | * limitations under the License. 17 | */ 18 | option java_package = "com.alibaba.wasp.ipc.protobuf.generated"; 19 | option java_outer_classname = "TestRpcServiceProtos"; 20 | option java_generic_services = true; 21 | option java_generate_equals_and_hash = true; 22 | 23 | import "test.proto"; 24 | 25 | 26 | /** 27 | * A protobuf service for use in tests 28 | */ 29 | service TestProtobufRpcProto { 30 | rpc ping(EmptyRequestProto) returns (EmptyResponseProto); 31 | rpc echo(EchoRequestProto) returns (EchoResponseProto); 32 | rpc error(EmptyRequestProto) returns (EmptyResponseProto); 33 | } 34 | --------------------------------------------------------------------------------