DBZ-7359 Added setQueryColumnValue fix and supporting Integration Tests

This commit is contained in:
Ciaran O'Reilly 2024-02-01 20:39:02 +00:00 committed by Jiri Pechanec
parent e8c3dbc571
commit fefcd4e79c
3 changed files with 1025 additions and 0 deletions

View File

@ -6,8 +6,12 @@
package io.debezium.connector.sqlserver;
import java.io.Reader;
import java.io.StringReader;
import java.sql.Connection;
import java.sql.DatabaseMetaData;
import java.sql.NClob;
import java.sql.PreparedStatement;
import java.sql.ResultSet;
import java.sql.ResultSetMetaData;
import java.sql.SQLException;
@ -605,6 +609,61 @@ public Object getColumnValue(ResultSet rs, int columnIndex, Column column, Table
}
}
// NOTE: fix for DBZ-7359
// @Override
public void setQueryColumnValue(PreparedStatement statement, Column column, int pos, Object value) throws SQLException {
boolean isColumnValueSet = false;
if (column.typeUsesCharset()) {
// For mappings between sqlserver and JDBC types see -
// https://learn.microsoft.com/en-us/sql/connect/jdbc/using-basic-data-types?view=sql-server-ver16
// For details on the methods to use with respect to the `sendStringParametersAsUnicode` JDBC property, see -
// https://learn.microsoft.com/en-us/sql/connect/jdbc/setting-the-connection-properties?view=sql-server-ver16
// "An application should use the setNString, setNCharacterStream, and setNClob national character methods
// of the SQLServerPreparedStatement and SQLServerCallableStatement classes for the NCHAR, NVARCHAR, and
// LONGNVARCHAR JDBC data types."
switch (column.jdbcType()) {
case Types.NCHAR:
if (value instanceof String) {
statement.setNString(pos, (String) value);
isColumnValueSet = true;
}
break;
case Types.NVARCHAR:
if (value instanceof String) {
statement.setNCharacterStream(pos, new StringReader((String) value));
isColumnValueSet = true;
}
else if (value instanceof Reader) {
statement.setNCharacterStream(pos, (Reader) value);
isColumnValueSet = true;
}
break;
case Types.LONGNVARCHAR:
if (value instanceof String) {
// we'll fall back on nvarchar handling
statement.setNCharacterStream(pos, new StringReader((String) value));
isColumnValueSet = true;
}
else if (value instanceof Reader) {
// we'll fall back on nvarchar handling
statement.setNCharacterStream(pos, (Reader) value);
isColumnValueSet = true;
}
else if (value instanceof NClob) {
statement.setNClob(pos, (NClob) value);
isColumnValueSet = true;
}
break;
}
}
// If not set, fall back on default implementation.
if (!isColumnValueSet) {
super.setQueryColumnValue(statement, column, pos, value);
}
}
@Override
public String buildSelectWithRowLimits(TableId tableId, int limit, String projection, Optional<String> condition,
Optional<String> additionalCondition, String orderBy) {

View File

@ -0,0 +1,391 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.sqlserver;
import static org.assertj.core.api.Assertions.assertThat;
import java.io.BufferedReader;
import java.io.IOException;
import java.io.InputStreamReader;
import java.sql.SQLException;
import java.util.*;
import java.util.function.BiFunction;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.function.Predicate;
import org.apache.kafka.connect.data.Struct;
import org.apache.kafka.connect.source.SourceRecord;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import io.debezium.config.CommonConnectorConfig;
import io.debezium.config.Configuration;
import io.debezium.connector.sqlserver.util.TestHelper;
import io.debezium.jdbc.JdbcConnection;
import io.debezium.pipeline.source.snapshot.incremental.AbstractSnapshotTest;
import io.debezium.util.Testing;
/**
* Tests for <a href="https://issues.redhat.com/browse/DBZ-7359">DBZ-7359</a>
*/
public class IncrementalSnapshotCollationSortOrderMismatchIT extends AbstractSnapshotTest<SqlServerConnector> {
private static final int POLLING_INTERVAL = 1;
// Sort order of SQL_ collation does not correspond with Unicode
// see - https://learn.microsoft.com/en-us/sql/relational-databases/collations/collation-and-unicode-support?view=sql-server-ver16#SQL-collations
private static final String SQL_COLLATION = "SQL_Latin1_General_CP1_CI_AS";
// IDs that simulate the sort order mismatch between the SQL_ collation and Unicode.
private static final List<String> ALL_IDS = new ArrayList<>();
// Contains the IDs that get skipped when there is a sort order mismatch.
private static final List<String> SKIPPED_IDS = new ArrayList<>();
private SqlServerConnection connection;
private boolean isSendStringParametersAsUnicode;
@BeforeClass
public static void beforeClass() throws IOException {
try (InputStreamReader isr = new InputStreamReader(
Objects.requireNonNull(IncrementalSnapshotCollationSortOrderMismatchIT.class.getClassLoader().getResourceAsStream("dbz-7359-ids.txt")))) {
try (BufferedReader reader = new BufferedReader(isr)) {
String line;
while ((line = reader.readLine()) != null) {
ALL_IDS.add(line);
}
}
// Of the IDS loaded, the 36 records with ids between Y-11-3-4 and Y1-01-1-1 exclusive, would be consistently skipped.
SKIPPED_IDS.addAll(ALL_IDS.subList(ALL_IDS.indexOf("Y-11-3-4") + 1, ALL_IDS.indexOf("Y1-01-1-1")));
}
}
//
// PK CHAR
@Test
public void orderMismatchPkCharValueIntParamsAsUnicodeFalse() throws Exception {
orderMismatchPkTypecharValueInt(false, ALL_IDS.size(), "char(50) COLLATE " + SQL_COLLATION);
}
@Test
public void orderMismatchPkCharValueIntParamsAsUnicodeTrueSkip36() throws Exception {
orderMismatchPkTypecharValueInt(true, ALL_IDS.size() - SKIPPED_IDS.size(), "char(50) COLLATE " + SQL_COLLATION);
}
//
// PK TEXT - Cannot be used as a PK column so not tested.
//
// PK VARCHAR
@Test
public void orderMismatchPkVarcharValueIntParamsAsUnicodeFalse() throws Exception {
orderMismatchPkTypecharValueInt(false, ALL_IDS.size(), "varchar(50) COLLATE " + SQL_COLLATION);
}
@Test
public void orderMismatchPkVarcharValueIntParamsAsUnicodeTrueSkip36() throws Exception {
orderMismatchPkTypecharValueInt(true, ALL_IDS.size() - SKIPPED_IDS.size(), "varchar(50) COLLATE " + SQL_COLLATION);
}
// Ensure unicode values are read back ok
@Test
public void orderMismatchPkVarcharValueNvarcharParamsAsUnicodeFalse() throws Exception {
orderMismatchPkVarcharValueNvarchar(false, ALL_IDS.size());
}
@Test
public void orderMismatchPkVarcharValueNvarcharParamsAsUnicodeTrueSkip36() throws Exception {
orderMismatchPkVarcharValueNvarchar(true, ALL_IDS.size() - SKIPPED_IDS.size());
}
//
// PK - NCHAR
@Test
public void orderMismatchPkNcharValueNvarcharParamsAsUnicodeFalse() throws Exception {
orderMismatchPkNtypeValueNvarchar(false, ALL_IDS.size(), "nchar(50)");
}
@Test
public void orderMismatchPkNcharValueNvarcharParamsAsUnicodeTrue() throws Exception {
orderMismatchPkNtypeValueNvarchar(true, ALL_IDS.size(), "nchar(50)");
}
//
// PK - NTEXT - Cannot be used as a PK column so not tested.
//
// PK - NVARCHAR
@Test
public void orderMismatchPkNvarcharValueNvarcharParamsAsUnicodeFalse() throws Exception {
orderMismatchPkNtypeValueNvarchar(false, ALL_IDS.size(), "nvarchar(50)");
}
@Test
public void orderMismatchPkNvarcharValueNvarcharParamsAsUnicodeTrue() throws Exception {
orderMismatchPkNtypeValueNvarchar(true, ALL_IDS.size(), "nvarchar(50)");
}
protected void orderMismatchPkTypecharValueInt(final boolean isSendStringParametersAsUnicode, int expectedRecordCount, String pkDataType) throws Exception {
runTest(isSendStringParametersAsUnicode,
expectedRecordCount,
pkDataType,
(pk, _idx) -> String.format("'%s'", pk),
// NOTE: we use .trim() to handle the padding added when a non-variable length type is used.
pk -> pk.getString(pkFieldName()).trim(),
"int",
(_pk, idx) -> String.format("%d", idx),
record -> ((Struct) record.value()).getStruct("after").getInt32(valueFieldName()),
dbChanges -> {
boolean result = true;
for (int i = 0; i < ALL_IDS.size(); i++) {
var id = ALL_IDS.get(i);
// NOTE: in the case where isSendStringParametersAsUnicode is true, the 36 records will
// have been skipped (i.e. you need to set it to false for it to work).
if (isSendStringParametersAsUnicode && SKIPPED_IDS.contains(id)) {
continue;
}
var val = dbChanges.get(id);
if (val == null || val != i) {
result = false;
System.err.println(ALL_IDS.get(i) + " value is not = " + i + ", is = " + val);
break;
}
}
return result;
});
}
protected void orderMismatchPkVarcharValueNvarchar(boolean isSendStringParametersAsUnicode, int expectedRecordCount) throws Exception {
runTest(isSendStringParametersAsUnicode,
expectedRecordCount,
"varchar(50) COLLATE " + SQL_COLLATION,
(pk, _idx) -> String.format("'%s'", pk),
pk -> pk.getString(pkFieldName()),
"nvarchar(100) not null",
(_pk, idx) -> String.format("N'%d Hiragana: の, は, でした, Katakana: コンサート, Kanji: 昨夜, 最高'", idx),
record -> ((Struct) record.value()).getStruct("after").getString(valueFieldName()),
dbChanges -> {
boolean result = true;
for (int i = 0; i < ALL_IDS.size(); i++) {
var id = ALL_IDS.get(i);
// NOTE: in the case where isSendStringParametersAsUnicode is true, the 36 records will
// have been skipped (i.e. you need to set it to false for it to work).
if (isSendStringParametersAsUnicode && SKIPPED_IDS.contains(id)) {
continue;
}
var expectedVal = String.format("%d Hiragana: の, は, でした, Katakana: コンサート, Kanji: 昨夜, 最高", i);
var val = dbChanges.get(ALL_IDS.get(i));
if (!expectedVal.equals(val)) {
result = false;
System.err.println(ALL_IDS.get(i) + " value is not = " + expectedVal + ", is = " + val);
break;
}
}
return result;
});
}
protected void orderMismatchPkNtypeValueNvarchar(boolean isSendStringParametersAsUnicode, int expectedRecordCount, String pkDataType) throws Exception {
runTest(isSendStringParametersAsUnicode,
expectedRecordCount,
pkDataType,
(pk, _idx) -> String.format("N'の %s'", pk),
// NOTE: we use .trim() to handle the padding added when a non-variable length type is used.
pk -> pk.getString(pkFieldName()).trim(),
"nvarchar(100) not null",
(_pk, idx) -> String.format("N'%d Hiragana: の, は, でした, Katakana: コンサート, Kanji: 昨夜, 最高'", idx),
record -> ((Struct) record.value()).getStruct("after").getString(valueFieldName()),
dbChanges -> {
boolean result = true;
for (int i = 0; i < ALL_IDS.size(); i++) {
var expectedVal = String.format("%d Hiragana: の, は, でした, Katakana: コンサート, Kanji: 昨夜, 最高", i);
var val = dbChanges.get(String.format("の %s", ALL_IDS.get(i)));
if (!expectedVal.equals(val)) {
result = false;
System.err.println(ALL_IDS.get(i) + " value is not = " + expectedVal + ", is = " + val);
break;
}
}
return result;
});
}
protected <P, V> void runTest(boolean isSendStringParametersAsUnicode,
int expectedRecordCount,
String pkDataType,
BiFunction<String, Integer, String> pkInsValFn,
Function<Struct, P> pkConverter,
String valueDataType,
BiFunction<String, Integer, String> valueInsValFn,
Function<SourceRecord, V> valueConverter,
Predicate<Map<P, V>> validateDbChanges)
throws Exception {
// NOTE: this value is referenced in the config() method below to ensure the property is
// configured properly when startConnector() is called.
this.isSendStringParametersAsUnicode = isSendStringParametersAsUnicode;
TestHelper.createTestDatabase();
try (SqlServerConnection connection = TestHelper.testConnection(TestHelper.TEST_DATABASE_1)) {
this.connection = connection;
connection.execute(
String.format("CREATE TABLE %s (%s %s primary key, %s %s)",
tableName(), pkFieldName(), pkDataType, valueFieldName(), valueDataType),
"CREATE TABLE debezium_signal (id varchar(64), type varchar(32), data varchar(2048))");
TestHelper.enableTableCdc(connection, "debezium_signal");
TestHelper.adjustCdcPollingInterval(connection, POLLING_INTERVAL);
initializeConnectorTestFramework();
Testing.Files.delete(TestHelper.SCHEMA_HISTORY_PATH);
populateTable(connection, pkInsValFn, valueInsValFn);
TestHelper.enableTableCdc(connection, tableName());
startConnector();
sendAdHocSnapshotSignal(tableName());
testIncrementalSnapshotConsumed(expectedRecordCount, pkConverter, valueConverter, validateDbChanges);
}
}
protected void populateTable(SqlServerConnection connection,
BiFunction<String, Integer, String> pkInsValFn,
BiFunction<String, Integer, String> valueInsValFn)
throws SQLException {
connection.setAutoCommit(false);
for (int i = 0; i < ALL_IDS.size(); i++) {
var id = ALL_IDS.get(i);
connection.executeWithoutCommitting(
String.format("INSERT INTO %s (%s, %s) VALUES(%s, %s)",
tableName(), pkFieldName(), valueFieldName(),
pkInsValFn.apply(id, i), valueInsValFn.apply(id, i)));
}
connection.commit();
}
protected <P, V> void testIncrementalSnapshotConsumed(int expectedRecordCount,
Function<Struct, P> pkConverter,
Function<SourceRecord, V> valueConverter,
Predicate<Map<P, V>> validateDbChanges)
throws InterruptedException {
final Map<P, V> dbChanges = consumeIncrementalSnapshot(
// recordCount
expectedRecordCount,
// dataCompleted
x -> true,
// pkConverter
pkConverter,
// valueConverter
valueConverter,
// topicName
topicName(),
// recordConsumer
null,
true);
assertThat(dbChanges).hasSize(expectedRecordCount);
Assert.assertTrue(validateDbChanges.test(dbChanges));
}
protected <P, V> Map<P, V> consumeIncrementalSnapshot(int recordCount,
Predicate<Map.Entry<P, V>> dataCompleted,
Function<Struct, P> pkConverter,
Function<SourceRecord, V> valueConverter,
String topicName,
Consumer<List<SourceRecord>> recordConsumer,
boolean assertRecords)
throws InterruptedException {
final Map<P, V> dbChanges = new HashMap<>();
int noRecords = 0;
for (;;) {
final SourceRecords records = consumeRecordsByTopic(1, assertRecords);
final List<SourceRecord> dataRecords = records.recordsForTopic(topicName);
if (records.allRecordsInOrder().isEmpty()) {
noRecords++;
assertThat(noRecords).describedAs(String.format("Too many no data record results, %d < %d", dbChanges.size(), recordCount))
.isLessThanOrEqualTo(5);
continue;
}
noRecords = 0;
if (dataRecords == null || dataRecords.isEmpty()) {
continue;
}
dataRecords.forEach(record -> {
final P id = pkConverter.apply((Struct) record.key());
final V value = valueConverter.apply(record);
dbChanges.put(id, value);
});
if (recordConsumer != null) {
recordConsumer.accept(dataRecords);
}
if (dbChanges.size() >= recordCount) {
if (dbChanges.entrySet().stream().noneMatch(dataCompleted.negate())) {
break;
}
}
}
return dbChanges;
}
@Override
protected Class<SqlServerConnector> connectorClass() {
return SqlServerConnector.class;
}
@Override
protected JdbcConnection databaseConnection() {
return connection;
}
@Override
protected String topicName() {
return "server1." + TestHelper.TEST_DATABASE_1 + ".dbo.c";
}
@Override
protected String tableName() {
return TestHelper.TEST_DATABASE_1 + ".dbo.c";
}
@Override
protected List<String> topicNames() {
throw new UnsupportedOperationException();
}
@Override
protected List<String> tableNames() {
throw new UnsupportedOperationException();
}
@Override
protected String signalTableName() {
return "dbo.debezium_signal";
}
@Override
protected Configuration.Builder config() {
return TestHelper.defaultConfig()
.with(CommonConnectorConfig.DATABASE_CONFIG_PREFIX + "sendStringParametersAsUnicode", isSendStringParametersAsUnicode)
.with(SqlServerConnectorConfig.SNAPSHOT_MODE, SqlServerConnectorConfig.SnapshotMode.SCHEMA_ONLY)
.with(SqlServerConnectorConfig.SIGNAL_DATA_COLLECTION, TestHelper.TEST_DATABASE_1 + ".dbo.debezium_signal")
.with(SqlServerConnectorConfig.INCREMENTAL_SNAPSHOT_CHUNK_SIZE, 250);
}
@Override
protected Configuration.Builder mutableConfig(boolean signalTableOnly, boolean storeOnlyCapturedDdl) {
throw new UnsupportedOperationException();
}
@Override
protected String connector() {
return "sql_server";
}
@Override
protected String server() {
return TestHelper.TEST_SERVER_NAME;
}
}

View File

@ -0,0 +1,575 @@
15-33-4-1
15-33-4-2
15-33-4-3
15-33-4-4
15-33-4-5
15-33-5-1
15-33-5-2
15-33-5-3
15-33-5-4
15-33-5-5
15-35-1-1
15-35-1-2
15-35-1-3
15-35-1-4
15-35-1-5
15-35-2-1
15-35-2-2
15-35-2-3
15-35-2-4
15-35-2-5
15-35-3-1
15-35-3-2
15-35-3-3
15-35-3-4
15-35-3-5
15-35-4-1
15-35-4-2
15-35-4-3
15-35-4-4
15-35-4-5
15-35-5-1
15-35-5-2
15-35-5-3
15-35-5-4
15-35-5-5
61-15-1-1
62-19-1-1
62-32-1-1
62-38-1-1
62-48-1-2
62-53-1-1
62-54-1-1
63-03-1-2
63-05-1-1
63-09-1-1
63-16-1-2
63-24-1-1
63-28-1-1
63-44-1-1
63-66-1-2
64-06-1-1
64-18-1-2
64-54-1-2
64-55-1-1
64-57-1-1
64-66-1-1
64-68-1-3
64-69-1-1
65-22-1-1
65-34-1-5
65-47-1-5
65-74-1-5
66-02-2-1
66-03-1-5
66-04-3-1
66-06-2-1
66-08-1-5
66-11-1-1
66-12-2-1
66-13-1-1
66-15-1-1
66-20-2-5
66-34-3-5
66-61-1-5
66-62-3-5
66-70-4-5
66-72-1-1
66-74-1-5
67-01-3-2
67-06-2-4
67-07-3-4
67-07-3-5
67-08-1-1
67-08-2-4
67-09-3-3
67-15-3-1
67-16-1-1
67-20-2-4
67-24-2-1
67-30-2-4
67-32-1-5
67-32-3-3
67-37-2-1
67-45-2-2
67-47-1-5
67-56-1-1
67-58-2-4
67-61-3-1
67-62-2-3
67-64-1-1
67-70-2-1
67-70-2-4
67-73-1-5
67-74-3-1
68-17-2-1
68-17-2-5
68-27-1-5
68-58-2-1
68-60-3-1
68-69-2-3
68-73-3-2
69-14-3-4
69-20-2-2
69-21-2-5
69-40-3-3
69-43-1-5
69-43-2-1
69-43-2-5
69-45-1-1
69-48-2-2
69-49-2-5
69-50-3-5
69-59-2-1
69-65-2-5
69-68-2-2
69-74-2-5
70-15-2-4
70-16-1-5
70-23-1-1
70-24-1-5
70-24-3-1
70-36-1-5
70-36-3-3
70-46-2-4
70-56-2-5
70-67-2-2
70-67-3-1
70-69-2-5
70-72-2-2
70-72-2-5
71-01-3-1
71-04-1-5
71-08-2-1
71-08-3-2
71-15-2-4
71-16-1-1
71-27-2-1
71-28-2-1
71-29-3-5
71-33-3-4
71-36-3-4
71-38-2-4
71-47-2-1
71-53-1-1
71-60-1-5
71-74-1-1
72-02-2-5
72-07-3-4
72-07-3-5
72-10-3-2
72-13-3-1
72-23-2-5
72-24-2-5
72-32-1-1
72-54-2-3
72-68-1-5
72-70-2-1
72-77-2-1
72-78-2-3
73-11-1-1
73-12-1-5
73-14-2-3
73-16-2-2
73-31-1-5
73-73-1-5
74-08-1-1
74-08-2-3
74-11-1-1
74-11-3-3
74-18-2-2
74-19-1-5
74-22-2-1
74-23-3-1
74-31-3-4
74-33-1-1
74-48-3-5
74-50-3-5
74-56-1-1
74-67-2-2
74-70-1-5
74-75-2-3
75-03-2-1
75-05-1-1
75-08-1-1
75-14-2-3
75-15-2-1
75-19-3-2
75-40-3-1
75-45-3-3
75-57-1-5
75-71-1-1
75-75-2-1
75-76-2-5
75-81-1-1
81-04-2-5
81-07-4-1
81-20-4-5
81-29-4-2
81-30-3-3
82-07-4-1
82-07-5-1
82-12-4-5
82-26-4-4
83-15-3-1
83-15-3-5
83-28-2-1
83-28-3-5
84-28-3-1
85-03-4-4
85-12-4-3
86-01-2-5
87-28-4-5
88-01-5-1
88-02-5-3
88-05-4-1
88-07-3-1
88-10-3-5
89-01-2-5
89-02-3-1
CHARGE001
DUPTOTE001
HOSPITAL
PACK001
PUTDROP
PUTHOSPITAL
START001
STARTPUT
WAIT001
WAIT010
Y-01-1-1
Y-01-1-2
Y-01-1-3
Y-01-1-4
Y-01-2-1
Y-01-2-2
Y-01-2-3
Y-01-2-4
Y-01-3-1
Y-01-3-2
Y-01-3-3
Y-01-3-4
Y-01-4-1
Y-01-4-2
Y-01-4-3
Y-01-4-4
Y-01-5-1
Y-01-5-2
Y-01-5-3
Y-01-5-4
Y-01-6-1
Y-01-6-2
Y-01-6-3
Y-01-6-4
Y-03-1-1
Y-03-1-2
Y-03-1-3
Y-03-1-4
Y-03-2-1
Y-03-2-2
Y-03-2-3
Y-03-2-4
Y-03-3-1
Y-03-3-2
Y-03-3-3
Y-03-3-4
Y-03-4-1
Y-03-4-2
Y-03-4-3
Y-03-4-4
Y-03-5-1
Y-03-5-2
Y-03-5-3
Y-03-5-4
Y-03-6-1
Y-03-6-2
Y-03-6-3
Y-03-6-4
Y-05-1-1
Y-05-1-2
Y-05-1-3
Y-05-1-4
Y-05-2-1
Y-05-2-2
Y-05-2-3
Y-05-2-4
Y-05-3-1
Y-05-3-2
Y-05-3-3
Y-05-3-4
Y-05-4-1
Y-05-4-2
Y-05-4-3
Y-05-4-4
Y-05-5-1
Y-05-5-2
Y-05-5-3
Y-05-5-4
Y-05-6-1
Y-05-6-2
Y-05-6-3
Y-05-6-4
Y-07-1-1
Y-07-1-2
Y-07-1-3
Y-07-1-4
Y-07-2-1
Y-07-2-2
Y-07-2-3
Y-07-2-4
Y-07-3-1
Y-07-3-2
Y-07-3-3
Y-07-3-4
Y-07-4-1
Y-07-4-2
Y-07-4-3
Y-07-4-4
Y-07-5-1
Y-07-5-2
Y-07-5-3
Y-07-5-4
Y-07-6-1
Y-07-6-2
Y-07-6-3
Y-07-6-4
Y-09-1-1
Y-09-1-2
Y-09-1-3
Y-09-1-4
Y-09-2-1
Y-09-2-2
Y-09-2-3
Y-09-2-4
Y-09-3-1
Y-09-3-2
Y-09-3-3
Y-09-3-4
Y-09-4-1
Y-09-4-2
Y-09-4-3
Y-09-4-4
Y-09-5-1
Y-09-5-2
Y-09-5-3
Y-09-5-4
Y-09-6-1
Y-09-6-2
Y-09-6-3
Y-09-6-4
Y-11-1-1
Y-11-1-2
Y-11-1-3
Y-11-1-4
Y-11-2-1
Y-11-2-2
Y-11-2-3
Y-11-2-4
Y-11-3-1
Y-11-3-2
Y-11-3-3
Y-11-3-4
Y-11-4-1
Y-11-4-2
Y-11-4-3
Y-11-4-4
Y-11-5-1
Y-11-5-2
Y-11-5-3
Y-11-5-4
Y-11-6-1
Y-11-6-2
Y-11-6-3
Y-11-6-4
Y-13-1-1
Y-13-1-2
Y-13-1-3
Y-13-1-4
Y-13-2-1
Y-13-2-2
Y-13-2-3
Y-13-2-4
Y-13-3-1
Y-13-3-2
Y-13-3-3
Y-13-3-4
Y-13-4-1
Y-13-4-2
Y-13-4-3
Y-13-4-4
Y-13-5-1
Y-13-5-2
Y-13-5-3
Y-13-5-4
Y-13-6-1
Y-13-6-2
Y-13-6-3
Y-13-6-4
Y1-01-1-1
Y1-01-1-2
Y1-01-1-3
Y1-01-1-4
Y1-01-2-1
Y1-01-2-2
Y1-01-2-3
Y1-01-2-4
Y1-01-3-1
Y1-01-3-2
Y1-01-3-3
Y1-01-3-4
Y1-01-4-1
Y1-01-4-2
Y1-01-4-3
Y1-01-4-4
Y1-01-5-1
Y1-01-5-2
Y1-01-5-3
Y1-01-5-4
Y1-01-6-1
Y1-01-6-2
Y1-01-6-3
Y1-01-6-4
Y1-03-1-1
Y1-03-1-2
Y1-03-1-3
Y1-03-1-4
Y1-03-2-1
Y1-03-2-2
Y1-03-2-3
Y1-03-2-4
Y1-03-3-1
Y1-03-3-2
Y1-03-3-3
Y1-03-3-4
Y1-03-4-1
Y1-03-4-2
Y1-03-4-3
Y1-03-4-4
Y1-03-5-1
Y1-03-5-2
Y1-03-5-3
Y1-03-5-4
Y1-03-6-1
Y1-03-6-2
Y1-03-6-3
Y1-03-6-4
Y1-05-1-1
Y1-05-1-2
Y1-05-1-3
Y1-05-1-4
Y1-05-2-1
Y1-05-2-2
Y1-05-2-3
Y1-05-2-4
Y1-05-3-1
Y1-05-3-2
Y1-05-3-3
Y1-05-3-4
Y1-05-4-1
Y1-05-4-2
Y1-05-4-3
Y1-05-4-4
Y1-05-5-1
Y1-05-5-2
Y1-05-5-3
Y1-05-5-4
Y1-05-6-1
Y1-05-6-2
Y1-05-6-3
Y1-05-6-4
Y1-07-1-1
Y1-07-1-2
Y1-07-1-3
Y1-07-1-4
Y1-07-2-1
Y1-07-2-2
Y1-07-2-3
Y1-07-2-4
Y1-07-3-1
Y1-07-3-2
Y1-07-3-3
Y1-07-3-4
Y1-07-4-1
Y1-07-4-2
Y1-07-4-3
Y1-07-4-4
Y1-07-5-1
Y1-07-5-2
Y1-07-5-3
Y1-07-5-4
Y1-07-6-1
Y1-07-6-2
Y1-07-6-3
Y1-07-6-4
Y1-09-1-1
Y1-09-1-2
Y1-09-1-3
Y1-09-1-4
Y1-09-2-1
Y1-09-2-2
Y1-09-2-3
Y1-09-2-4
Y1-09-3-1
Y1-09-3-2
Y1-09-3-3
Y1-09-3-4
Y1-09-4-1
Y1-09-4-2
Y1-09-4-3
Y1-09-4-4
Y1-09-5-1
Y1-09-5-2
Y1-09-5-3
Y1-09-5-4
Y1-09-6-1
Y1-09-6-2
Y1-09-6-3
Y1-09-6-4
Y1-11-1-1
Y1-11-1-2
Y1-11-1-3
Y1-11-1-4
Y1-11-2-1
Y1-11-2-2
Y1-11-2-3
Y1-11-2-4
Y1-11-3-1
Y1-11-3-2
Y1-11-3-3
Y1-11-3-4
Y1-11-4-1
Y1-11-4-2
Y1-11-4-3
Y1-11-4-4
Y1-11-5-1
Y1-11-5-2
Y1-11-5-3
Y1-11-5-4
Y1-11-6-1
Y1-11-6-2
Y1-11-6-3
Y1-11-6-4
Y1-13-1-1
Y1-13-1-2
Y1-13-1-3
Y1-13-1-4
Y1-13-2-1
Y1-13-2-2
Y1-13-2-3
Y1-13-2-4
Y1-13-3-1
Y1-13-3-2
Y1-13-3-3
Y1-13-3-4
Y1-13-4-1
Y1-13-4-2
Y1-13-4-3
Y1-13-4-4
Y1-13-5-1
Y1-13-5-2
Y1-13-5-3
Y1-13-5-4
Y1-13-6-1
Y1-13-6-2
Y1-13-6-3
Y1-13-6-4