[
https://issues.apache.org/jira/browse/PHOENIX-2925?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15295404#comment-15295404
]
Rajeshbabu Chintaguntla commented on PHOENIX-2925:
--------------------------------------------------
Here are some details about the bug
First in map we are collecting the column indexes map at mapper we use this to
write column indexes and corresponding values as aggregated byte array in
mapper.
{noformat}
private void initColumnsMap(PhoenixConnection conn) throws SQLException {
for (String tableName : logicalNames) {
PTable table = PhoenixRuntime.getTable(conn, tableName);
emptyFamilyName.add(SchemaUtil.getEmptyColumnFamilyPtr(table));
List<PColumn> cls = table.getColumns();
List<Pair<byte[], byte[]>> list = new ArrayList(cls.size());
for (int i = 0; i < cls.size(); i++) {
PColumn c = cls.get(i);
if (c.getFamilyName() == null) {
list.add(null); // Skip PK column
continue;
}
byte[] family = c.getFamilyName().getBytes();
byte[] name = c.getName().getBytes();
list.add(new Pair(family, name));
}
columnIndexes.add(list);
}
}
{noformat}
When we write to map output we are writing physical table name
{noformat}
context.write(new TableRowkeyPair(tableNames.get(tableIndex),
outputKey), aggregatedArray);
{noformat}
At reducer when we try to get columns indexes from physical table name we
always get column indexes of first index shared by the physical table.
{noformat}
int tableIndex = tableNames.indexOf(key.getTableName());
List<Pair<byte[], byte[]>> columns = columnIndexes.get(tableIndex);
for (ImmutableBytesWritable aggregatedArray : values) {
DataInputStream input = new DataInputStream(new
ByteArrayInputStream(aggregatedArray.get()));
{noformat}
So for 2nd index onwards we always prepare wrong KVs with wrong column names.
That's why we scanning from second index giving no results.
So in patch using logical names till end of job and at the end while writing in
reducer using physical name to move data into proper table.
> CsvBulkloadTool not working properly if there are multiple local indexes to
> the same table(After PHOENIX-1973)
> --------------------------------------------------------------------------------------------------------------
>
> Key: PHOENIX-2925
> URL: https://issues.apache.org/jira/browse/PHOENIX-2925
> Project: Phoenix
> Issue Type: Bug
> Affects Versions: 4.7.0
> Reporter: Rajeshbabu Chintaguntla
> Assignee: Rajeshbabu Chintaguntla
> Fix For: 4.8.0
>
> Attachments: PHOENIX-2925.patch, PHOENIX-2925.patch
>
>
> When there are multiple local indexes then only for first index data is
> getting generated properly and other indexes doesn't have any data. Changing
> testImportWithLocalIndex test as below is failing. ping [~sergey.soldatov]?
> {noformat}
> @Test
> public void testImportWithLocalIndex() throws Exception {
> Statement stmt = conn.createStatement();
> stmt.execute("CREATE TABLE TABLE6 (ID INTEGER NOT NULL PRIMARY KEY, "
> +
> "FIRST_NAME VARCHAR, LAST_NAME VARCHAR) SPLIt ON (1,2)");
> String ddl = "CREATE LOCAL INDEX TABLE6_IDX ON TABLE6 "
> + " (FIRST_NAME ASC)";
> stmt.execute(ddl);
> ddl = "CREATE LOCAL INDEX TABLE6_IDX2 ON TABLE6 " + " (LAST_NAME
> ASC)";
> stmt.execute(ddl);
> FileSystem fs = FileSystem.get(getUtility().getConfiguration());
> FSDataOutputStream outputStream = fs.create(new
> Path("/tmp/input3.csv"));
> PrintWriter printWriter = new PrintWriter(outputStream);
> printWriter.println("1,FirstName 1,LastName 1");
> printWriter.println("2,FirstName 2,LastName 2");
> printWriter.close();
> CsvBulkLoadTool csvBulkLoadTool = new CsvBulkLoadTool();
> csvBulkLoadTool.setConf(getUtility().getConfiguration());
> int exitCode = csvBulkLoadTool.run(new String[] {
> "--input", "/tmp/input3.csv",
> "--table", "table6",
> "--zookeeper", zkQuorum});
> assertEquals(0, exitCode);
> ResultSet rs = stmt.executeQuery("SELECT id, FIRST_NAME FROM TABLE6
> where first_name='FirstName 2'");
> assertTrue(rs.next());
> assertEquals(2, rs.getInt(1));
> assertEquals("FirstName 2", rs.getString(2));
> rs = stmt.executeQuery("SELECT LAST_NAME FROM TABLE6 where
> last_name='LastName 1'");
> assertTrue(rs.next());
> assertEquals("LastName 1", rs.getString(1));
> rs.close();
> stmt.close();
> }
> {noformat}
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)