Return-Path: Delivered-To: apmail-mahout-user-archive@www.apache.org Received: (qmail 25012 invoked from network); 30 Jun 2010 15:04:33 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 30 Jun 2010 15:04:33 -0000 Received: (qmail 36095 invoked by uid 500); 30 Jun 2010 15:04:32 -0000 Delivered-To: apmail-mahout-user-archive@mahout.apache.org Received: (qmail 35494 invoked by uid 500); 30 Jun 2010 15:04:31 -0000 Mailing-List: contact user-help@mahout.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@mahout.apache.org Delivered-To: mailing list user@mahout.apache.org Received: (qmail 35279 invoked by uid 99); 30 Jun 2010 15:04:30 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 30 Jun 2010 15:04:30 +0000 X-ASF-Spam-Status: No, hits=2.2 required=10.0 tests=FREEMAIL_FROM,HTML_MESSAGE,SPF_PASS,T_TO_NO_BRKTS_FREEMAIL X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of jake.mannix@gmail.com designates 209.85.161.170 as permitted sender) Received: from [209.85.161.170] (HELO mail-gx0-f170.google.com) (209.85.161.170) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 30 Jun 2010 15:04:23 +0000 Received: by gxk3 with SMTP id 3so585368gxk.1 for ; Wed, 30 Jun 2010 08:04:02 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=gmail.com; s=gamma; h=domainkey-signature:received:mime-version:received:in-reply-to :references:from:date:message-id:subject:to:content-type; bh=NMyUiE2/HZjGPI6Eno6BIw5vEXiiRE6PY9sNIQJLiAY=; b=e8A+gcHuD6EK+VulMVCX0h7K4iQXutTiGioPbkP4ZlLzTraT/EKAudsB43O7Xfkzpw uXFjHnS+r76ESbPaKvaQQPDyczkcdms/OIircvzOI9ZhTsmmee+eItOcdjWbAdy7Ns28 PKAMMQb0zIcqEoJRZ8AY5G6Z67Smn8ME5Cm2U= DomainKey-Signature: a=rsa-sha1; c=nofws; d=gmail.com; s=gamma; h=mime-version:in-reply-to:references:from:date:message-id:subject:to :content-type; b=sBGscp4cqRcHeForkStqycIXj92eEpmZl67wEhIQ4BMRlqcK1TRlpdRaVvpkxozQYB dOXYzf9mFOJ66Z0MY1MpjehfOEAHEoMEX8p2ntqXupW4X03TAJecZPJ9Ti+AK/ImPQ7z KBwxWvYgPGS9DoEiS4i3kopMW+sCHcnVpSAtk= Received: by 10.90.199.3 with SMTP id w3mr7196736agf.24.1277910242190; Wed, 30 Jun 2010 08:04:02 -0700 (PDT) MIME-Version: 1.0 Received: by 10.90.71.19 with HTTP; Wed, 30 Jun 2010 08:03:42 -0700 (PDT) In-Reply-To: <4805215288100484793@unknownmsgid> References: <4805215288100484793@unknownmsgid> From: Jake Mannix Date: Wed, 30 Jun 2010 17:03:42 +0200 Message-ID: Subject: Re: DistributedLanczosSolver input To: user@mahout.apache.org Content-Type: multipart/alternative; boundary=001636284ac40a6b5c048a40aa54 X-Virus-Checked: Checked by ClamAV on apache.org --001636284ac40a6b5c048a40aa54 Content-Type: text/plain; charset=ISO-8859-1 SequentialAccessSparseVector does not implement Writable. You need a Reducer. In this reducer, you do everything you are currently doing, except that instead of writing this SequentialAccessSparseVector directly (via Context or OutputCollector's write methods), you: SequentialAccessSparseVectorWritable v = // build it up // ... output.write(new VectorWritable(v)); This should fix your current problem. Another thing you should be careful of is that if your itemIds and userIds are of type long, they need to be somehow turned into int. The keys on vector types currently in Mahout are restricted to 32 bits. -jake On Wed, Jun 30, 2010 at 10:28 AM, Laszlo Dosa wrote: > Hi, > > I try to run the > org.apache.mahout.math.hadoop.decomposer.DistributedLanczosSolver. > My input look like (userid, itemid) as follows: > ... > 122641863,5060057723326 > 123441107,9789020282948 > ... > > How can I transform my input to the format that DistributedLanczosSolver > needs (rows = users, columns=items, elements=number of items/user)? > > I tried to write a MapReduce Job with Mapper IntWritable> > that maps the row to userid as key and itemid as value > and a > Reducer > that instantiates a SequentialAccessSparseVector with itemid as key and > itemid as index and sum(itemid) as value. > > I am getting this exception with the attached code: > > 2010-06-29 09:04:59,172 WARN org.apache.hadoop.mapred.TaskTracker: Error > running child > java.lang.NullPointerException > at > > org.apache.hadoop.io.serializer.SerializationFactory.getSerializer(Serializa > tionFactory.java:73) > at > org.apache.hadoop.mapred.MapTask$MapOutputBuffer.(MapTask.java:759) > at > > org.apache.hadoop.mapred.MapTask$NewOutputCollector.(MapTask.java:487) > at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:575) > at org.apache.hadoop.mapred.MapTask.run(MapTask.java:305) > at org.apache.hadoop.mapred.Child.main(Child.java:170) > > > Can you suggest any other way? > > Regards, > Laszlo > > > > import java.io.IOException; > > import java.util.HashMap; > > import java.util.Map; > > > > import org.apache.hadoop.conf.Configuration; > > import org.apache.hadoop.fs.Path; > > import org.apache.hadoop.io.IntWritable; > > import org.apache.hadoop.io.Text; > > import org.apache.hadoop.mapreduce.Job; > > import org.apache.hadoop.mapreduce.Mapper; > > import org.apache.hadoop.mapreduce.Reducer; > > import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; > > import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; > > import org.apache.hadoop.util.GenericOptionsParser; > > import org.apache.mahout.math.SequentialAccessSparseVector; > > > > public class UserItemMatrix { > > public static class TokenizerMapper extends Mapper IntWritable, IntWritable>{ > > private final static String DELIMITER = ","; > > > > public void map(Object key, Text value, Context context) > throws IOException, InterruptedException { > > String[] values = DELIMITER.split(value.toString()); > > IntWritable userId = new > IntWritable(Integer.parseInt(values[0])); > > IntWritable itemId = new > IntWritable(Integer.parseInt(values[1])); > > context.write(userId, itemId); > > } > > } > > > > public static class ItemReducer extends > Reducer { > > > > private SequentialAccessSparseVector vector = new > SequentialAccessSparseVector(); > > > > public void reduce(IntWritable key, Iterable > values, Context context) throws IOException, InterruptedException { > > Map cooccurence = new > HashMap(); > > for (IntWritable val : values) { > > int num = cooccurence.get(val.get()); > > num++; > > cooccurence.put(val.get(), num); > > } > > for(Map.Entry entry : > cooccurence.entrySet()) { > > vector.set(entry.getKey(), entry.getValue()); > > } > > context.write(key, vector); > > } > > } > > > > public static void main(String[] args) throws Exception { > > Configuration conf = new Configuration(); > > String[] otherArgs = new GenericOptionsParser(conf, > args).getRemainingArgs(); > > if (otherArgs.length != 2) { > > System.err.println("Usage: User Item cooccurence > matrix "); > > System.exit(2); > > } > > > > Job job = new Job(conf, "User Item cooccurence matrix"); > > job.setJarByClass(UserItemMatrix.class); > > job.setMapperClass(TokenizerMapper.class); > > job.setCombinerClass(ItemReducer.class); > > job.setReducerClass(ItemReducer.class); > > job.setOutputKeyClass(IntWritable.class); > > job.setOutputValueClass(SequentialAccessSparseVector.class); > > FileInputFormat.addInputPath(job, new Path(otherArgs[0])); > > FileOutputFormat.setOutputPath(job, new Path(otherArgs[1])); > > System.exit(job.waitForCompletion(true) ? 0 : 1); > > } > > } > > > > --001636284ac40a6b5c048a40aa54--