HHH-17023 Implements AltibaseDialect

This commit is contained in:
yjpark 2023-11-14 11:24:17 +09:00 committed by Christian Beikov
parent e6d5c72ec6
commit 46b064633b
59 changed files with 1476 additions and 3 deletions

View File

@ -67,6 +67,8 @@ elif [ "$RDBMS" == "hana_cloud" ]; then
goal="-Pdb=hana_cloud"
elif [ "$RDBMS" == "cockroachdb" ]; then
goal="-Pdb=cockroachdb"
elif [ "$RDBMS" == "altibase" ]; then
goal="-Pdb=altibase"
fi
# Only run checkstyle in the H2 build,

View File

@ -0,0 +1,7 @@
/*
* Hibernate, Relational Persistence for Idiomatic Java
*
* License: GNU Lesser General Public License (LGPL), version 2.1 or later.
* See the lgpl.txt file in the root directory or <http://www.gnu.org/licenses/lgpl-2.1.html>.
*/
jdbcDependency 'com.altibase:altibase-jdbc:7.3.0.0.2'

View File

@ -0,0 +1,25 @@
#
# Hibernate, Relational Persistence for Idiomatic Java
#
# License: GNU Lesser General Public License (LGPL), version 2.1 or later.
# See the lgpl.txt file in the root directory or <http://www.gnu.org/licenses/lgpl-2.1.html>.
#
hibernate.dialect org.hibernate.community.dialect.AltibaseDialect
hibernate.connection.driver_class Altibase.jdbc.driver.AltibaseDriver
hibernate.connection.url jdbc:Altibase://localhost:20300/mydb
hibernate.connection.username sys
hibernate.connection.password manager
hibernate.connection.pool_size 5
hibernate.show_sql false
hibernate.format_sql true
hibernate.max_fetch_depth 5
hibernate.cache.region_prefix hibernate.test
hibernate.cache.region.factory_class org.hibernate.testing.cache.CachingRegionFactory
javax.persistence.validation.mode=NONE
hibernate.service.allow_crawling=false
hibernate.session.events.log=true

View File

@ -292,6 +292,14 @@ ext {
'jdbc.url' : 'jdbc:firebirdsql://' + dbHost +'/hibernate_orm_test?charSet=utf-8;TRANSACTION_READ_COMMITTED=read_committed,rec_version,wait,lock_timeout=5',
'connection.init_sql' : ''
],
altibase : [
'db.dialect' : 'org.hibernate.community.dialect.AltibaseDialect',
'jdbc.driver': 'Altibase.jdbc.driver.AltibaseDriver',
'jdbc.user' : 'sys',
'jdbc.pass' : 'manager',
'connection.init_sql' : '',
'jdbc.url' : 'jdbc:Altibase://' + dbHost + ':20300/mydb?force_clob_bind=true'
],
]
}

View File

@ -128,6 +128,9 @@ dependencies {
testRuntimeOnly dbLibs.oracleXml
testRuntimeOnly dbLibs.oracleXmlParser
}
else if ( db.startsWith( 'altibase' ) ) {
testRuntimeOnly dbLibs.altibase
}
annotationProcessor libs.loggingProcessor
annotationProcessor libs.logging

View File

@ -6,6 +6,7 @@
*/
package org.hibernate.test.agroal;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.TiDBDialect;
import org.hibernate.engine.jdbc.connections.spi.ConnectionProvider;
import org.hibernate.agroal.internal.AgroalConnectionProvider;
@ -17,6 +18,7 @@ import org.hibernate.testing.common.connections.BaseTransactionIsolationConfigTe
* @author Steve Ebersole
*/
@SkipForDialect(value = TiDBDialect.class, comment = "Doesn't support SERIALIZABLE isolation")
@SkipForDialect(value = AltibaseDialect.class, comment = "Altibase cannot change isolation level in autocommit mode")
public class AgroalTransactionIsolationConfigTest extends BaseTransactionIsolationConfigTest {
@Override
protected ConnectionProvider getConnectionProviderUnderTest() {

View File

@ -9,6 +9,7 @@ package org.hibernate.test.c3p0;
import org.hibernate.boot.registry.StandardServiceRegistry;
import org.hibernate.boot.registry.StandardServiceRegistryBuilder;
import org.hibernate.c3p0.internal.C3P0ConnectionProvider;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.TiDBDialect;
import org.hibernate.engine.jdbc.connections.spi.ConnectionProvider;
import org.hibernate.service.spi.ServiceRegistryImplementor;
@ -21,6 +22,7 @@ import org.junit.Before;
* @author Steve Ebersole
*/
@SkipForDialect(value = TiDBDialect.class, comment = "Doesn't support SERIALIZABLE isolation")
@SkipForDialect(value = AltibaseDialect.class, comment = "Altibase cannot change isolation level in autocommit mode")
public class C3p0TransactionIsolationConfigTest extends BaseTransactionIsolationConfigTest {
private StandardServiceRegistry ssr;

View File

@ -0,0 +1,653 @@
/*
* Hibernate, Relational Persistence for Idiomatic Java
*
* License: GNU Lesser General Public License (LGPL), version 2.1 or later.
* See the lgpl.txt file in the root directory or <http://www.gnu.org/licenses/lgpl-2.1.html>.
*/
package org.hibernate.community.dialect;
import java.sql.*;
import java.time.temporal.TemporalAccessor;
import java.util.Date;
import java.util.TimeZone;
import jakarta.persistence.TemporalType;
import org.hibernate.boot.model.FunctionContributions;
import org.hibernate.boot.model.TypeContributions;
import org.hibernate.community.dialect.sequence.*;
import org.hibernate.dialect.*;
import org.hibernate.dialect.function.CommonFunctionFactory;
import org.hibernate.community.dialect.pagination.AltibaseLimitHandler;
import org.hibernate.dialect.function.OracleTruncFunction;
import org.hibernate.dialect.pagination.LimitHandler;
import org.hibernate.dialect.sequence.SequenceSupport;
import org.hibernate.engine.jdbc.dialect.spi.DialectResolutionInfo;
import org.hibernate.engine.jdbc.env.spi.IdentifierHelper;
import org.hibernate.engine.jdbc.env.spi.IdentifierHelperBuilder;
import org.hibernate.engine.jdbc.env.spi.NameQualifierSupport;
import org.hibernate.engine.spi.SessionFactoryImplementor;
import org.hibernate.exception.ConstraintViolationException;
import org.hibernate.exception.LockTimeoutException;
import org.hibernate.exception.spi.SQLExceptionConversionDelegate;
import org.hibernate.internal.util.JdbcExceptionHelper;
import org.hibernate.query.sqm.*;
import org.hibernate.query.sqm.produce.function.FunctionParameterType;
import org.hibernate.service.ServiceRegistry;
import org.hibernate.sql.ast.SqlAstTranslator;
import org.hibernate.sql.ast.SqlAstTranslatorFactory;
import org.hibernate.sql.ast.spi.SqlAppender;
import org.hibernate.sql.ast.spi.StandardSqlAstTranslatorFactory;
import org.hibernate.sql.ast.tree.Statement;
import org.hibernate.sql.exec.spi.JdbcOperation;
import org.hibernate.tool.schema.extract.spi.SequenceInformationExtractor;
import org.hibernate.type.StandardBasicTypes;
import org.hibernate.type.descriptor.java.PrimitiveByteArrayJavaType;
import org.hibernate.type.descriptor.sql.internal.CapacityDependentDdlType;
import org.hibernate.type.descriptor.sql.spi.DdlTypeRegistry;
import org.hibernate.type.spi.TypeConfiguration;
import static org.hibernate.type.SqlTypes.*;
import static org.hibernate.type.descriptor.DateTimeUtils.JDBC_ESCAPE_END;
import static org.hibernate.type.descriptor.DateTimeUtils.JDBC_ESCAPE_START_TIMESTAMP;
import static org.hibernate.type.descriptor.DateTimeUtils.appendAsTimestampWithMicros;
/**
* An SQL dialect for Altibase 7.3 and above.
*
* @author Geoffrey Park
*/
public class AltibaseDialect extends Dialect {
private static final DatabaseVersion MINIMUM_VERSION = DatabaseVersion.make( 7, 3 );
@SuppressWarnings("unused")
public AltibaseDialect() {
this( MINIMUM_VERSION );
}
public AltibaseDialect(DialectResolutionInfo info) {
this( info.makeCopy() );
registerKeywords( info );
}
public AltibaseDialect(DatabaseVersion version) {
super( version );
}
@Override
protected String columnType(int sqlTypeCode) {
switch ( sqlTypeCode ) {
case BOOLEAN:
return "char(1)";
case FLOAT:
case DOUBLE:
return "double";
case TINYINT:
return "smallint";
case TIME:
case TIMESTAMP:
case TIME_WITH_TIMEZONE:
case TIMESTAMP_WITH_TIMEZONE:
return "date";
case BINARY:
return "byte($l)";
case VARBINARY:
return "varbyte($l)";
case LONGVARBINARY:
return "blob";
case BIT:
return "varbit($l)";
case LONGVARCHAR:
case NCLOB:
return "clob";
default:
return super.columnType( sqlTypeCode );
}
}
@Override
protected void registerColumnTypes(TypeContributions typeContributions, ServiceRegistry serviceRegistry) {
super.registerColumnTypes( typeContributions, serviceRegistry );
final DdlTypeRegistry ddlTypeRegistry = typeContributions.getTypeConfiguration().getDdlTypeRegistry();
ddlTypeRegistry.addDescriptor(
CapacityDependentDdlType.builder( BINARY, columnType( LONGVARBINARY ), this )
.withTypeCapacity( getMaxVarbinaryLength(), columnType( BINARY ) )
.build()
);
ddlTypeRegistry.addDescriptor(
CapacityDependentDdlType.builder( BIT, columnType( LONGVARBINARY ), this )
.withTypeCapacity( 64000, columnType( BIT ) )
.build()
);
}
@Override
public int getMaxVarcharLength() {
return 32_000;
}
@Override
public int getMaxVarbinaryLength() {
return 32_000;
}
@Override
public int getDefaultStatementBatchSize() {
return 15;
}
@Override
public String trimPattern(TrimSpec specification, char character) {
switch ( specification ) {
case BOTH:
return character == ' '
? "trim(?1)"
: "trim(?1, '" + character + "')";
case LEADING:
return character == ' '
? "ltrim(?1)"
: "ltrim(?1,'" + character + "')";
case TRAILING:
return character == ' '
? "rtrim(?1)"
: "rtrim(?1,'" + character + "')";
}
return super.trimPattern( specification, character );
}
@Override
public int getPreferredSqlTypeCodeForBoolean() {
return Types.BIT;
}
@Override
public void initializeFunctionRegistry(FunctionContributions functionContributions) {
super.initializeFunctionRegistry(functionContributions);
final TypeConfiguration typeConfiguration = functionContributions.getTypeConfiguration();
functionContributions.getFunctionRegistry().registerBinaryTernaryPattern(
"locate",
typeConfiguration.getBasicTypeRegistry().resolve( StandardBasicTypes.INTEGER ),
"instr(?2,?1)",
"instr(?2,?1,?3)",
FunctionParameterType.STRING, FunctionParameterType.STRING, FunctionParameterType.INTEGER,
typeConfiguration
).setArgumentListSignature("(pattern, string[, start])");
CommonFunctionFactory functionFactory = new CommonFunctionFactory(functionContributions);
functionFactory.ceiling_ceil();
functionFactory.trim2();
functionFactory.stddev();
functionFactory.variance();
functionFactory.char_chr();
functionFactory.concat_pipeOperator();
functionFactory.coalesce();
functionFactory.initcap();
functionFactory.repeat_rpad();
functionFactory.radians_acos();
functionFactory.degrees_acos();
functionFactory.ascii();
functionFactory.toCharNumberDateTimestamp();
functionFactory.lastDay();
functionFactory.sysdate();
functionFactory.rownum();
functionFactory.instr();
functionFactory.substr();
functionFactory.cosh();
functionFactory.sinh();
functionFactory.tanh();
functionFactory.log();
functionFactory.log10_log();
functionFactory.substring_substr();
functionFactory.leftRight_substr();
functionFactory.translate();
functionFactory.addMonths();
functionFactory.listagg( null );
functionFactory.monthsBetween();
functionFactory.windowFunctions();
functionFactory.hypotheticalOrderedSetAggregates();
functionFactory.bitLength_pattern( "bit_length(?1)", "lengthb(?1)*8" );
functionFactory.octetLength_pattern( "octet_length(?1)", "lengthb(?1)" );
functionContributions.getFunctionRegistry().register(
"trunc",
new OracleTruncFunction( functionContributions.getTypeConfiguration() )
);
functionContributions.getFunctionRegistry().registerAlternateKey( "truncate", "trunc" );
}
@Override
public String currentDate() {
return currentTimestamp();
}
@Override
public String currentTime() {
return currentTimestamp();
}
@Override
public String currentTimestamp() {
return "sysdate";
}
@Override
public String currentLocalTime() {
return currentTimestamp();
}
@Override
public String currentLocalTimestamp() {
return currentTimestamp();
}
@Override
public String currentTimestampWithTimeZone() {
return currentTimestamp();
}
@Override
public SqlAstTranslatorFactory getSqlAstTranslatorFactory() {
return new StandardSqlAstTranslatorFactory() {
@Override
protected <T extends JdbcOperation> SqlAstTranslator<T> buildTranslator(
SessionFactoryImplementor sessionFactory, Statement statement) {
return new AltibaseSqlAstTranslator<>( sessionFactory, statement );
}
};
}
@Override
public long getFractionalSecondPrecisionInNanos() {
return 1_000; //microseconds
}
/**
* Altibase supports a limited list of temporal fields in the
* extract() function, but we can emulate some of them by
* using to_char() with a format string instead of extract().
* Thus, the additional supported fields are
* {@link TemporalUnit#DAY_OF_YEAR},
* {@link TemporalUnit#DAY_OF_MONTH},
* {@link TemporalUnit#DAY_OF_YEAR},
* and {@link TemporalUnit#WEEK}.
*/
@Override
public String extractPattern(TemporalUnit unit) {
switch (unit) {
case DAY_OF_WEEK:
return "extract(?2, 'DAYOFWEEK')";
case DAY_OF_MONTH:
return "extract(?2, 'DAY')";
case DAY_OF_YEAR:
return "extract(?2,'DAYOFYEAR')";
case WEEK:
return "to_number(to_char(?2,'IW'))"; //the ISO week number
case WEEK_OF_YEAR:
return "extract(?2, 'WEEK')";
case EPOCH:
return timestampdiffPattern( TemporalUnit.SECOND, TemporalType.TIMESTAMP, TemporalType.TIMESTAMP )
.replace( "?2", "TO_DATE('1970-01-01 00:00:00','YYYY-MM-DD HH24:MI:SS')" )
.replace( "?3", "?2" );
case QUARTER:
return "extract(?2, 'QUARTER')";
default:
return super.extractPattern( unit );
}
}
@Override
public String timestampaddPattern(TemporalUnit unit, TemporalType temporalType, IntervalType intervalType) {
switch (unit) {
case NANOSECOND:
return "timestampadd(MICROSECOND,(?2)/1e3,?3)";
case NATIVE:
return "timestampadd(MICROSECOND, ?2, ?3)";
default:
return "timestampadd(?1, ?2, ?3)";
}
}
@Override
public String timestampdiffPattern(TemporalUnit unit, TemporalType fromTemporalType, TemporalType toTemporalType) {
switch (unit) {
case SECOND:
return "datediff(?2, ?3, 'SECOND')";
case NANOSECOND:
return "datediff(?2, ?3, 'MICROSECOND')*1e3";
case NATIVE:
return "datediff(?2, ?3, 'MICROSECOND')";
default:
return "datediff(?2, ?3, '?1')";
}
}
@Override
public void appendBinaryLiteral(SqlAppender appender, byte[] bytes) {
appender.appendSql( "VARBYTE'" );
PrimitiveByteArrayJavaType.INSTANCE.appendString( appender, bytes );
appender.appendSql( '\'' );
}
@Override
public void appendDatetimeFormat(SqlAppender appender, String format) {
appender.appendSql( OracleDialect.datetimeFormat( format, false, false ).result() );
}
@Override
public String castPattern(CastType from, CastType to) {
String result;
switch ( to ) {
case INTEGER:
case LONG:
result = BooleanDecoder.toInteger( from );
if ( result != null ) {
return result;
}
break;
case INTEGER_BOOLEAN:
result = BooleanDecoder.toIntegerBoolean( from );
if ( result != null ) {
return result;
}
break;
case YN_BOOLEAN:
result = BooleanDecoder.toYesNoBoolean( from );
if ( result != null ) {
return result;
}
break;
case BOOLEAN:
case TF_BOOLEAN:
result = BooleanDecoder.toTrueFalseBoolean( from );
if ( result != null ) {
return result;
}
break;
case STRING:
switch ( from ) {
case INTEGER_BOOLEAN:
case TF_BOOLEAN:
case YN_BOOLEAN:
return BooleanDecoder.toString( from );
case DATE:
return "to_char(?1,'YYYY-MM-DD')";
case TIME:
return "to_char(?1,'HH24:MI:SS')";
case TIMESTAMP:
case OFFSET_TIMESTAMP:
case ZONE_TIMESTAMP:
return "to_char(?1,'YYYY-MM-DD HH24:MI:SS.FF6')";
}
break;
case CLOB:
// Altibase doesn't support cast to clob
return "cast(?1 as varchar(32000))";
case DATE:
if ( from == CastType.STRING ) {
return "to_date(?1,'YYYY-MM-DD')";
}
break;
case TIME:
if ( from == CastType.STRING ) {
return "to_date(?1,'HH24:MI:SS')";
}
break;
case TIMESTAMP:
case OFFSET_TIMESTAMP:
case ZONE_TIMESTAMP:
if ( from == CastType.STRING ) {
return "to_date(?1,'YYYY-MM-DD HH24:MI:SS.FF6')";
}
break;
}
return super.castPattern(from, to);
}
@Override
public void appendDateTimeLiteral(
SqlAppender appender,
TemporalAccessor temporalAccessor,
TemporalType precision,
TimeZone jdbcTimeZone) {
if (precision == TemporalType.TIMESTAMP) {
appender.appendSql(JDBC_ESCAPE_START_TIMESTAMP);
appendAsTimestampWithMicros(appender, temporalAccessor, supportsTemporalLiteralOffset(), jdbcTimeZone);
appender.appendSql(JDBC_ESCAPE_END);
return;
}
super.appendDateTimeLiteral(appender, temporalAccessor, precision, jdbcTimeZone);
}
@Override
public void appendDateTimeLiteral(SqlAppender appender, Date date, TemporalType precision, TimeZone jdbcTimeZone) {
if (precision == TemporalType.TIMESTAMP) {
appender.appendSql(JDBC_ESCAPE_START_TIMESTAMP);
appendAsTimestampWithMicros( appender, date, jdbcTimeZone );
appender.appendSql(JDBC_ESCAPE_END);
return;
}
super.appendDateTimeLiteral(appender, date, precision, jdbcTimeZone);
}
@Override
public String translateDurationField(TemporalUnit unit) {
//use microsecond as the "native" precision
if ( unit == TemporalUnit.NATIVE ) {
return "microsecond";
}
return super.translateDurationField( unit );
}
@Override
public NullOrdering getNullOrdering() {
return NullOrdering.LAST;
}
@Override
public String getAddColumnString() {
return "add column (";
}
@Override
public String getAddColumnSuffixString() {
return ")";
}
@Override
public int getMaxIdentifierLength() {
return 40;
}
@Override
public IdentifierHelper buildIdentifierHelper(
IdentifierHelperBuilder builder,
DatabaseMetaData dbMetaData) throws SQLException {
// Any use of keywords as identifiers will result in syntax error, so enable auto quote always
builder.setAutoQuoteKeywords( true );
builder.setAutoQuoteInitialUnderscore( false );
builder.applyReservedWords( dbMetaData );
return super.buildIdentifierHelper( builder, dbMetaData );
}
@Override
public boolean canCreateSchema() {
return false;
}
@Override
public NameQualifierSupport getNameQualifierSupport() {
return NameQualifierSupport.SCHEMA;
}
@Override
public String[] getCreateSchemaCommand(String schemaName) {
throw new UnsupportedOperationException( "No create schema syntax supported by " + getClass().getName() );
}
@Override
public String[] getDropSchemaCommand(String schemaName) {
throw new UnsupportedOperationException( "No drop schema syntax supported by " + getClass().getName() );
}
@Override
public boolean qualifyIndexName() {
return false;
}
@Override
public boolean supportsInsertReturningGeneratedKeys() {
return true;
}
@Override
public boolean supportsTruncateWithCast(){
return false;
}
@Override
public boolean supportsCommentOn() {
return true;
}
@Override
public boolean supportsUnboundedLobLocatorMaterialization() {
return false;
}
@Override
public boolean supportsTupleDistinctCounts() {
return false;
}
@Override
public boolean supportsExistsInSelect() {
return false;
}
@Override
public boolean supportsPartitionBy() {
return true;
}
@Override
public SequenceSupport getSequenceSupport() {
return AltibaseSequenceSupport.INSTANCE;
}
@Override
public String getQuerySequencesString() {
return "SELECT a.user_name USER_NAME, b.table_name SEQUENCE_NAME, c.current_seq CURRENT_VALUE, "
+ "c.start_seq START_VALUE, c.min_seq MIN_VALUE, c.max_seq MAX_VALUE, c.increment_seq INCREMENT_BY, "
+ "c.flag CYCLE_, c.sync_interval CACHE_SIZE "
+ "FROM system_.sys_users_ a, system_.sys_tables_ b, x$seq c "
+ "WHERE a.user_id = b.user_id AND b.table_oid = c.seq_oid AND a.user_name <> 'SYSTEM_' AND b.table_type = 'S' "
+ "ORDER BY 1,2";
}
@Override
public SequenceInformationExtractor getSequenceInformationExtractor() {
return SequenceInformationExtractorAltibaseDatabaseImpl.INSTANCE;
}
@Override
public LimitHandler getLimitHandler() {
return AltibaseLimitHandler.INSTANCE;
}
@Override
public boolean supportsCurrentTimestampSelection() {
return true;
}
@Override
public String getCurrentTimestampSelectString() {
return "select sysdate from dual";
}
@Override
public boolean isCurrentTimestampSelectStringCallable() {
return false;
}
@Override
public String getCascadeConstraintsString() {
return " cascade constraints";
}
@Override
public boolean supportsValuesListForInsert() {
return false;
}
@Override
public boolean supportsOrderByInSubquery() {
return false;
}
@Override
public boolean supportsWindowFunctions() {
return true;
}
@Override
public NationalizationSupport getNationalizationSupport() {
return NationalizationSupport.IMPLICIT;
}
@Override
public boolean supportsTemporaryTables() {
return false;
}
@Override
public boolean supportsTemporaryTablePrimaryKey() {
return false;
}
@Override
protected boolean supportsPredicateAsExpression() {
return false;
}
@Override
public String translateExtractField(TemporalUnit unit) {
switch ( unit ) {
case DAY_OF_MONTH: return "day";
case DAY_OF_YEAR: return "dayofyear";
case DAY_OF_WEEK: return "dayofweek";
default: return super.translateExtractField( unit );
}
}
@Override
public SQLExceptionConversionDelegate buildSQLExceptionConversionDelegate() {
return (sqlException, message, sql) -> {
final int errorCode = JdbcExceptionHelper.extractErrorCode(sqlException );
if ( errorCode == 334393 || errorCode == 4164) {
// 334393 - response timeout , 4164 - query timeout.
return new LockTimeoutException(message, sqlException, sql );
}
// 200820 - Cannot insert NULL or update to NULL
// 69720 - Unique constraint violated
// 200823 - foreign key constraint violation
// 200822 - failed on update or delete by foreign key constraint violation
if ( errorCode == 200820 || errorCode == 69720 || errorCode == 200823 || errorCode == 200822 ) {
final String constraintName = getViolatedConstraintNameExtractor().extractConstraintName( sqlException );
return new ConstraintViolationException(message, sqlException, sql, constraintName );
}
return null;
};
}
}

View File

@ -0,0 +1,178 @@
/*
* Hibernate, Relational Persistence for Idiomatic Java
*
* License: GNU Lesser General Public License (LGPL), version 2.1 or later.
* See the lgpl.txt file in the root directory or <http://www.gnu.org/licenses/lgpl-2.1.html>.
*/
package org.hibernate.community.dialect;
import java.util.List;
import org.hibernate.engine.spi.SessionFactoryImplementor;
import org.hibernate.query.sqm.ComparisonOperator;
import org.hibernate.query.sqm.FrameExclusion;
import org.hibernate.query.sqm.FrameKind;
import org.hibernate.sql.ast.SqlAstJoinType;
import org.hibernate.sql.ast.spi.AbstractSqlAstTranslator;
import org.hibernate.sql.ast.tree.Statement;
import org.hibernate.sql.ast.tree.expression.Expression;
import org.hibernate.sql.ast.tree.expression.FunctionExpression;
import org.hibernate.sql.ast.tree.expression.Literal;
import org.hibernate.sql.ast.tree.expression.Over;
import org.hibernate.sql.ast.tree.expression.QueryLiteral;
import org.hibernate.sql.ast.tree.expression.Summarization;
import org.hibernate.sql.ast.tree.from.QueryPartTableReference;
import org.hibernate.sql.ast.tree.from.TableGroupJoin;
import org.hibernate.sql.ast.tree.from.ValuesTableReference;
import org.hibernate.sql.ast.tree.predicate.BooleanExpressionPredicate;
import org.hibernate.sql.ast.tree.predicate.Predicate;
import org.hibernate.sql.ast.tree.select.QueryPart;
import org.hibernate.sql.ast.tree.select.QuerySpec;
import org.hibernate.sql.exec.spi.JdbcOperation;
/**
* A SQL AST translator for Altibase.
*
* @author Geoffrey Park
*/
public class AltibaseSqlAstTranslator<T extends JdbcOperation> extends AbstractSqlAstTranslator<T> {
public AltibaseSqlAstTranslator(SessionFactoryImplementor sessionFactory, Statement statement) {
super( sessionFactory, statement );
}
@Override
public void visitOffsetFetchClause(QueryPart queryPart) {
if ( !isRowNumberingCurrentQueryPart() ) {
// Use limit because Altibase does not support fetch first rows only.
renderCombinedLimitClause( queryPart );
}
}
@Override
protected void renderComparison(Expression lhs, ComparisonOperator operator, Expression rhs) {
// Altibase does not support is distinct from clause. So override renderComparion() and use not exists
renderComparisonEmulateIntersect( lhs, operator, rhs );
}
@Override
public void visitOver(Over<?> over) {
final Expression expression = over.getExpression();
if ( expression instanceof FunctionExpression && "row_number".equals( ( (FunctionExpression) expression ).getFunctionName() ) ) {
if ( over.getPartitions().isEmpty() && over.getOrderList().isEmpty()
&& over.getStartKind() == FrameKind.UNBOUNDED_PRECEDING
&& over.getEndKind() == FrameKind.CURRENT_ROW
&& over.getExclusion() == FrameExclusion.NO_OTHERS ) {
// Altibase doesn't allow an empty over clause for the row_number() function,
append( "row_number() over(order by 1)" );
return;
}
}
super.visitOver( over );
}
@Override
public void visitQuerySpec(QuerySpec querySpec) {
if ( shouldEmulateFetchClause( querySpec ) ) {
// Altibase does not support row_with_ties
emulateFetchOffsetWithWindowFunctions( querySpec, true );
}
else {
super.visitQuerySpec( querySpec );
}
}
protected boolean shouldEmulateFetchClause(QueryPart queryPart) {
// Check if current query part is already row numbering to avoid infinite recursion
return useOffsetFetchClause( queryPart ) && getQueryPartForRowNumbering() != queryPart
&& getDialect().supportsWindowFunctions() && !isRowsOnlyFetchClauseType( queryPart );
}
protected void renderTableGroupJoin(TableGroupJoin tableGroupJoin, List<TableGroupJoin> tableGroupJoinCollector) {
// Use join instead because Altibase does not support cross apply
appendSql( WHITESPACE );
if ( tableGroupJoin.getJoinType() != SqlAstJoinType.CROSS ) {
// No support for cross joins, so we emulate it with an inner join and always true on condition
appendSql( tableGroupJoin.getJoinType().getText() );
}
appendSql( "join " );
final Predicate predicate;
if ( tableGroupJoin.getPredicate() == null ) {
predicate = new BooleanExpressionPredicate( new QueryLiteral<>( true, getBooleanType() ) );
}
else {
predicate = tableGroupJoin.getPredicate();
}
if ( predicate != null && !predicate.isEmpty() ) {
renderTableGroup( tableGroupJoin.getJoinedGroup(), predicate, tableGroupJoinCollector );
}
else {
renderTableGroup( tableGroupJoin.getJoinedGroup(), null, tableGroupJoinCollector );
}
}
@Override
protected void renderPartitionItem(Expression expression) {
if ( expression instanceof Literal ) {
appendSql( "'0' || '0'" );
}
else if ( expression instanceof Summarization ) {
// This could theoretically be emulated by rendering all grouping variations of the query and
// connect them via union all but that's probably pretty inefficient and would have to happen
// on the query spec level
throw new UnsupportedOperationException( "Summarization is not supported by DBMS!" );
}
else {
expression.accept( this );
}
}
@Override
protected void renderOffsetExpression(Expression offsetExpression) {
// Altibase offset starts from 1
appendSql( "1+" );
offsetExpression.accept( this );
}
@Override
public void visitValuesTableReference(ValuesTableReference tableReference) {
emulateValuesTableReferenceColumnAliasing( tableReference );
}
@Override
public void visitQueryPartTableReference(QueryPartTableReference tableReference) {
emulateQueryPartTableReferenceColumnAliasing( tableReference );
}
protected String getFromDual() {
return " from dual";
}
@Override
protected String getFromDualForSelectOnly() {
return getFromDual();
}
@Override
protected boolean needsRecursiveKeywordInWithClause() {
return false;
}
@Override
protected boolean supportsRowValueConstructorSyntaxInQuantifiedPredicates() {
return false;
}
@Override
protected boolean supportsWithClauseInSubquery() {
return false;
}
@Override
protected boolean supportsSimpleQueryGrouping() {
return false;
}
}

View File

@ -113,6 +113,21 @@ public enum CommunityDatabase {
}
},
ALTIBASE {
@Override
public Dialect createDialect(DialectResolutionInfo info) {
return new AltibaseDialect( info );
}
@Override
public boolean productNameMatches(String databaseName) {
return "Altibase".equalsIgnoreCase( databaseName );
}
@Override
public String getDriverClassName(String jdbcUrl) {
return "Altibase.jdbc.driver.AltibaseDriver";
}
},
MIMER {
@Override
public Dialect createDialect(DialectResolutionInfo info) {

View File

@ -32,6 +32,8 @@ public class CommunityDialectSelector implements DialectSelector {
return Cache71Dialect.class;
case "CUBRID":
return CUBRIDDialect.class;
case "Altibase":
return AltibaseDialect.class;
case "DerbyTenFive":
return DerbyTenFiveDialect.class;
case "DerbyTenSix":

View File

@ -0,0 +1,36 @@
/*
* Hibernate, Relational Persistence for Idiomatic Java
*
* License: GNU Lesser General Public License (LGPL), version 2.1 or later.
* See the lgpl.txt file in the root directory or <http://www.gnu.org/licenses/lgpl-2.1.html>.
*/
/*
* Hibernate, Relational Persistence for Idiomatic Java
*
* License: GNU Lesser General Public License (LGPL), version 2.1 or later.
* See the lgpl.txt file in the root directory or <http://www.gnu.org/licenses/lgpl-2.1.html>.
*/
package org.hibernate.community.dialect.pagination;
import org.hibernate.dialect.pagination.LimitLimitHandler;
/**
* Limit handler for {@link org.hibernate.community.dialect.AltibaseDialect}.
*
* @author Geoffrey park
*/
public class AltibaseLimitHandler extends LimitLimitHandler {
public static final AltibaseLimitHandler INSTANCE = new AltibaseLimitHandler();
@Override
protected String limitClause(boolean hasFirstRow) {
return hasFirstRow ? " limit 1+?,?" : " limit ?";
}
@Override
protected String offsetOnlyClause() {
return " limit 1+?," + Integer.MAX_VALUE;
}
}

View File

@ -0,0 +1,31 @@
/*
* Hibernate, Relational Persistence for Idiomatic Java
*
* License: GNU Lesser General Public License (LGPL), version 2.1 or later.
* See the lgpl.txt file in the root directory or <http://www.gnu.org/licenses/lgpl-2.1.html>.
*/
package org.hibernate.community.dialect.sequence;
import org.hibernate.dialect.sequence.NextvalSequenceSupport;
import org.hibernate.dialect.sequence.SequenceSupport;
/**
* Sequence support for {@link org.hibernate.community.dialect.AltibaseDialect}.
*
* @author Geoffrey park
*/
public class AltibaseSequenceSupport extends NextvalSequenceSupport {
public static final SequenceSupport INSTANCE = new AltibaseSequenceSupport();
@Override
public String getFromDual() {
return " from dual";
}
@Override
public boolean sometimesNeedsStartingValue() {
return true;
}
}

View File

@ -0,0 +1,58 @@
/*
* Hibernate, Relational Persistence for Idiomatic Java
*
* License: GNU Lesser General Public License (LGPL), version 2.1 or later.
* See the lgpl.txt file in the root directory or <http://www.gnu.org/licenses/lgpl-2.1.html>.
*/
package org.hibernate.community.dialect.sequence;
import org.hibernate.tool.schema.extract.internal.SequenceInformationExtractorLegacyImpl;
/**
* An SequenceInfomation for Altibase
*
* @author Geoffrey Park
*/
public class SequenceInformationExtractorAltibaseDatabaseImpl extends SequenceInformationExtractorLegacyImpl {
/**
* Singleton access
*/
public static final SequenceInformationExtractorAltibaseDatabaseImpl INSTANCE = new SequenceInformationExtractorAltibaseDatabaseImpl();
@Override
protected String sequenceNameColumn() {
return "sequence_name";
}
@Override
protected String sequenceCatalogColumn() {
return null;
}
@Override
protected String sequenceSchemaColumn() {
return null;
}
@Override
protected String sequenceStartValueColumn() {
return "start_value";
}
@Override
protected String sequenceMinValueColumn() {
return "min_value";
}
@Override
protected String sequenceMaxValueColumn() {
return "max_value";
}
@Override
protected String sequenceIncrementColumn() {
return "increment_by";
}
}

View File

@ -0,0 +1,74 @@
/*
* Hibernate, Relational Persistence for Idiomatic Java
*
* License: GNU Lesser General Public License (LGPL), version 2.1 or later.
* See the lgpl.txt file in the root directory or <http://www.gnu.org/licenses/lgpl-2.1.html>.
*/
package org.hibernate.community.dialect;
import java.util.Locale;
import org.hibernate.dialect.DatabaseVersion;
import org.hibernate.dialect.Dialect;
import org.hibernate.query.spi.Limit;
import org.hibernate.testing.junit4.BaseUnitTestCase;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
/**
* Unit test of the behavior of the AltibaseDialect utility methods
*
* @author Geoffrey Park
*/
public class AltibaseDialectTestCase extends BaseUnitTestCase {
private Dialect dialect;
@Before
public void setUp() {
dialect = new AltibaseDialect( DatabaseVersion.make( 7, 3 ));
}
@After
public void tearDown() {
dialect = null;
}
@Test
public void testSupportLimits() {
assertTrue(dialect.getLimitHandler().supportsLimit());
}
@Test
public void testSelectWithLimitOnly() {
assertEquals( "select c1, c2 from t1 order by c1, c2 desc limit ?",
dialect.getLimitHandler().processSql("select c1, c2 from t1 order by c1, c2 desc",
toRowSelection( 0, 15 ) ).toLowerCase( Locale.ROOT));
}
@Test
public void testSelectWithOffsetLimit() {
assertEquals( "select c1, c2 from t1 order by c1, c2 desc limit 1+?,?",
dialect.getLimitHandler().processSql("select c1, c2 from t1 order by c1, c2 desc",
toRowSelection( 5, 15 ) ).toLowerCase(Locale.ROOT));
}
@Test
public void testSelectWithNoLimit() {
assertEquals( "select c1, c2 from t1 order by c1, c2 desc",
dialect.getLimitHandler().processSql("select c1, c2 from t1 order by c1, c2 desc",
null ).toLowerCase(Locale.ROOT));
}
private Limit toRowSelection(int firstRow, int maxRows) {
Limit selection = new Limit();
selection.setFirstRow( firstRow );
selection.setMaxRows( maxRows );
return selection;
}
}

View File

@ -0,0 +1,142 @@
/*
* Hibernate, Relational Persistence for Idiomatic Java
*
* License: GNU Lesser General Public License (LGPL), version 2.1 or later.
* See the lgpl.txt file in the root directory or <http://www.gnu.org/licenses/lgpl-2.1.html>.
*/
package org.hibernate.community.dialect;
import jakarta.persistence.Tuple;
import org.hibernate.query.Query;
import org.hibernate.testing.orm.junit.DomainModel;
import org.hibernate.testing.orm.junit.RequiresDialect;
import org.hibernate.testing.orm.junit.SessionFactory;
import org.hibernate.testing.orm.junit.SessionFactoryScope;
import org.junit.jupiter.api.*;
import java.sql.Date;
import java.sql.SQLException;
import java.time.Month;
import java.time.ZoneId;
import java.time.ZonedDateTime;
import java.util.List;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.MatcherAssert.assertThat;
import static org.junit.jupiter.api.Assertions.assertNotNull;
@DomainModel(
annotatedClasses = { Person.class },
xmlMappings = "org/hibernate/community/dialect/Person.hbm.xml"
)
@RequiresDialect(AltibaseDialect.class)
@SessionFactory
public class AltibaseFunctionsTest {
@BeforeAll
public void createTestData(SessionFactoryScope scope) {
scope.inTransaction(
session -> {
final Person person = new Person();
person.setId( 1 );
person.setName( "test.1" );
ZonedDateTime zonedDateTime = ZonedDateTime.of(
1990, Month.APRIL.getValue(), 15,
0, 0, 0, 0,
ZoneId.of( "UTC" )
);
person.setBirthDate( new java.sql.Date( zonedDateTime.toInstant().toEpochMilli() ) ) ;
person.setWeightInKilograms( 66.0 );
person.setHeightInMeters( 167.1 );
person.setIsMarried( false );
byte[] arry = new byte[ 15 ];
for (int i = 0; i < arry.length; i++)
{
arry[i] = (byte)i;
}
person.setBinaryData( session.getLobHelper().createBlob(arry) );
person.setComments( session.getLobHelper().createClob("blahblah") );
session.save( person );
}
);
}
@AfterAll
public static void dropTestData(SessionFactoryScope scope) {
scope.inTransaction(
session -> {
session.createQuery( "delete Person" ).executeUpdate();
}
);
}
@Test
public void testSimpleFunction(SessionFactoryScope scope) throws Throwable {
scope.inTransaction(
session -> {
final Person result = session
.createQuery( "from Person", Person.class )
.uniqueResult();
assertNotNull( result );
assertThat( result.getId(), is( 1 ) );
assertThat( result.getName(), is( "test.1" ) );
Date birthDate = result.getBirthDate();
assertThat( birthDate.toString(), is( "1990-04-15" ) );
assertThat( result.getWeightInKilograms(), is( 66.0 ) );
assertThat( result.getHeightInMeters(), is( 167.1 ) );
assertThat( result.getIsMarried(), is( false ) );
try {
assertThat( result.getBinaryData().length(), is( 15L ));
assertThat( result.getComments().length(), is( 8L ));
} catch (SQLException e) {
throw new RuntimeException(e);
}
}
);
}
@Test
public void testLocateFunction(SessionFactoryScope scope) {
scope.inTransaction(
session -> {
final Person result = session
.createQuery( "select p from Person p where locate('.', p.name) > 0", Person.class )
.uniqueResult();
assertNotNull( result );
assertThat( result.getName(), is( "test.1" ) );
}
);
}
@Test
public void testSubstringFunction(SessionFactoryScope scope) {
scope.inTransaction(
session -> {
final Person result = session
.createQuery( "select p from Person p where substring(p.name, 1, 4) = 'test'", Person.class )
.uniqueResult();
assertNotNull( result );
assertThat( result.getName(), is( "test.1" ) );
}
);
}
@Test
public void testBitLengthFunction(SessionFactoryScope scope) {
scope.inTransaction(
session -> {
final Query<Tuple> query = session.createQuery(
"select bit_length(p.comments) from Person p",
Tuple.class
);
final List<Tuple> results = query.getResultList();
assertThat( results.size(), is( 1 ) );
final Tuple testEntity = results.get( 0 );
assertThat( testEntity.get( 0, Integer.class ), is( 64 ) );
}
);
}
}

View File

@ -23,6 +23,7 @@ public class CommunityDialectSelectorTest {
testDialectNamingResolution( Cache71Dialect.class );
testDialectNamingResolution( CUBRIDDialect.class );
testDialectNamingResolution( AltibaseDialect.class );
testDialectNamingResolution( DerbyTenFiveDialect.class );
testDialectNamingResolution( DerbyTenSixDialect.class );

View File

@ -0,0 +1,27 @@
<?xml version="1.0"?>
<!--
~ Hibernate, Relational Persistence for Idiomatic Java
~
~ License: GNU Lesser General Public License (LGPL), version 2.1 or later
~ See the lgpl.txt file in the root directory or http://www.gnu.org/licenses/lgpl-2.1.html
-->
<!DOCTYPE hibernate-mapping PUBLIC
"-//Hibernate/Hibernate Mapping DTD 3.0//EN"
"http://www.hibernate.org/dtd/hibernate-mapping-3.0.dtd">
<hibernate-mapping package="org.hibernate.community.dialect">
<class name="Person" table="person">
<id name="id" column="id" type="int">
<generator class="sequence">
<param name="sequence">PERSON_SEQ</param>
</generator>
</id>
<property name="name" column="name" type="string"/>
<property name="birthDate" column="birth_date" type="date"/>
<property name="weightInKilograms" column="weight_kg" type="double"/>
<property name="heightInMeters" column="height_m" type="double"/>
<property name="isMarried" column="is_married" type="boolean"/>
<property name="binaryData" column="binary_data" type="blob"/>
<property name="comments" column="comments" type="clob"/>
</class>
</hibernate-mapping>

View File

@ -0,0 +1,85 @@
/*
* Hibernate, Relational Persistence for Idiomatic Java
*
* License: GNU Lesser General Public License (LGPL), version 2.1 or later.
* See the lgpl.txt file in the root directory or <http://www.gnu.org/licenses/lgpl-2.1.html>.
*/
package org.hibernate.community.dialect;
import java.sql.*;
public class Person {
private int id;
private String name;
private Date birthDate;
private double weightInKilograms;
private double heightInMeters;
private boolean isMarried;
private Blob binaryData;
private Clob comments;
public int getId() {
return id;
}
public void setId(int id) {
this.id = id;
}
public void setName(String name) {
this.name = name;
}
public String getName() {
return name;
}
public void setBirthDate(Date birthDate) {
this.birthDate = birthDate;
}
public Date getBirthDate() {
return birthDate;
}
public void setWeightInKilograms(double weightInKilograms) {
this.weightInKilograms = weightInKilograms;
}
public void setIsMarried(boolean isMarried) {
this.isMarried = isMarried;
}
public double getWeightInKilograms() {
return weightInKilograms;
}
public void setHeightInMeters(double heightInMeters) {
this.heightInMeters = heightInMeters;
}
public double getHeightInMeters() {
return heightInMeters;
}
public boolean getIsMarried() {
return isMarried;
}
public Blob getBinaryData() {
return binaryData;
}
public void setBinaryData(Blob binaryData) {
this.binaryData = binaryData;
}
public Clob getComments() {
return comments;
}
public void setComments(Clob comments) {
this.comments = comments;
}
}

View File

@ -15,6 +15,7 @@ import java.util.Date;
import java.util.Locale;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.Dialect;
import org.hibernate.dialect.OracleDialect;
import org.hibernate.dialect.PostgresPlusDialect;
@ -45,6 +46,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
@SkipForDialect(dialectClass = OracleDialect.class, reason = "HHH-6834")
@SkipForDialect(dialectClass = PostgresPlusDialect.class, reason = "HHH-6834")
@SkipForDialect(dialectClass = SybaseASEDialect.class, reason = "jConnect reports the type code 11 for bigdatetime columns, which is an unknown type code..")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Altibase reports the type code 93 for date columns")
@RequiresDialectFeature(feature = DialectFeatureChecks.SupportsExpectedLobUsagePattern.class, jiraKey = "HHH-6834")
@DomainModel(
annotatedClasses = { SomeEntity.class, SomeOtherEntity.class }

View File

@ -7,11 +7,13 @@
package org.hibernate.orm.test.annotations.idclass.xml;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.query.Query;
import org.hibernate.testing.orm.junit.DomainModel;
import org.hibernate.testing.orm.junit.SessionFactory;
import org.hibernate.testing.orm.junit.SessionFactoryScope;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.junit.jupiter.api.Test;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -26,6 +28,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
annotatedClasses = HabitatSpeciesLink.class,
xmlMappings = "org/hibernate/orm/test/annotations/idclass/xml/HabitatSpeciesLink.xml"
)
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "'link' is not escaped even though autoQuoteKeywords is enabled" )
@SessionFactory
public class IdClassXmlTest {
@Test

View File

@ -6,6 +6,7 @@
*/
package org.hibernate.orm.test.annotations.manytoonewithformula;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.HSQLDialect;
import org.hibernate.dialect.OracleDialect;
import org.hibernate.dialect.SQLServerDialect;
@ -149,6 +150,7 @@ public class ManyToOneWithFormulaTest {
@Test
@SkipForDialect( dialectClass =HSQLDialect.class, reason = "The used join conditions does not work in HSQLDB. See HHH-4497." )
@SkipForDialect( dialectClass = OracleDialect.class, reason = "Oracle do not support 'substring' function" )
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = " Altibase char type returns with trailing spaces")
public void testManyToOneFromNonPkToNonPk(SessionFactoryScope scope) {
// also tests usage of the stand-alone @JoinFormula annotation
// (i.e. not wrapped within @JoinColumnsOrFormulas)

View File

@ -16,6 +16,7 @@ import org.hibernate.engine.spi.SessionFactoryImplementor;
import org.hibernate.testing.orm.junit.BootstrapServiceRegistry;
import org.hibernate.testing.orm.junit.BootstrapServiceRegistry.JavaService;
import org.hibernate.testing.orm.junit.DomainModel;
import org.hibernate.testing.orm.junit.RequiresDialect;
import org.hibernate.testing.orm.junit.ServiceRegistry;
import org.hibernate.testing.orm.junit.SessionFactory;
import org.hibernate.testing.orm.junit.SessionFactoryScope;
@ -40,6 +41,7 @@ import static org.hamcrest.Matchers.instanceOf;
)
@DomainModel
@SessionFactory
@RequiresDialect(H2Dialect.class)
public class DiscoveredDialectResolverTests {
@Test
public void testRegistration(SessionFactoryScope scope) {

View File

@ -18,6 +18,7 @@ import org.hibernate.annotations.Filter;
import org.hibernate.annotations.FilterDef;
import org.hibernate.annotations.JdbcTypeCode;
import org.hibernate.annotations.ParamDef;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.community.dialect.FirebirdDialect;
import org.hibernate.dialect.AbstractHANADialect;
import org.hibernate.dialect.CockroachDialect;
@ -87,6 +88,7 @@ public class FilterParameterTests {
@SkipForDialect(dialectClass = CockroachDialect.class, matchSubTypes = true, reason = "Cockroach silently converts a boolean to string types")
@SkipForDialect(dialectClass = PostgresPlusDialect.class, reason = "PostgresPlus silently converts a boolean to string types")
@SkipForDialect(dialectClass = FirebirdDialect.class, reason = "Firebird silently converts a boolean to string")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Altibase silently converts a boolean to string")
@SkipForDialect(dialectClass = OracleDialect.class, majorVersion = 23, reason = "Oracle 23 interprets Y and T as true and N and F as false, so this works")
public void testYesNoMismatch(SessionFactoryScope scope) {
scope.inTransaction( (session) -> {
@ -136,6 +138,7 @@ public class FilterParameterTests {
@SkipForDialect(dialectClass = MariaDBDialect.class, reason = "MariaDB silently converts a boolean to integral types")
@SkipForDialect(dialectClass = TiDBDialect.class, reason = "TiDB silently converts a boolean to integral types")
@SkipForDialect(dialectClass = SQLServerDialect.class, reason = "SQL Server silently converts a boolean to integral types")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Altibase silently converts a boolean to integral types")
@SkipForDialect(dialectClass = SybaseDialect.class, matchSubTypes = true, reason = "Sybase silently converts a boolean to integral types")
@SkipForDialect(dialectClass = AbstractHANADialect.class, matchSubTypes = true, reason = "HANA silently converts a boolean to integral types")
@SkipForDialect(dialectClass = FirebirdDialect.class, matchSubTypes = true, reason = "Firebird silently converts a boolean to integral types")

View File

@ -22,8 +22,10 @@ import org.hibernate.annotations.FetchMode;
import org.hibernate.annotations.SQLRestriction;
import org.hibernate.annotations.SortNatural;
import org.hibernate.annotations.Where;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.testing.TestForIssue;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.junit.jupiter.api.Test;
/**
@ -31,6 +33,7 @@ import org.junit.jupiter.api.Test;
* @author Nathan Xu
*/
@TestForIssue(jiraKey = "HHH-14227")
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "'TYPE' is not escaped even though autoQuoteKeywords is enabled")
public class InsertOrderingSelfReferenceTest extends BaseInsertOrderingTest {
@Override

View File

@ -17,11 +17,13 @@ import jakarta.persistence.criteria.Root;
import org.hibernate.Hibernate;
import org.hibernate.Session;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.jdbc.AbstractWork;
import org.hibernate.testing.orm.junit.DomainModel;
import org.hibernate.testing.orm.junit.SessionFactory;
import org.hibernate.testing.orm.junit.SessionFactoryScope;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.junit.jupiter.api.Test;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -39,6 +41,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class JoinTest {
@Test
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "In line view in left join is not possible in Altibase")
public void testSequentialSelects(SessionFactoryScope scope) {
scope.inTransaction(
s -> {

View File

@ -15,6 +15,7 @@ import jakarta.persistence.Inheritance;
import jakarta.persistence.InheritanceType;
import jakarta.persistence.Table;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.persister.entity.EntityPersister;
import org.hibernate.persister.entity.JoinedSubclassEntityPersister;
@ -22,6 +23,7 @@ import org.hibernate.testing.TestForIssue;
import org.hibernate.testing.orm.junit.DomainModel;
import org.hibernate.testing.orm.junit.SessionFactory;
import org.hibernate.testing.orm.junit.SessionFactoryScope;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.junit.jupiter.api.Test;
import static org.junit.jupiter.api.Assertions.assertEquals;
@ -44,6 +46,7 @@ import static org.junit.jupiter.api.Assertions.fail;
public class JoinedSubclassWithExplicitDiscriminatorTest {
@Test
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "'TYPE' is a keyword in Altibase and escaped here")
public void metadataAssertions(SessionFactoryScope scope) {
EntityPersister p = scope.getSessionFactory().getMappingMetamodel().getEntityDescriptor(Dog.class.getName());
assertNotNull( p );

View File

@ -15,6 +15,7 @@ import java.util.Collections;
import java.util.List;
import org.hibernate.Session;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.DB2Dialect;
import org.hibernate.dialect.DerbyDialect;
import org.hibernate.dialect.PostgresPlusDialect;
@ -308,6 +309,7 @@ public class ExpressionsTest extends AbstractMetamodelSpecificTest {
@Test @SkipForDialect(dialectClass = SybaseDialect.class, matchSubTypes = true, reason = "numeric overflows")
@SkipForDialect(dialectClass = PostgresPlusDialect.class, reason = "does not support extract(epoch)")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "datediff overflow limits")
public void testDateTimeOperations() {
HibernateCriteriaBuilder builder = (HibernateCriteriaBuilder) this.builder;
doInJPA(

View File

@ -19,6 +19,7 @@ import org.hibernate.LockOptions;
import org.hibernate.Session;
import org.hibernate.TransactionException;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.community.dialect.FirebirdDialect;
import org.hibernate.dialect.AbstractHANADialect;
import org.hibernate.dialect.CockroachDialect;
@ -101,6 +102,7 @@ public class LockTest extends BaseEntityManagerFunctionalTestCase {
comment = "Test verifies proper exception throwing when a lock timeout is specified.",
jiraKey = "HHH-7252" )
@SkipForDialect(value = CockroachDialect.class, comment = "for update clause does not imply locking. See https://github.com/cockroachdb/cockroach/issues/88995")
@SkipForDialect(value = AltibaseDialect.class, comment = "Altibase close socket after lock timeout occurred")
public void testFindWithPessimisticWriteLockTimeoutException() {
Lock lock = new Lock();
lock.setName( "name" );
@ -150,6 +152,7 @@ public class LockTest extends BaseEntityManagerFunctionalTestCase {
comment = "Test verifies proper exception throwing when a lock timeout is specified for Query#getSingleResult.",
jiraKey = "HHH-13364" )
@SkipForDialect(value = CockroachDialect.class, comment = "for update clause does not imply locking. See https://github.com/cockroachdb/cockroach/issues/88995")
@SkipForDialect(value = AltibaseDialect.class, comment = "Altibase close socket after lock timeout occurred")
public void testQuerySingleResultPessimisticWriteLockTimeoutException() {
Lock lock = new Lock();
lock.setName( "name" );
@ -196,6 +199,7 @@ public class LockTest extends BaseEntityManagerFunctionalTestCase {
comment = "Test verifies proper exception throwing when a lock timeout is specified for Query#getResultList.",
jiraKey = "HHH-13364" )
@SkipForDialect(value = CockroachDialect.class, comment = "for update clause does not imply locking. See https://github.com/cockroachdb/cockroach/issues/88995")
@SkipForDialect(value = AltibaseDialect.class, comment = "Altibase close socket after lock timeout occurred")
public void testQueryResultListPessimisticWriteLockTimeoutException() {
Lock lock = new Lock();
lock.setName( "name" );
@ -245,6 +249,7 @@ public class LockTest extends BaseEntityManagerFunctionalTestCase {
comment = "Test verifies proper exception throwing when a lock timeout is specified for NamedQuery#getResultList.",
jiraKey = "HHH-13364" )
@SkipForDialect(value = CockroachDialect.class, comment = "for update clause does not imply locking. See https://github.com/cockroachdb/cockroach/issues/88995")
@SkipForDialect(value = AltibaseDialect.class, comment = "Altibase close socket after lock timeout occurred")
public void testNamedQueryResultListPessimisticWriteLockTimeoutException() {
Lock lock = new Lock();
lock.setName( "name" );
@ -1183,6 +1188,7 @@ public class LockTest extends BaseEntityManagerFunctionalTestCase {
@SkipForDialect(value = AbstractHANADialect.class, comment = "Seems like FK constraint checks are not compatible with exclusive locks")
@SkipForDialect(value = CockroachDialect.class, comment = "Cockroach supports the 'for no key update' syntax but it doesn't work")
@SkipForDialect(value = FirebirdDialect.class, comment = "Seems like FK constraint checks are not compatible with exclusive locks")
@SkipForDialect(value = AltibaseDialect.class, comment = "Seems like FK constraint checks are not compatible with exclusive locks")
public void testLockInsertFkTarget() {
Lock lock = new Lock();
lock.setName( "name" );
@ -1221,6 +1227,7 @@ public class LockTest extends BaseEntityManagerFunctionalTestCase {
@SkipForDialect(value = AbstractHANADialect.class, comment = "Seems like FK constraint checks are not compatible with exclusive locks")
@SkipForDialect(value = CockroachDialect.class, comment = "Cockroach supports the 'for no key update' syntax but it doesn't work")
@SkipForDialect(value = FirebirdDialect.class, comment = "Seems like FK constraint checks are not compatible with exclusive locks")
@SkipForDialect(value = AltibaseDialect.class, comment = "FK constraint checks are not compatible with exclusive locks")
public void testLockUpdateFkTarget() {
Lock lock1 = new Lock();
lock1.setName( "l1" );

View File

@ -12,6 +12,7 @@ import java.util.HashMap;
import java.util.Map;
import jakarta.persistence.LockModeType;
import org.hibernate.Session;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.CockroachDialect;
import org.hibernate.orm.test.jpa.BaseEntityManagerFunctionalTestCase;
import org.hibernate.testing.orm.jdbc.PreparedStatementSpyConnectionProvider;
@ -34,6 +35,7 @@ import static org.junit.Assert.fail;
*/
@RequiresDialectFeature({DialectChecks.SupportsLockTimeouts.class})
@SkipForDialect(value = CockroachDialect.class, comment = "for update clause does not imply locking. See https://github.com/cockroachdb/cockroach/issues/88995")
@SkipForDialect(value = AltibaseDialect.class, comment = "Altibase does not close Statement after lock timeout")
public class StatementIsClosedAfterALockExceptionTest extends BaseEntityManagerFunctionalTestCase {
private static final PreparedStatementSpyConnectionProvider CONNECTION_PROVIDER = new PreparedStatementSpyConnectionProvider();

View File

@ -6,6 +6,7 @@
*/
package org.hibernate.orm.test.jpa.naturalid;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.AbstractHANADialect;
import org.hibernate.dialect.OracleDialect;
@ -25,6 +26,8 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
reason = "Oracle do not support identity key generation")
@SkipForDialect(dialectClass = AbstractHANADialect.class, matchSubTypes = true,
reason = "Hana do not support identity key generation")
@SkipForDialect(dialectClass = AltibaseDialect.class,
reason = "Altibase do not support identity key generation")
public class MutableNaturalIdTest extends AbstractJPATest {
@Override
protected Class<?>[] getAnnotatedClasses() {

View File

@ -24,6 +24,7 @@ import org.hibernate.Session;
import org.hibernate.annotations.JdbcTypeCode;
import org.hibernate.annotations.Nationalized;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.community.dialect.FirebirdDialect;
import org.hibernate.dialect.AbstractHANADialect;
import org.hibernate.dialect.AbstractTransactSQLDialect;
@ -128,6 +129,7 @@ public class NativeQueryResultTypeAutoDiscoveryTest {
@SkipForDialect(dialectClass = SybaseDialect.class, matchSubTypes = true, reason = "No support for the bit datatype so we use tinyint")
@SkipForDialect(dialectClass = OracleDialect.class, reason = "No support for the bit datatype so we use number(1,0)")
@SkipForDialect(dialectClass = DB2Dialect.class, majorVersion = 10, reason = "No support for the bit datatype so we use smallint")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "No support for the bit datatype so we use char(1)")
public void booleanType() {
createEntityManagerFactory( BooleanEntity.class );
doTest( BooleanEntity.class, true );
@ -137,6 +139,7 @@ public class NativeQueryResultTypeAutoDiscoveryTest {
@SkipForDialect(dialectClass = SybaseDialect.class, matchSubTypes = true, reason = "No support for the bit datatype so we use tinyint")
@SkipForDialect(dialectClass = OracleDialect.class, reason = "No support for the bit datatype so we use number(1,0)")
@SkipForDialect(dialectClass = DB2Dialect.class, majorVersion = 10, reason = "No support for the bit datatype so we use smallint")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "No support for the bit datatype so we use char(1)")
public void bitType() {
createEntityManagerFactory( BitEntity.class );
doTest( BitEntity.class, false );
@ -152,6 +155,7 @@ public class NativeQueryResultTypeAutoDiscoveryTest {
@SkipForDialect(dialectClass = AbstractHANADialect.class, matchSubTypes = true, reason = "No support for the tinyint datatype so we use smallint")
@SkipForDialect(dialectClass = OracleDialect.class, reason = "Oracle maps tinyint to number")
@SkipForDialect(dialectClass = FirebirdDialect.class, reason = "No support for the tinyint datatype so we use smallint")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Altibase maps tinyint to smallint")
public void tinyintType() {
createEntityManagerFactory( TinyintEntity.class );
doTest( TinyintEntity.class, (byte)127 );
@ -160,6 +164,7 @@ public class NativeQueryResultTypeAutoDiscoveryTest {
@Test
@SkipForDialect(dialectClass = H2Dialect.class, reason = "Turns floats into doubles in result sets and advertises the type as double in the metadata")
@SkipForDialect(dialectClass = HSQLDialect.class, reason = "Turns floats into doubles in result sets and advertises the type as double in the metadata")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Turns floats into doubles in result sets and advertises the type as double in the metadata")
public void floatType() {
createEntityManagerFactory( FloatEntity.class );
doTest( FloatEntity.class, 15516.125f );
@ -182,6 +187,7 @@ public class NativeQueryResultTypeAutoDiscoveryTest {
@SkipForDialect(dialectClass = AbstractTransactSQLDialect.class, matchSubTypes = true, reason = "Value is too big for the maximum allowed precision of SQL Server and Sybase")
@SkipForDialect(dialectClass = AbstractHANADialect.class, matchSubTypes = true, reason = "Value is too big for the maximum allowed precision of HANA")
@SkipForDialect(dialectClass = FirebirdDialect.class, reason = "Value is too big for the maximum allowed precision of Firebird")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Value is too big for the maximum allowed precision of Altibase")
public void numericType() {
createEntityManagerFactory(
NumericEntity.class
@ -196,6 +202,7 @@ public class NativeQueryResultTypeAutoDiscoveryTest {
@SkipForDialect(dialectClass = AbstractTransactSQLDialect.class, matchSubTypes = true, reason = "Value is too big for the maximum allowed precision of SQL Server and Sybase")
@SkipForDialect(dialectClass = AbstractHANADialect.class, matchSubTypes = true, reason = "Value is too big for the maximum allowed precision of HANA")
@SkipForDialect(dialectClass = FirebirdDialect.class, reason = "Value is too big for the maximum allowed precision of Firebird")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Value is too big for the maximum allowed precision of Altibase")
public void decimalType() {
createEntityManagerFactory( DecimalEntity.class );
doTest( DecimalEntity.class, new BigDecimal( "5464384284258458485484848458.48465843584584684" ) );
@ -217,6 +224,7 @@ public class NativeQueryResultTypeAutoDiscoveryTest {
@SkipForDialect(dialectClass = DB2Dialect.class, reason = "DB2 maps LONGVARCHAR to CLOB")
@SkipForDialect(dialectClass = SybaseDialect.class, matchSubTypes = true, reason = "Sybase maps LONGVARCHAR to CLOB")
@SkipForDialect(dialectClass = AbstractHANADialect.class, matchSubTypes = true, reason = "HANA maps LONGVARCHAR to CLOB")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Altibase maps LONGVARCHAR to CLOB")
public void longCharType() {
createEntityManagerFactory(
LongvarcharEntity.class
@ -254,6 +262,7 @@ public class NativeQueryResultTypeAutoDiscoveryTest {
@SkipForDialect(dialectClass = OracleDialect.class, reason = "Oracle maps LONGVARBINARY to BLOB")
@SkipForDialect(dialectClass = DB2Dialect.class, reason = "DB2 maps LONGVARBINARY to BLOB")
@SkipForDialect(dialectClass = SybaseDialect.class, matchSubTypes = true, reason = "Sybase maps LONGVARBINARY to BLOB")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Altibase maps LONGVARBINARY to BLOB")
@SkipForDialect(dialectClass = AbstractHANADialect.class, matchSubTypes = true, reason = "HANA maps LONGVARCHAR to BLOB")
public void longBinaryType() {
createEntityManagerFactory(
@ -288,6 +297,7 @@ public class NativeQueryResultTypeAutoDiscoveryTest {
@SkipForDialect(dialectClass = OracleDialect.class, reason = "Oracle maps DATE and TIME to TIMESTAMP")
@SkipForDialect(dialectClass = PostgresPlusDialect.class, reason = "EDB maps DATE and TIME to TIMESTAMP")
@SkipForDialect(dialectClass = SybaseDialect.class, reason = "Sybase maps DATE and TIME to TIMESTAMP", matchSubTypes = true)
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Altibase maps DATE and TIME to TIMESTAMP")
public void dateTimeTypes() {
createEntityManagerFactory(
DateEntity.class,

View File

@ -18,6 +18,7 @@ import org.hibernate.LockOptions;
import org.hibernate.Session;
import org.hibernate.boot.registry.StandardServiceRegistryBuilder;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.CockroachDialect;
import org.hibernate.dialect.SQLServerDialect;
import org.hibernate.dialect.SybaseASEDialect;
@ -82,6 +83,7 @@ public class LockModeTest extends BaseSessionFactoryFunctionalTest {
@Test
@RequiresDialectFeature( feature = DialectFeatureChecks.SupportsLockTimeouts.class )
@SkipForDialect(dialectClass = CockroachDialect.class, reason = "for update clause does not imply locking. See https://github.com/cockroachdb/cockroach/issues/88995")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Can't commit transaction because Altibase closes socket after lock timeout")
@SuppressWarnings( {"deprecation"})
public void testLoading() {
// open a session, begin a transaction and lock row
@ -99,6 +101,7 @@ public class LockModeTest extends BaseSessionFactoryFunctionalTest {
@Test
@RequiresDialectFeature( feature = DialectFeatureChecks.SupportsLockTimeouts.class )
@SkipForDialect(dialectClass = CockroachDialect.class, reason = "for update clause does not imply locking. See https://github.com/cockroachdb/cockroach/issues/88995")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Can't commit transaction because Altibase closes socket after lock timeout")
public void testCriteria() {
// open a session, begin a transaction and lock row
doInHibernate( this::sessionFactory, session -> {
@ -122,6 +125,7 @@ public class LockModeTest extends BaseSessionFactoryFunctionalTest {
@Test
@RequiresDialectFeature( feature = DialectFeatureChecks.SupportsLockTimeouts.class )
@SkipForDialect(dialectClass = CockroachDialect.class, reason = "for update clause does not imply locking. See https://github.com/cockroachdb/cockroach/issues/88995")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Can't commit transaction because Altibase closes socket after lock timeout")
public void testCriteriaAliasSpecific() {
// open a session, begin a transaction and lock row
doInHibernate( this::sessionFactory, session -> {
@ -147,6 +151,7 @@ public class LockModeTest extends BaseSessionFactoryFunctionalTest {
@Test
@RequiresDialectFeature( feature = DialectFeatureChecks.SupportsLockTimeouts.class )
@SkipForDialect(dialectClass = CockroachDialect.class, reason = "for update clause does not imply locking. See https://github.com/cockroachdb/cockroach/issues/88995")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Can't commit transaction because Altibase closes socket after lock timeout")
public void testQuery() {
// open a session, begin a transaction and lock row
doInHibernate( this::sessionFactory, session -> {

View File

@ -10,6 +10,7 @@ import java.util.List;
import java.util.Map;
import java.util.concurrent.TimeUnit;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.CockroachDialect;
import org.hibernate.dialect.HSQLDialect;
import org.hibernate.dialect.OracleDialect;
@ -41,6 +42,7 @@ import static org.hibernate.jpa.SpecHints.HINT_SPEC_QUERY_TIMEOUT;
@SkipForDialect(dialectClass = HSQLDialect.class, reason = "Seems HSQLDB doesn't cancel the query if it waits for a lock?!")
@SkipForDialect(dialectClass = CockroachDialect.class, reason = "Cockroach allows the concurrent access but cancels one or both transactions at the end")
@SkipForDialect(dialectClass = OracleDialect.class, majorVersion = 11, reason = "Timeouts don't work on Oracle 11 when using a driver other than ojdbc6, but we can't test with that driver")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Altibase does not support timeout in statement level")
public class FollowOnLockingTest {
@Test

View File

@ -10,6 +10,7 @@ import java.sql.Types;
import org.hibernate.boot.model.FunctionContributions;
import org.hibernate.boot.model.FunctionContributor;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.AbstractHANADialect;
import org.hibernate.dialect.DB2Dialect;
import org.hibernate.dialect.OracleDialect;
@ -358,6 +359,7 @@ public class BooleanMappingTests {
@SkipForDialect(dialectClass = SybaseDialect.class, matchSubTypes = true)
@SkipForDialect(dialectClass = AbstractHANADialect.class, matchSubTypes = true)
@SkipForDialect(dialectClass = DB2Dialect.class, majorVersion = 10)
@SkipForDialect(dialectClass = AltibaseDialect.class)
public void testBooleanFunctionInPredicate(SessionFactoryScope scope) {
// Not strictly relevant to boolean mappings, but test that boolean
// functions work *in a* predicate after HHH-16182

View File

@ -14,6 +14,7 @@ import java.util.Map;
import org.hibernate.annotations.JdbcTypeCode;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.AbstractHANADialect;
import org.hibernate.dialect.DerbyDialect;
import org.hibernate.dialect.OracleDialect;
@ -165,6 +166,7 @@ public abstract class JsonMappingTests {
@SkipForDialect(dialectClass = AbstractHANADialect.class, matchSubTypes = true, reason = "HANA doesn't support comparing LOBs with the = operator")
@SkipForDialect(dialectClass = SybaseDialect.class, matchSubTypes = true, reason = "Sybase doesn't support comparing LOBs with the = operator")
@SkipForDialect(dialectClass = OracleDialect.class, matchSubTypes = true, reason = "Oracle doesn't support comparing JSON with the = operator")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Altibase doesn't support comparing CLOBs with the = operator")
public void verifyComparisonWorks(SessionFactoryScope scope) {
scope.inTransaction(
(session) -> {

View File

@ -20,6 +20,7 @@ import org.hibernate.annotations.TimeZoneColumn;
import org.hibernate.annotations.TimeZoneStorage;
import org.hibernate.annotations.TimeZoneStorageType;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.H2Dialect;
import org.hibernate.testing.orm.junit.DialectFeatureChecks;
@ -29,6 +30,7 @@ import org.hibernate.testing.orm.junit.ServiceRegistry;
import org.hibernate.testing.orm.junit.SessionFactory;
import org.hibernate.testing.orm.junit.SessionFactoryScope;
import org.hibernate.testing.orm.junit.Setting;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
@ -107,12 +109,14 @@ public class TimeZoneStorageMappingTests {
@Test
@RequiresDialectFeature(feature = DialectFeatureChecks.SupportsFormat.class)
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "Altibase doesn't allow function in date format string")
public void testOffsetRetainedFormatAuto(SessionFactoryScope scope) {
testOffsetRetainedFormat( scope, "Auto" );
}
@Test
@RequiresDialectFeature(feature = DialectFeatureChecks.SupportsFormat.class)
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "Altibase doesn't allow function in date format string")
public void testOffsetRetainedFormatColumn(SessionFactoryScope scope) {
testOffsetRetainedFormat( scope, "Column" );
}

View File

@ -11,6 +11,7 @@ import java.util.Map;
import org.hibernate.annotations.JdbcTypeCode;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.AbstractHANADialect;
import org.hibernate.dialect.DerbyDialect;
import org.hibernate.dialect.OracleDialect;
@ -140,6 +141,7 @@ public abstract class XmlMappingTests {
@SkipForDialect(dialectClass = AbstractHANADialect.class, matchSubTypes = true, reason = "HANA doesn't support comparing LOBs with the = operator")
@SkipForDialect(dialectClass = SybaseDialect.class, matchSubTypes = true, reason = "Sybase doesn't support comparing LOBs with the = operator")
@SkipForDialect(dialectClass = OracleDialect.class, matchSubTypes = true, reason = "Oracle doesn't support comparing JSON with the = operator")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Altibase doesn't support comparing CLOBs with the = operator")
public void verifyComparisonWorks(SessionFactoryScope scope) {
scope.inTransaction(
(session) -> {

View File

@ -4,6 +4,7 @@ import jakarta.persistence.Column;
import jakarta.persistence.Entity;
import jakarta.persistence.Id;
import org.hibernate.annotations.GeneratedColumn;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.DerbyDialect;
import org.hibernate.dialect.H2Dialect;
import org.hibernate.dialect.HSQLDialect;
@ -31,6 +32,7 @@ import static org.junit.Assert.assertEquals;
@SkipForDialect(dialectClass = SybaseASEDialect.class)
@SkipForDialect(dialectClass = PostgreSQLDialect.class, majorVersion = 10, matchSubTypes = true)
@SkipForDialect(dialectClass = PostgreSQLDialect.class, majorVersion = 11, matchSubTypes = true) // 'generated always' was added in 12
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "generated always is not supported in Altibase")
public class GeneratedAlwaysTest {
@Test

View File

@ -9,11 +9,13 @@ package org.hibernate.orm.test.mapping.where;
import java.util.Set;
import org.hibernate.annotations.SQLRestriction;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.testing.orm.junit.DomainModel;
import org.hibernate.testing.orm.junit.JiraKey;
import org.hibernate.testing.orm.junit.SessionFactory;
import org.hibernate.testing.orm.junit.SessionFactoryScope;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.junit.jupiter.api.Test;
import jakarta.persistence.DiscriminatorColumn;
@ -33,6 +35,7 @@ import jakarta.persistence.Table;
})
@SessionFactory
@JiraKey( "https://hibernate.atlassian.net/browse/HHH-14977" )
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "'TYPE' is not escaped even though autoQuoteKeywords is enabled")
public class DiscriminatorWhereTest {
@Test
public void testAddDiscriminatedEntityToCollectionWithWhere(SessionFactoryScope scope) {

View File

@ -13,9 +13,11 @@ import org.hibernate.annotations.NotFoundAction;
import org.hibernate.boot.registry.StandardServiceRegistryBuilder;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.testing.jdbc.SQLStatementInspector;
import org.hibernate.testing.junit4.BaseNonConfigCoreFunctionalTestCase;
import org.hibernate.testing.orm.junit.Jira;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
@ -253,6 +255,7 @@ public class IsNullAndNotFoundTest extends BaseNonConfigCoreFunctionalTestCase {
@Test
@Jira( "https://hibernate.atlassian.net/browse/HHH-17384" )
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "left join cannot be used inside exists clause")
public void testDeleteAdditionalPredicate() {
inspector.clear();

View File

@ -10,6 +10,7 @@ import java.util.Date;
import java.util.List;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.DB2Dialect;
import org.hibernate.dialect.SQLServerDialect;
import org.hibernate.query.criteria.HibernateCriteriaBuilder;
@ -214,6 +215,7 @@ public class CriteriaWindowFunctionTest {
@Test
@SkipForDialect(dialectClass = DB2Dialect.class, majorVersion = 10, reason = "No support for percent_rank and cume_dist functions")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "No support for percent_rank and cume_dist functions with over clause")
public void testReusableWindow(SessionFactoryScope scope) {
scope.inTransaction(
session -> {

View File

@ -7,6 +7,7 @@
package org.hibernate.orm.test.query.hql;
import org.hibernate.QueryException;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.CockroachDialect;
import org.hibernate.dialect.DB2Dialect;
import org.hibernate.dialect.DerbyDialect;
@ -980,6 +981,7 @@ public class FunctionTests {
@SkipForDialect(dialectClass = DB2Dialect.class, matchSubTypes = true)
@SkipForDialect(dialectClass = DerbyDialect.class)
@SkipForDialect(dialectClass = SybaseDialect.class, matchSubTypes = true)
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Altibase does not support offset of datetime")
public void testCastToOffsetDatetime(SessionFactoryScope scope) {
scope.inTransaction( session -> {
session.createQuery("select cast(datetime 1911-10-09 12:13:14-02:00 as String)", String.class).getSingleResult();
@ -1007,6 +1009,7 @@ public class FunctionTests {
@SkipForDialect(dialectClass = DerbyDialect.class, reason = "Derby doesn't support casting to the binary types")
@SkipForDialect(dialectClass = OracleDialect.class, reason = "Oracle treats the cast value as a hexadecimal literal")
@SkipForDialect(dialectClass = HSQLDialect.class, reason = "HSQL treats the cast value as a hexadecimal literal")
@SkipForDialect(dialectClass = AltibaseDialect.class, reason = "Altibase doesn't support casting varchar to binary")
public void testCastFunctionBinary(SessionFactoryScope scope) {
scope.inTransaction(
session -> {
@ -1030,6 +1033,7 @@ public class FunctionTests {
}
@Test
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "Altibase cast to char does not do truncatation")
public void testCastFunctionWithLength(SessionFactoryScope scope) {
scope.inTransaction(
session -> {
@ -1051,6 +1055,7 @@ public class FunctionTests {
@SkipForDialect(dialectClass = CockroachDialect.class, matchSubTypes = true, reason = "CockroachDB bytes doesn't have a length")
@SkipForDialect(dialectClass = OracleDialect.class, reason = "Oracle cast to raw does not do truncatation")
@SkipForDialect(dialectClass = DB2Dialect.class, majorVersion = 10, minorVersion = 5, reason = "On this version the length of the cast to the parameter appears to be > 2")
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "Altibase cast to raw does not do truncatation")
@SkipForDialect(dialectClass = HSQLDialect.class, reason = "HSQL interprets string as hex literal and produces error")
public void testCastBinaryWithLength(SessionFactoryScope scope) {
scope.inTransaction(
@ -1546,6 +1551,7 @@ public class FunctionTests {
@Test
@SkipForDialect( dialectClass = TiDBDialect.class, reason = "Bug in the TiDB timestampadd function (https://github.com/pingcap/tidb/issues/41052)")
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "Altibase returns 2025-03-31 as a result of select {2024-02-29} + 13 month")
public void testDurationArithmetic(SessionFactoryScope scope) {
scope.inTransaction(
session -> {

View File

@ -7,6 +7,7 @@
package org.hibernate.orm.test.query.hql;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.SybaseASEDialect;
import org.hibernate.jpa.spi.MutableJpaCompliance;
@ -48,6 +49,7 @@ public class JpaCrossJoinTests {
}
@Test
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "Altibase dialect emulate cross join with inner join")
public void test2Roots(SessionFactoryScope scope) {
final SQLStatementInspector statementInspector = scope.getCollectingStatementInspector();
statementInspector.clear();
@ -60,6 +62,7 @@ public class JpaCrossJoinTests {
}
@Test
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "Altibase dialect emulate cross join with inner join")
public void test2Roots2(SessionFactoryScope scope) {
final SQLStatementInspector statementInspector = scope.getCollectingStatementInspector();
statementInspector.clear();

View File

@ -15,6 +15,7 @@ import jakarta.persistence.Entity;
import jakarta.persistence.Id;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.community.dialect.FirebirdDialect;
import org.hibernate.dialect.DB2Dialect;
import org.hibernate.dialect.DerbyDialect;
@ -29,6 +30,7 @@ import org.junit.Test;
@SkipForDialect(value = DB2Dialect.class, comment = "DB2 is far more resistant to the reserved keyword usage. See HHH-12832.")
@SkipForDialect(value = DerbyDialect.class, comment = "Derby is far more resistant to the reserved keyword usage.")
@SkipForDialect(value = FirebirdDialect.class, comment = "FirebirdDialect has autoQuoteKeywords enabled, so it is far more resistant to the reserved keyword usage.")
@SkipForDialect(value = AltibaseDialect.class, comment = "AltibaseDialect has autoQuoteKeywords enabled, so it is far more resistant to the reserved keyword usage.")
public class SchemaMigratorHaltOnErrorTest extends BaseEntityManagerFunctionalTestCase {
@Override

View File

@ -20,6 +20,7 @@ import org.hibernate.boot.MetadataSources;
import org.hibernate.boot.registry.StandardServiceRegistry;
import org.hibernate.boot.registry.StandardServiceRegistryBuilder;
import org.hibernate.boot.spi.MetadataImplementor;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.community.dialect.FirebirdDialect;
import org.hibernate.dialect.DB2Dialect;
import org.hibernate.testing.SkipForDialect;
@ -42,6 +43,7 @@ import org.junit.runner.RunWith;
@SkipForDialect(value = DB2Dialect.class, comment = "DB2 is far more resistant to the reserved keyword usage. See HHH-12832.")
@SkipForDialect(value = DerbyDialect.class, comment = "Derby is far more resistant to the reserved keyword usage.")
@SkipForDialect(value = FirebirdDialect.class, comment = "FirebirdDialect has autoQuoteKeywords enabled, so it is far more resistant to the reserved keyword usage.")
@SkipForDialect(value = AltibaseDialect.class, comment = "AltibaseDialect has autoQuoteKeywords enabled, so it is far more resistant to the reserved keyword usage.")
@RunWith(CustomRunner.class)
public class SchemaUpdateHaltOnErrorTest {

View File

@ -17,11 +17,13 @@ import java.util.List;
import org.hibernate.Session;
import org.hibernate.boot.model.naming.ImplicitNamingStrategyJpaCompliantImpl;
import org.hibernate.cfg.Configuration;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.engine.spi.SessionImplementor;
import org.hibernate.jdbc.Work;
import org.hibernate.testing.junit4.BaseCoreFunctionalTestCase;
import org.hibernate.testing.orm.junit.JiraKey;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.junit.Assert;
import org.junit.Test;
@ -142,6 +144,7 @@ public class AutoDiscoveryTest extends BaseCoreFunctionalTestCase {
@Test
@JiraKey( "HHH-16697" )
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "Altibase sum(39.74) returns Float" )
public void testAggregateQueryAutoDiscovery() {
Session session = openSession();
session.beginTransaction();

View File

@ -7,8 +7,8 @@
package org.hibernate.orm.test.type;
import java.util.Map;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.AbstractHANADialect;
import org.hibernate.dialect.DB2Dialect;
import org.hibernate.dialect.DerbyDialect;
@ -77,6 +77,7 @@ public class EnumArrayTest extends BaseNonConfigCoreFunctionalTestCase {
}
@Test
@SkipForDialect(value = AltibaseDialect.class, comment = "When length 0 byte array is inserted, Altibase returns with null")
public void testById() {
inSession( em -> {
TableWithEnumArrays tableRecord;
@ -103,6 +104,7 @@ public class EnumArrayTest extends BaseNonConfigCoreFunctionalTestCase {
@Test
@SkipForDialect( value = AbstractHANADialect.class, comment = "For some reason, HANA can't intersect VARBINARY values, but funnily can do a union...")
@SkipForDialect(value = AltibaseDialect.class, comment = "When length 0 byte array is inserted, Altibase returns with null")
public void testQuery() {
inSession( em -> {
TypedQuery<TableWithEnumArrays> tq = em.createNamedQuery( "TableWithEnumArrays.JPQL.getByData", TableWithEnumArrays.class );

View File

@ -4,6 +4,7 @@ import java.util.Comparator;
import java.util.Locale;
import org.hibernate.annotations.JavaType;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.metamodel.spi.MappingMetamodelImplementor;
import org.hibernate.persister.entity.EntityPersister;
import org.hibernate.type.descriptor.WrapperOptions;
@ -16,6 +17,7 @@ import org.hibernate.testing.orm.junit.DomainModel;
import org.hibernate.testing.orm.junit.JiraKey;
import org.hibernate.testing.orm.junit.SessionFactory;
import org.hibernate.testing.orm.junit.SessionFactoryScope;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.junit.jupiter.api.Test;
import jakarta.persistence.Entity;
@ -62,6 +64,7 @@ public class OverrideStandardJavaTypeTest {
}
@Test
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "'LANGUAGE' is not escaped even though autoQuoteKeywords is enabled")
public void validateNative(SessionFactoryScope scope) {
final var id = scope.fromTransaction(
session -> {

View File

@ -15,6 +15,7 @@ import jakarta.persistence.Query;
import jakarta.transaction.Status;
import jakarta.transaction.TransactionManager;
import org.hibernate.cfg.AvailableSettings;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.CockroachDialect;
import org.hibernate.dialect.OracleDialect;
import org.hibernate.envers.RevisionType;
@ -41,7 +42,9 @@ import static org.junit.Assert.assertNotNull;
*/
@TestForIssue(jiraKey = "HHH-6349")
@SkipForDialect(value = OracleDialect.class,
comment = "Oracle does not support identity key generation")
comment = "Oracle does not support identity key generation")
@SkipForDialect(value = AltibaseDialect.class,
comment = "Altibase does not support identity key generation")
public class DetachedMultipleCollectionChangeTest extends BaseEnversJPAFunctionalTestCase {
private TransactionManager tm = null;

View File

@ -11,11 +11,13 @@ import java.util.Iterator;
import java.util.List;
import jakarta.persistence.EntityManager;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.orm.test.envers.BaseEnversJPAFunctionalTestCase;
import org.hibernate.orm.test.envers.Priority;
import org.hibernate.mapping.Formula;
import org.hibernate.mapping.PersistentClass;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.junit.Assert;
import org.junit.Test;
@ -107,6 +109,7 @@ public class DiscriminatorFormulaTest extends BaseEnversJPAFunctionalTestCase {
}
@Test
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "'TYPE' is not escaped even though autoQuoteKeywords is enabled")
public void testRevisionsCounts() {
Assert.assertEquals(
Arrays.asList( 1, 3 ), getAuditReader().getRevisions(
@ -123,18 +126,21 @@ public class DiscriminatorFormulaTest extends BaseEnversJPAFunctionalTestCase {
}
@Test
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "'TYPE' is not escaped even though autoQuoteKeywords is enabled")
public void testHistoryOfParent() {
Assert.assertEquals( parentVer1, getAuditReader().find( ParentEntity.class, parentVer1.getId(), 2 ) );
Assert.assertEquals( parentVer2, getAuditReader().find( ParentEntity.class, parentVer2.getId(), 4 ) );
}
@Test
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "'TYPE' is not escaped even though autoQuoteKeywords is enabled")
public void testHistoryOfChild() {
Assert.assertEquals( childVer1, getAuditReader().find( ChildEntity.class, childVer1.getId(), 1 ) );
Assert.assertEquals( childVer2, getAuditReader().find( ChildEntity.class, childVer2.getId(), 3 ) );
}
@Test
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "'TYPE' is not escaped even though autoQuoteKeywords is enabled")
public void testPolymorphicQuery() {
Assert.assertEquals(
childVer1, getAuditReader().createQuery()

View File

@ -9,6 +9,7 @@ package org.hibernate.orm.test.envers.integration.lazy;
import java.util.List;
import jakarta.persistence.EntityManager;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.OracleDialect;
import org.hibernate.envers.query.AuditEntity;
import org.hibernate.orm.test.envers.BaseEnversJPAFunctionalTestCase;
@ -34,6 +35,7 @@ import org.hibernate.testing.bytecode.enhancement.EnhancementOptions;
@RunWith(BytecodeEnhancerRunner.class)
@EnhancementOptions(lazyLoading = true)
@SkipForDialect(value = OracleDialect.class, comment = "Oracle does not support identity key generation")
@SkipForDialect(value = AltibaseDialect.class, comment = "Altibase does not support identity key generation")
public class IsCollectionInitializedBytecodeEnhancementTest extends BaseEnversJPAFunctionalTestCase {
private Long mce1Id = null;

View File

@ -9,6 +9,7 @@ package org.hibernate.orm.test.envers.integration.lazy;
import java.util.List;
import jakarta.persistence.EntityManager;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.OracleDialect;
import org.hibernate.envers.query.AuditEntity;
import org.hibernate.orm.test.envers.BaseEnversJPAFunctionalTestCase;
@ -29,6 +30,7 @@ import org.hibernate.envers.AuditReaderFactory;
*/
@TestForIssue(jiraKey = "HHH-15522")
@SkipForDialect(value = OracleDialect.class, comment = "Oracle does not support identity key generation")
@SkipForDialect(value = AltibaseDialect.class, comment = "Altibase does not support identity key generation")
public class IsCollectionInitializedTest extends BaseEnversJPAFunctionalTestCase {
private Long mce1Id = null;

View File

@ -21,12 +21,14 @@ import jakarta.persistence.Table;
import org.hibernate.annotations.SQLRestriction;
import org.hibernate.annotations.Where;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.envers.AuditJoinTable;
import org.hibernate.envers.Audited;
import org.hibernate.envers.RelationTargetAuditMode;
import org.hibernate.orm.test.envers.BaseEnversJPAFunctionalTestCase;
import org.hibernate.orm.test.envers.Priority;
import org.hibernate.testing.TestForIssue;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.hibernate.testing.transaction.TransactionUtil;
import org.junit.Test;
@ -43,6 +45,7 @@ import static org.junit.Assert.assertEquals;
* @author Chris Cranford
*/
@TestForIssue(jiraKey = "HHH-9432")
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "'TYPE' is not escaped even though autoQuoteKeywords is enabled")
public class BasicWhereTest extends BaseEnversJPAFunctionalTestCase {
private Integer aId;
private Integer xId;

View File

@ -10,6 +10,7 @@ import java.util.ArrayList;
import java.util.List;
import jakarta.persistence.EntityManager;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.OracleDialect;
import org.hibernate.orm.test.envers.Priority;
import org.hibernate.orm.test.envers.entities.collection.MultipleCollectionEntity;
@ -29,7 +30,9 @@ import static org.junit.Assert.assertEquals;
*/
@TestForIssue(jiraKey = "HHH-7437")
@SkipForDialect(value = OracleDialect.class,
comment = "Oracle does not support identity key generation")
comment = "Oracle does not support identity key generation")
@SkipForDialect(value = AltibaseDialect.class,
comment = "Altibase does not support identity key generation")
public class HasChangedDetachedMultipleCollection extends AbstractModifiedFlagsEntityTest {
private Long mce1Id = null;
private Long mce2Id = null;

View File

@ -21,12 +21,14 @@ import jakarta.persistence.Table;
import org.hibernate.annotations.SQLRestriction;
import org.hibernate.annotations.Where;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.envers.AuditJoinTable;
import org.hibernate.envers.Audited;
import org.hibernate.envers.RelationTargetAuditMode;
import org.hibernate.orm.test.envers.BaseEnversJPAFunctionalTestCase;
import org.hibernate.orm.test.envers.Priority;
import org.hibernate.testing.TestForIssue;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.hibernate.testing.transaction.TransactionUtil;
import org.junit.Test;
@ -45,6 +47,7 @@ import static org.junit.Assert.assertEquals;
* @author Chris Cranford
*/
@TestForIssue(jiraKey = "HHH-9432")
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "'TYPE' is not escaped even though autoQuoteKeywords is enabled")
public class BasicWhereTest extends BaseEnversJPAFunctionalTestCase {
private Integer aId;
private Integer xId;

View File

@ -9,6 +9,7 @@ package org.hibernate.orm.test.envers.integration.onetomany.embeddedid;
import java.util.Arrays;
import jakarta.persistence.EntityManager;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.orm.test.envers.BaseEnversJPAFunctionalTestCase;
import org.hibernate.orm.test.envers.Priority;
import org.hibernate.orm.test.envers.integration.onetomany.embeddedid.Constant;
@ -16,6 +17,7 @@ import org.hibernate.orm.test.envers.integration.onetomany.embeddedid.Person;
import org.hibernate.orm.test.envers.integration.onetomany.embeddedid.PersonTuple;
import org.hibernate.testing.TestForIssue;
import org.hibernate.testing.orm.junit.SkipForDialect;
import org.junit.Assert;
import org.junit.Test;
@ -23,6 +25,7 @@ import org.junit.Test;
* @author Lukasz Antoniak (lukasz dot antoniak at gmail dot com)
*/
@TestForIssue(jiraKey = "HHH-7157")
@SkipForDialect( dialectClass = AltibaseDialect.class, reason = "'CONSTANT' is not escaped even though autoQuoteKeywords is enabled")
public class MapsIdTest extends BaseEnversJPAFunctionalTestCase {
private PersonTuple tuple1Ver1 = null;
private PersonTuple tuple2Ver1 = null;

View File

@ -6,6 +6,7 @@
*/
package org.hibernate.test.hikaricp;
import org.hibernate.community.dialect.AltibaseDialect;
import org.hibernate.dialect.SybaseDialect;
import org.hibernate.dialect.TiDBDialect;
import org.hibernate.engine.jdbc.connections.spi.ConnectionProvider;
@ -19,6 +20,7 @@ import org.hibernate.testing.common.connections.BaseTransactionIsolationConfigTe
*/
@SkipForDialect(value = SybaseDialect.class, comment = "The jTDS driver doesn't implement Connection#getNetworkTimeout() so this fails")
@SkipForDialect(value = TiDBDialect.class, comment = "Doesn't support SERIALIZABLE isolation")
@SkipForDialect(value = AltibaseDialect.class, comment = "Altibase cannot change isolation level in autocommit mode")
public class HikariTransactionIsolationConfigTest extends BaseTransactionIsolationConfigTest {
@Override
protected ConnectionProvider getConnectionProviderUnderTest() {

View File

@ -230,6 +230,7 @@ dependencyResolutionManagement {
def pgsqlVersion = version "pgsql", "42.6.0"
def sybaseVersion = version "sybase", "1.3.1"
def tidbVersion = version "tidb", mysqlVersion
def altibaseVersion = version "altibase", "7.3.0.0.2"
library( "h2", "com.h2database", "h2" ).versionRef( h2Version )
library( "h2gis", "org.orbisgis", "h2gis" ).versionRef( h2gisVersion )
@ -249,6 +250,7 @@ dependencyResolutionManagement {
library( "sybase", "net.sourceforge.jtds", "jtds" ).versionRef( sybaseVersion )
library( "informix", "com.ibm.informix", "jdbc" ).versionRef( informixVersion )
library( "firebird", "org.firebirdsql.jdbc", "jaybird" ).versionRef( firebirdVersion )
library( "altibase", "com.altibase", "altibase-jdbc" ).versionRef( altibaseVersion )
}
mavenLibs {
def mavenCoreVersion = version "mavenCore", "3.8.1"