cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Benyi Wang (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (CASSANDRA-7768) Error when creating multiple CQLSSTableWriters for more than one column family in the same keyspace
Date Sat, 27 Dec 2014 22:49:14 GMT

    [ https://issues.apache.org/jira/browse/CASSANDRA-7768?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14259509#comment-14259509
] 

Benyi Wang commented on CASSANDRA-7768:
---------------------------------------

I encounter this problem in cassandra-all.2.1.2 if you write two column families of the same
keyspace in one JVM. The following code can reproduce the error. 

{code}
package org.apache.cassandra.hadoop.cql3;

import java.lang.reflect.Field;
import java.util.concurrent.ScheduledExecutorService;

import org.apache.cassandra.config.Config;
import org.apache.cassandra.cql3.QueryProcessor;
import org.apache.cassandra.io.sstable.CQLSSTableWriter;

public class TWriter2 {

	public static void main(String[] args) {
		String schema = "create table mgindex.test (x int primary key, y int)";
		String insert = "insert into mgindex.test (x,y) values(?,?)";

		String schema1 = "create table mgindex.test1 (x int primary key, y int)";
		String insert1 = "insert into mgindex.test1 (x,y) values(?,?)";

		Config.setClientMode(true);

		try {
			CQLSSTableWriter w = CQLSSTableWriter.builder()
					.inDirectory("/tmp/feat").forTable(schema).using(insert)
					.build();
			w.addRow(1, 1);
			w.close();

			CQLSSTableWriter w1 = CQLSSTableWriter.builder()
					.inDirectory("/tmp/feat").forTable(schema1).using(insert1)
					.build();
			w1.addRow(4, 4);
			w1.close();
		} catch (Exception e) {
			System.out.println(e);
		}

	}
}
{code}

You will see this exception when the second {{forTable}} is called.

{code}
Exception in thread "main" java.lang.ExceptionInInitializerError
	at org.apache.cassandra.io.sstable.CQLSSTableWriter$Builder.forTable(CQLSSTableWriter.java:360)
	at org.apache.cassandra.hadoop.cql3.TWriter2.main(TWriter2.java:39)
Caused by: java.lang.NullPointerException
	at org.apache.cassandra.config.DatabaseDescriptor.createAllDirectories(DatabaseDescriptor.java:710)
	at org.apache.cassandra.db.Keyspace.<clinit>(Keyspace.java:72)
	... 2 more
{code}

Here is how the code fails for the second column family:
# For the first column family: ksm is null, and it will work correctly
# For the second column family: ksm is not null, it goes into the second branch to check if
the table exists. If not, load the this.schema into Schema.instance. 

{code}
                    // We need to register the keyspace/table metadata through Schema, otherwise
we won't be able to properly
                    // build the insert statement in using().
                    KSMetaData ksm = Schema.instance.getKSMetaData(this.schema.ksName);
                    if (ksm == null)
                    {
                        ksm = KSMetaData.newKeyspace(this.schema.ksName,
                                AbstractReplicationStrategy.getClass("org.apache.cassandra.locator.SimpleStrategy"),
                                ImmutableMap.of("replication_factor", "1"),
                                true,
                                Collections.singleton(this.schema));
                        Schema.instance.load(ksm);
                    }
                    else if (Schema.instance.getCFMetaData(this.schema.ksName, this.schema.cfName)
== null)
                    {
                        Schema.instance.load(this.schema);
                        ksm = KSMetaData.cloneWith(ksm, Iterables.concat(ksm.cfMetaData().values(),
Collections.singleton(this.schema)));
                        Schema.instance.setKeyspaceDefinition(ksm);
                        Keyspace.open(ksm.name).initCf(this.schema.cfId, this.schema.cfName,
false);
                    }
                    return this;
{code}

The problem happens when Keyspace class is initialized at {{Keyspace.open...}}, 

{code}
    static
    {
        if (!StorageService.instance.isClientMode())
            DatabaseDescriptor.createAllDirectories();
    }
{code}

At this time, StorageService is not in client mode, and createAllDirectories will fail because
conf.data_file_directories is null.

I'm not quite sure why {{Keyspace.open(...}} is needed. When I comment out this line, everything
seems working.

Could you reopen this jira and fix it?

> Error when creating multiple CQLSSTableWriters for more than one column family in the
same keyspace
> ---------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-7768
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7768
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Hadoop
>            Reporter: Paul Pak
>            Assignee: Paul Pak
>            Priority: Minor
>              Labels: cql3, hadoop
>         Attachments: trunk-7768-v1.txt
>
>
> The reason why this occurs is if the keyspace has already been loaded (due to another
column family being previously loaded in the same keyspace), CQLSSTableWriter builder only
loads the column family via Schema.load(CFMetaData). However, Schema.load(CFMetaData) only
adds to the Schema.cfIdMap without making the proper addition to the CFMetaData map belonging
to the KSMetaData map.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Mime
View raw message