From e127d09bbf3fb454d314b769bda54f02582c846a Mon Sep 17 00:00:00 2001 From: Costin Leau Date: Fri, 17 Jan 2014 00:48:53 +0200 Subject: [PATCH] allow raw json as a source add support for indexing raw json without any type conversion currently only the M/R module has been tested properly the data is passed as is, without any transformation if needed, the raw document can be inspected to extra 'id' information relates #9 fixes #75 fixes #126 --- .../cascading/CascadingValueWriter.java | 4 +- .../hadoop/cascading/EsHadoopScheme.java | 8 +- .../hadoop/cascading/EsLocalScheme.java | 7 +- .../hadoop/cascading/EsLocalTap.java | 2 +- .../hadoop/cfg/ConfigurationOptions.java | 4 + .../elasticsearch/hadoop/cfg/Settings.java | 6 +- .../elasticsearch/hadoop/hive/EsSerDe.java | 7 +- .../hadoop/hive/EsStorageHandler.java | 5 +- .../hadoop/hive/HiveFieldExtractor.java | 2 +- .../hadoop/hive/HiveValueWriter.java | 2 +- .../hadoop/mr/EsInputFormat.java | 5 +- .../hadoop/mr/EsOutputFormat.java | 5 +- .../hadoop/mr/WritableValueReader.java | 2 +- .../hadoop/mr/WritableValueWriter.java | 6 +- .../elasticsearch/hadoop/pig/EsStorage.java | 5 +- .../hadoop/pig/PigFieldExtractor.java | 11 +- .../hadoop/pig/PigValueReader.java | 2 +- .../hadoop/pig/PigValueWriter.java | 2 +- .../hadoop/rest/InitializationUtils.java | 61 +- .../hadoop/rest/RestRepository.java | 4 +- .../hadoop/serialization/ParsingUtils.java | 120 +++ .../hadoop/serialization/ScrollReader.java | 1 + .../serialization/SerializationUtils.java | 52 - .../{ => builder}/ContentBuilder.java | 4 +- .../{ => builder}/JdkValueReader.java | 5 +- .../{ => builder}/JdkValueWriter.java | 4 +- .../builder/NoOpValueWriter.java | 33 + .../{ => builder}/ValueReader.java | 5 +- .../{ => builder}/ValueWriter.java | 4 +- .../{ => command}/AbstractCommandFactory.java | 120 ++- .../{ => command}/BulkCommands.java | 2 +- .../serialization/{ => command}/Command.java | 2 +- .../{ => command}/CommandFactory.java | 3 +- .../{ => command}/CreateCommandFactory.java | 2 +- .../{ => command}/IndexCommandFactory.java | 2 +- .../command/JsonTemplatedCommand.java | 85 ++ .../{ => command}/TemplatedCommand.java | 24 +- .../{ => command}/UpdateCommandFactory.java | 3 +- .../{ => field}/ConstantFieldExtractor.java | 3 +- .../{ => field}/FieldExtractor.java | 4 +- .../field/JsonFieldExtractors.java | 133 +++ .../MapWritableFieldExtractor.java | 2 +- .../field/NoOpFieldExtractor.java | 27 + .../serialization/json/JacksonJsonParser.java | 6 +- .../integration/mr/MRNewApiSaveTest.java | 90 +- .../integration/mr/MROldApiSaveTest.java | 103 +- .../hadoop/integration/mr/MRSuite.java | 5 +- .../integration/rest/RestQueryTest.java | 4 +- .../hadoop/integration/rest/RestSaveTest.java | 2 +- .../hadoop/rest/ScrollReaderTest.java | 2 +- .../hadoop/serialization/CommandTest.java | 5 +- .../serialization/HiveTypeToJsonTest.java | 1 + .../serialization/JdkTypeToJsonTest.java | 2 + .../serialization/JsonValuePathTest.java | 64 ++ .../serialization/MapFieldExtractor.java | 5 +- .../serialization/PigTypeToJsonTest.java | 1 + .../hadoop/serialization/ValueReaderTest.java | 1 + .../WritableTypeFromJsonTest.java | 1 + .../serialization/WritableTypeToJsonTest.java | 1 + src/test/resources/artists.json | 953 ++++++++++++++++++ .../serialization/parser-test-nested.json | 35 + 61 files changed, 1835 insertions(+), 236 deletions(-) delete mode 100644 src/main/java/org/elasticsearch/hadoop/serialization/SerializationUtils.java rename src/main/java/org/elasticsearch/hadoop/serialization/{ => builder}/ContentBuilder.java (97%) rename src/main/java/org/elasticsearch/hadoop/serialization/{ => builder}/JdkValueReader.java (95%) rename src/main/java/org/elasticsearch/hadoop/serialization/{ => builder}/JdkValueWriter.java (97%) create mode 100644 src/main/java/org/elasticsearch/hadoop/serialization/builder/NoOpValueWriter.java rename src/main/java/org/elasticsearch/hadoop/serialization/{ => builder}/ValueReader.java (88%) rename src/main/java/org/elasticsearch/hadoop/serialization/{ => builder}/ValueWriter.java (90%) rename src/main/java/org/elasticsearch/hadoop/serialization/{ => command}/AbstractCommandFactory.java (56%) rename src/main/java/org/elasticsearch/hadoop/serialization/{ => command}/BulkCommands.java (96%) rename src/main/java/org/elasticsearch/hadoop/serialization/{ => command}/Command.java (95%) rename src/main/java/org/elasticsearch/hadoop/serialization/{ => command}/CommandFactory.java (93%) rename src/main/java/org/elasticsearch/hadoop/serialization/{ => command}/CreateCommandFactory.java (95%) rename src/main/java/org/elasticsearch/hadoop/serialization/{ => command}/IndexCommandFactory.java (95%) create mode 100644 src/main/java/org/elasticsearch/hadoop/serialization/command/JsonTemplatedCommand.java rename src/main/java/org/elasticsearch/hadoop/serialization/{ => command}/TemplatedCommand.java (78%) rename src/main/java/org/elasticsearch/hadoop/serialization/{ => command}/UpdateCommandFactory.java (94%) rename src/main/java/org/elasticsearch/hadoop/serialization/{ => field}/ConstantFieldExtractor.java (94%) rename src/main/java/org/elasticsearch/hadoop/serialization/{ => field}/FieldExtractor.java (89%) create mode 100644 src/main/java/org/elasticsearch/hadoop/serialization/field/JsonFieldExtractors.java rename src/main/java/org/elasticsearch/hadoop/serialization/{ => field}/MapWritableFieldExtractor.java (96%) create mode 100644 src/main/java/org/elasticsearch/hadoop/serialization/field/NoOpFieldExtractor.java create mode 100644 src/test/java/org/elasticsearch/hadoop/serialization/JsonValuePathTest.java create mode 100644 src/test/resources/artists.json create mode 100644 src/test/resources/org/elasticsearch/hadoop/serialization/parser-test-nested.json diff --git a/src/main/java/org/elasticsearch/hadoop/cascading/CascadingValueWriter.java b/src/main/java/org/elasticsearch/hadoop/cascading/CascadingValueWriter.java index de7d2688c..04ab1b0c7 100644 --- a/src/main/java/org/elasticsearch/hadoop/cascading/CascadingValueWriter.java +++ b/src/main/java/org/elasticsearch/hadoop/cascading/CascadingValueWriter.java @@ -23,8 +23,8 @@ import org.apache.hadoop.io.Writable; import org.elasticsearch.hadoop.mr.WritableValueWriter; import org.elasticsearch.hadoop.serialization.Generator; -import org.elasticsearch.hadoop.serialization.JdkValueWriter; -import org.elasticsearch.hadoop.serialization.ValueWriter; +import org.elasticsearch.hadoop.serialization.builder.JdkValueWriter; +import org.elasticsearch.hadoop.serialization.builder.ValueWriter; import cascading.scheme.SinkCall; import cascading.tuple.Tuple; diff --git a/src/main/java/org/elasticsearch/hadoop/cascading/EsHadoopScheme.java b/src/main/java/org/elasticsearch/hadoop/cascading/EsHadoopScheme.java index 52e9d5446..3504aae2d 100644 --- a/src/main/java/org/elasticsearch/hadoop/cascading/EsHadoopScheme.java +++ b/src/main/java/org/elasticsearch/hadoop/cascading/EsHadoopScheme.java @@ -34,8 +34,8 @@ import org.elasticsearch.hadoop.mr.EsInputFormat; import org.elasticsearch.hadoop.mr.EsOutputFormat; import org.elasticsearch.hadoop.mr.HadoopCfgUtils; -import org.elasticsearch.hadoop.serialization.JdkValueReader; -import org.elasticsearch.hadoop.serialization.SerializationUtils; +import org.elasticsearch.hadoop.rest.InitializationUtils; +import org.elasticsearch.hadoop.serialization.builder.JdkValueReader; import org.elasticsearch.hadoop.util.FieldAlias; import org.elasticsearch.hadoop.util.StringUtils; @@ -118,8 +118,8 @@ public void sinkConfInit(FlowProcess flowProcess, Tap { @@ -67,6 +68,9 @@ else if (writable instanceof UTF8) { UTF8 utf8 = (UTF8) writable; generator.writeUTF8String(utf8.getBytes(), 0, utf8.getLength()); } + else if (writable instanceof ShortWritable) { + generator.writeNumber(((ShortWritable) writable).get()); + } else if (writable instanceof IntWritable) { generator.writeNumber(((IntWritable) writable).get()); } diff --git a/src/main/java/org/elasticsearch/hadoop/pig/EsStorage.java b/src/main/java/org/elasticsearch/hadoop/pig/EsStorage.java index 75d8f5bc6..67ae9f54c 100644 --- a/src/main/java/org/elasticsearch/hadoop/pig/EsStorage.java +++ b/src/main/java/org/elasticsearch/hadoop/pig/EsStorage.java @@ -59,7 +59,6 @@ import org.elasticsearch.hadoop.cfg.SettingsManager; import org.elasticsearch.hadoop.mr.EsOutputFormat; import org.elasticsearch.hadoop.rest.InitializationUtils; -import org.elasticsearch.hadoop.serialization.SerializationUtils; import org.elasticsearch.hadoop.util.IOUtils; import org.elasticsearch.hadoop.util.ObjectUtils; import org.elasticsearch.hadoop.util.StringUtils; @@ -149,8 +148,8 @@ private void init(String location, Job job) { boolean changed = false; InitializationUtils.checkIdForOperation(settings); - changed |= SerializationUtils.setValueWriterIfNotSet(settings, PigValueWriter.class, log); - changed |= SerializationUtils.setValueReaderIfNotSet(settings, PigValueReader.class, log); + changed |= InitializationUtils.setValueWriterIfNotSet(settings, PigValueWriter.class, log); + changed |= InitializationUtils.setValueReaderIfNotSet(settings, PigValueReader.class, log); changed |= InitializationUtils.setFieldExtractorIfNotSet(settings, PigFieldExtractor.class, log); settings.save(); } diff --git a/src/main/java/org/elasticsearch/hadoop/pig/PigFieldExtractor.java b/src/main/java/org/elasticsearch/hadoop/pig/PigFieldExtractor.java index 4e374b756..de6242dc4 100644 --- a/src/main/java/org/elasticsearch/hadoop/pig/PigFieldExtractor.java +++ b/src/main/java/org/elasticsearch/hadoop/pig/PigFieldExtractor.java @@ -22,13 +22,11 @@ import org.apache.pig.backend.executionengine.ExecException; import org.apache.pig.data.DataType; import org.elasticsearch.hadoop.cfg.Settings; -import org.elasticsearch.hadoop.serialization.ConstantFieldExtractor; +import org.elasticsearch.hadoop.serialization.field.ConstantFieldExtractor; import org.elasticsearch.hadoop.util.Assert; public class PigFieldExtractor extends ConstantFieldExtractor { - private String fieldName; - @Override protected String extractField(Object target) { if (target instanceof PigTuple) { @@ -37,14 +35,14 @@ protected String extractField(Object target) { for (int i = 0; i < fields.length; i++) { ResourceFieldSchema field = fields[i]; - if (fieldName.equals(field.getName())) { + if (getFieldName().equals(field.getName())) { byte type = field.getType(); Assert.isTrue(DataType.isAtomic(type), - String.format("Unsupported data type [%s] for field [%s]; use only 'primitives'", DataType.findTypeName(type), fieldName)); + String.format("Unsupported data type [%s] for field [%s]; use only 'primitives'", DataType.findTypeName(type), getFieldName())); try { return pt.getTuple().get(i).toString(); } catch (ExecException ex) { - throw new IllegalStateException(String.format("Cannot retrieve field [%s]", fieldName), ex); + throw new IllegalStateException(String.format("Cannot retrieve field [%s]", getFieldName()), ex); } } } @@ -56,6 +54,5 @@ protected String extractField(Object target) { @Override public void setSettings(Settings settings) { super.setSettings(settings); - fieldName = getFieldName(); } } diff --git a/src/main/java/org/elasticsearch/hadoop/pig/PigValueReader.java b/src/main/java/org/elasticsearch/hadoop/pig/PigValueReader.java index 144ba75bf..293593a7f 100644 --- a/src/main/java/org/elasticsearch/hadoop/pig/PigValueReader.java +++ b/src/main/java/org/elasticsearch/hadoop/pig/PigValueReader.java @@ -22,7 +22,7 @@ import org.apache.pig.data.DataByteArray; import org.apache.pig.data.TupleFactory; -import org.elasticsearch.hadoop.serialization.JdkValueReader; +import org.elasticsearch.hadoop.serialization.builder.JdkValueReader; public class PigValueReader extends JdkValueReader { diff --git a/src/main/java/org/elasticsearch/hadoop/pig/PigValueWriter.java b/src/main/java/org/elasticsearch/hadoop/pig/PigValueWriter.java index 8d9b838cb..ea56dd76f 100644 --- a/src/main/java/org/elasticsearch/hadoop/pig/PigValueWriter.java +++ b/src/main/java/org/elasticsearch/hadoop/pig/PigValueWriter.java @@ -31,7 +31,7 @@ import org.elasticsearch.hadoop.serialization.Generator; import org.elasticsearch.hadoop.serialization.SerializationException; import org.elasticsearch.hadoop.serialization.SettingsAware; -import org.elasticsearch.hadoop.serialization.ValueWriter; +import org.elasticsearch.hadoop.serialization.builder.ValueWriter; import org.elasticsearch.hadoop.util.FieldAlias; import org.elasticsearch.hadoop.util.StringUtils; diff --git a/src/main/java/org/elasticsearch/hadoop/rest/InitializationUtils.java b/src/main/java/org/elasticsearch/hadoop/rest/InitializationUtils.java index afde26c62..4a7ea80ae 100644 --- a/src/main/java/org/elasticsearch/hadoop/rest/InitializationUtils.java +++ b/src/main/java/org/elasticsearch/hadoop/rest/InitializationUtils.java @@ -29,9 +29,12 @@ import org.elasticsearch.hadoop.cfg.InternalConfigurationOptions; import org.elasticsearch.hadoop.cfg.Settings; import org.elasticsearch.hadoop.cfg.SettingsManager; -import org.elasticsearch.hadoop.serialization.ContentBuilder; -import org.elasticsearch.hadoop.serialization.FieldExtractor; -import org.elasticsearch.hadoop.serialization.ValueWriter; +import org.elasticsearch.hadoop.serialization.builder.ContentBuilder; +import org.elasticsearch.hadoop.serialization.builder.NoOpValueWriter; +import org.elasticsearch.hadoop.serialization.builder.ValueReader; +import org.elasticsearch.hadoop.serialization.builder.ValueWriter; +import org.elasticsearch.hadoop.serialization.field.FieldExtractor; +import org.elasticsearch.hadoop.serialization.field.NoOpFieldExtractor; import org.elasticsearch.hadoop.util.Assert; import org.elasticsearch.hadoop.util.BytesArray; import org.elasticsearch.hadoop.util.FastByteArrayOutputStream; @@ -93,9 +96,20 @@ public static void checkIndexExistence(Settings settings, RestRepository client) public static boolean setFieldExtractorIfNotSet(Settings settings, Class clazz, Log log) { if (!StringUtils.hasText(settings.getMappingIdExtractorClassName())) { - settings.setProperty(ConfigurationOptions.ES_MAPPING_DEFAULT_EXTRACTOR_CLASS, clazz.getName()); Log logger = (log != null ? log : LogFactory.getLog(clazz)); - logger.debug(String.format("Using pre-defined field extractor [%s] as default", settings.getMappingIdExtractorClassName())); + + String name = clazz.getName(); + if (settings.getInputAsJson()) { + name = NoOpFieldExtractor.class.getName(); + if (logger.isDebugEnabled()) { + logger.debug(String.format("Elasticsearch input marked as JSON; using dedicated field extractor [%s] instead of [%s]", name, clazz)); + } + } + + settings.setProperty(ConfigurationOptions.ES_MAPPING_DEFAULT_EXTRACTOR_CLASS, name); + if (logger.isDebugEnabled()) { + logger.debug(String.format("Using pre-defined field extractor [%s] as default", settings.getMappingIdExtractorClassName())); + } return true; } @@ -126,4 +140,39 @@ public static void saveSchemaIfNeeded(Object conf, ValueWriter schemaWrit client.close(); } } -} + + public static boolean setValueWriterIfNotSet(Settings settings, Class> clazz, Log log) { + if (!StringUtils.hasText(settings.getSerializerValueWriterClassName())) { + Log logger = (log != null ? log : LogFactory.getLog(clazz)); + + String name = clazz.getName(); + if (settings.getInputAsJson()) { + name = NoOpValueWriter.class.getName(); + if (logger.isDebugEnabled()) { + logger.debug(String.format("Elasticsearch input marked as JSON; bypassing serialization through [%s] instead of [%s]", name, clazz)); + } + } + settings.setProperty(ConfigurationOptions.ES_SERIALIZATION_WRITER_CLASS, name); + if (logger.isDebugEnabled()) { + logger.debug(String.format("Using pre-defined writer serializer [%s] as default", settings.getSerializerValueWriterClassName())); + } + return true; + } + + return false; + } + + public static boolean setValueReaderIfNotSet(Settings settings, Class clazz, Log log) { + + if (!StringUtils.hasText(settings.getSerializerValueReaderClassName())) { + settings.setProperty(ConfigurationOptions.ES_SERIALIZATION_READER_CLASS, clazz.getName()); + Log logger = (log != null ? log : LogFactory.getLog(clazz)); + if (logger.isDebugEnabled()) { + logger.debug(String.format("Using pre-defined reader serializer [%s] as default", settings.getSerializerValueReaderClassName())); + } + return true; + } + + return false; + } +} \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/hadoop/rest/RestRepository.java b/src/main/java/org/elasticsearch/hadoop/rest/RestRepository.java index 496030060..51c3c91c6 100644 --- a/src/main/java/org/elasticsearch/hadoop/rest/RestRepository.java +++ b/src/main/java/org/elasticsearch/hadoop/rest/RestRepository.java @@ -30,9 +30,9 @@ import org.elasticsearch.hadoop.rest.dto.Node; import org.elasticsearch.hadoop.rest.dto.Shard; import org.elasticsearch.hadoop.rest.dto.mapping.Field; -import org.elasticsearch.hadoop.serialization.BulkCommands; -import org.elasticsearch.hadoop.serialization.Command; import org.elasticsearch.hadoop.serialization.ScrollReader; +import org.elasticsearch.hadoop.serialization.command.BulkCommands; +import org.elasticsearch.hadoop.serialization.command.Command; import org.elasticsearch.hadoop.util.Assert; import org.elasticsearch.hadoop.util.BytesArray; import org.elasticsearch.hadoop.util.BytesRef; diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/ParsingUtils.java b/src/main/java/org/elasticsearch/hadoop/serialization/ParsingUtils.java index 3486cdc4e..af9dbc54e 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/ParsingUtils.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/ParsingUtils.java @@ -18,7 +18,10 @@ */ package org.elasticsearch.hadoop.serialization; +import java.util.ArrayList; +import java.util.LinkedHashSet; import java.util.List; +import java.util.Set; import org.elasticsearch.hadoop.serialization.Parser.Token; import org.elasticsearch.hadoop.util.StringUtils; @@ -96,4 +99,121 @@ else if (path2 != null && currentName.equals(path2[index2])) { return null; } + + private static class Matcher { + private final List tokens; + private int tokenIndex = 0; + private boolean matched = false; + private Object value; + + Matcher(String path) { + tokens = StringUtils.tokenize(path, "."); + } + + boolean matches(String value) { + boolean match = tokens.get(tokenIndex).equals(value); + if (match) { + if (tokenIndex < tokens.size() - 1) { + tokenIndex++; + } + else { + matched = true; + this.value = value; + } + } + return match; + }; + } + + public static List values(Parser parser, String... paths) { + List matchers = new ArrayList(paths.length); + for (String path : paths) { + matchers.add(new Matcher(path)); + } + + List active = new ArrayList(matchers); + Set inactive = new LinkedHashSet(); + + doFind(parser, new ArrayList(matchers), active, inactive); + + List matches = new ArrayList(); + for (Matcher matcher : matchers) { + matches.add(matcher.matched ? matcher.value.toString() : null); + } + + return matches; + } + + private static void doFind(Parser parser, List current, List active, Set inactive) { + Token token = null; + List matchingCurrentLevel = null; + + String currentName; + token = parser.currentToken(); + if (token == null) { + token = parser.nextToken(); + } + + while ((token = parser.nextToken()) != null) { + if (token == Token.START_OBJECT) { + token = parser.nextToken(); + if (matchingCurrentLevel == null) { + parser.skipChildren(); + } + else { + doFind(parser, matchingCurrentLevel, active, inactive); + } + } + else if (token == Token.FIELD_NAME) { + currentName = parser.currentName(); + + Object value = null; + boolean valueRead = false; + + for (Matcher matcher : current) { + if (matcher.matches(currentName)) { + if (matcher.matched) { + inactive.add(matcher); + if (!valueRead) { + switch (parser.nextToken()) { + case VALUE_NUMBER: + value = parser.numberValue(); + break; + case VALUE_BOOLEAN: + value = Boolean.valueOf(parser.booleanValue()); + break; + case VALUE_NULL: + value = null; + break; + case VALUE_STRING: + value = parser.text(); + break; + default: + throw new IllegalArgumentException(String.format( + "Incorrect parsing; expected value but found [%s]", parser.currentToken())); + } + } + matcher.value = value; + } + else { + if (matchingCurrentLevel == null) { + matchingCurrentLevel = new ArrayList(current.size()); + } + matchingCurrentLevel.add(matcher); + } + } + } + } + else if (token == Token.END_OBJECT) { + // once matching, the matcher needs to match all the way - if it's not inactive (since it matched) + if (matchingCurrentLevel != null) { + for (Matcher matcher : matchingCurrentLevel) { + active.remove(matcher); + inactive.add(matcher); + } + } + } + // ignore other tokens + } + } } \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/ScrollReader.java b/src/main/java/org/elasticsearch/hadoop/serialization/ScrollReader.java index 65a0261cd..9e8067c28 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/ScrollReader.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/ScrollReader.java @@ -30,6 +30,7 @@ import org.elasticsearch.hadoop.rest.dto.mapping.Field; import org.elasticsearch.hadoop.serialization.Parser.NumberType; import org.elasticsearch.hadoop.serialization.Parser.Token; +import org.elasticsearch.hadoop.serialization.builder.ValueReader; import org.elasticsearch.hadoop.serialization.json.JacksonJsonParser; import org.elasticsearch.hadoop.util.Assert; import org.elasticsearch.hadoop.util.BytesArray; diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/SerializationUtils.java b/src/main/java/org/elasticsearch/hadoop/serialization/SerializationUtils.java deleted file mode 100644 index 0e74760d5..000000000 --- a/src/main/java/org/elasticsearch/hadoop/serialization/SerializationUtils.java +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.elasticsearch.hadoop.serialization; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.elasticsearch.hadoop.cfg.ConfigurationOptions; -import org.elasticsearch.hadoop.cfg.Settings; -import org.elasticsearch.hadoop.util.StringUtils; - -public abstract class SerializationUtils { - - public static boolean setValueWriterIfNotSet(Settings settings, Class> clazz, Log log) { - - if (!StringUtils.hasText(settings.getSerializerValueWriterClassName())) { - settings.setProperty(ConfigurationOptions.ES_SERIALIZATION_WRITER_CLASS, clazz.getName()); - Log logger = (log != null ? log : LogFactory.getLog(clazz)); - logger.debug(String.format("Using pre-defined writer serializer [%s] as default", settings.getSerializerValueWriterClassName())); - return true; - } - - return false; - } - - public static boolean setValueReaderIfNotSet(Settings settings, Class clazz, Log log) { - - if (!StringUtils.hasText(settings.getSerializerValueReaderClassName())) { - settings.setProperty(ConfigurationOptions.ES_SERIALIZATION_READER_CLASS, clazz.getName()); - Log logger = (log != null ? log : LogFactory.getLog(clazz)); - logger.debug(String.format("Using pre-defined reader serializer [%s] as default", settings.getSerializerValueReaderClassName())); - return true; - } - - return false; - } -} diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/ContentBuilder.java b/src/main/java/org/elasticsearch/hadoop/serialization/builder/ContentBuilder.java similarity index 97% rename from src/main/java/org/elasticsearch/hadoop/serialization/ContentBuilder.java rename to src/main/java/org/elasticsearch/hadoop/serialization/builder/ContentBuilder.java index 7810f27f7..2e23662f6 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/ContentBuilder.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/builder/ContentBuilder.java @@ -16,10 +16,12 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.builder; import java.io.OutputStream; +import org.elasticsearch.hadoop.serialization.Generator; +import org.elasticsearch.hadoop.serialization.SerializationException; import org.elasticsearch.hadoop.serialization.json.JacksonJsonGenerator; import org.elasticsearch.hadoop.util.Assert; import org.elasticsearch.hadoop.util.FastByteArrayOutputStream; diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/JdkValueReader.java b/src/main/java/org/elasticsearch/hadoop/serialization/builder/JdkValueReader.java similarity index 95% rename from src/main/java/org/elasticsearch/hadoop/serialization/JdkValueReader.java rename to src/main/java/org/elasticsearch/hadoop/serialization/builder/JdkValueReader.java index 259ba6aa2..7c80363b9 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/JdkValueReader.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/builder/JdkValueReader.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.builder; import java.util.Collections; import java.util.LinkedHashMap; @@ -24,6 +24,9 @@ import java.util.Map; import org.elasticsearch.hadoop.cfg.Settings; +import org.elasticsearch.hadoop.serialization.FieldType; +import org.elasticsearch.hadoop.serialization.Parser; +import org.elasticsearch.hadoop.serialization.SettingsAware; import org.elasticsearch.hadoop.util.StringUtils; diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/JdkValueWriter.java b/src/main/java/org/elasticsearch/hadoop/serialization/builder/JdkValueWriter.java similarity index 97% rename from src/main/java/org/elasticsearch/hadoop/serialization/JdkValueWriter.java rename to src/main/java/org/elasticsearch/hadoop/serialization/builder/JdkValueWriter.java index ff749621d..9307098ab 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/JdkValueWriter.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/builder/JdkValueWriter.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.builder; import java.sql.Timestamp; import java.util.Calendar; @@ -26,6 +26,8 @@ import javax.xml.bind.DatatypeConverter; +import org.elasticsearch.hadoop.serialization.Generator; + /** * Value writer for JDK types. */ diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/builder/NoOpValueWriter.java b/src/main/java/org/elasticsearch/hadoop/serialization/builder/NoOpValueWriter.java new file mode 100644 index 000000000..00d31e14f --- /dev/null +++ b/src/main/java/org/elasticsearch/hadoop/serialization/builder/NoOpValueWriter.java @@ -0,0 +1,33 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.hadoop.serialization.builder; + +import org.elasticsearch.hadoop.serialization.Generator; + +/** + * A no-op value writer throwing exceptions if called. + * Used when the JSON is not generated (because it already exists). + */ +public class NoOpValueWriter implements ValueWriter { + + @Override + public boolean write(Object object, Generator generator) { + throw new IllegalStateException("Incorrect configuration - NoOpValueWriter should not have been called"); + } +} diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/ValueReader.java b/src/main/java/org/elasticsearch/hadoop/serialization/builder/ValueReader.java similarity index 88% rename from src/main/java/org/elasticsearch/hadoop/serialization/ValueReader.java rename to src/main/java/org/elasticsearch/hadoop/serialization/builder/ValueReader.java index b9d1cb8e0..56779af0b 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/ValueReader.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/builder/ValueReader.java @@ -16,10 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.builder; import java.util.List; +import org.elasticsearch.hadoop.serialization.FieldType; +import org.elasticsearch.hadoop.serialization.Parser; + /** * Translates a JSON field to an actual object. Implementations should only handle the conversion and not influence the parser outside the conversion. diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/ValueWriter.java b/src/main/java/org/elasticsearch/hadoop/serialization/builder/ValueWriter.java similarity index 90% rename from src/main/java/org/elasticsearch/hadoop/serialization/ValueWriter.java rename to src/main/java/org/elasticsearch/hadoop/serialization/builder/ValueWriter.java index 20c7fd702..a1590f353 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/ValueWriter.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/builder/ValueWriter.java @@ -16,7 +16,9 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.builder; + +import org.elasticsearch.hadoop.serialization.Generator; /** * Translates a value to its JSON-like structure. diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/AbstractCommandFactory.java b/src/main/java/org/elasticsearch/hadoop/serialization/command/AbstractCommandFactory.java similarity index 56% rename from src/main/java/org/elasticsearch/hadoop/serialization/AbstractCommandFactory.java rename to src/main/java/org/elasticsearch/hadoop/serialization/command/AbstractCommandFactory.java index 3d318997e..798d3d987 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/AbstractCommandFactory.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/command/AbstractCommandFactory.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.command; import java.util.ArrayList; import java.util.List; @@ -25,7 +25,11 @@ import org.apache.commons.logging.LogFactory; import org.elasticsearch.hadoop.cfg.ConfigurationOptions; import org.elasticsearch.hadoop.cfg.Settings; -import org.elasticsearch.hadoop.serialization.TemplatedCommand.FieldWriter; +import org.elasticsearch.hadoop.serialization.builder.ValueWriter; +import org.elasticsearch.hadoop.serialization.command.TemplatedCommand.FieldWriter; +import org.elasticsearch.hadoop.serialization.field.ConstantFieldExtractor; +import org.elasticsearch.hadoop.serialization.field.FieldExtractor; +import org.elasticsearch.hadoop.serialization.field.JsonFieldExtractors; import org.elasticsearch.hadoop.util.ObjectUtils; import org.elasticsearch.hadoop.util.StringUtils; @@ -34,6 +38,9 @@ abstract class AbstractCommandFactory implements CommandFactory { private static Log log = LogFactory.getLog(AbstractCommandFactory.class); + private boolean jsonInput; + private JsonFieldExtractors jsonExtractors; + private Settings settings; private ValueWriter valueWriter; private FieldExtractor idExtractor, parentExtractor, routingExtractor, versionExtractor, ttlExtractor, @@ -41,59 +48,75 @@ abstract class AbstractCommandFactory implements CommandFactory { AbstractCommandFactory(Settings settings) { this.settings = settings; - this.valueWriter = ObjectUtils.instantiate(settings.getSerializerValueWriterClassName(), settings); + initFieldExtractors(settings); + } - // init extractors (if needed) - if (settings.getMappingId() != null) { - settings.setProperty(ConstantFieldExtractor.PROPERTY, ConfigurationOptions.ES_MAPPING_ID); - idExtractor = ObjectUtils. instantiate(settings.getMappingIdExtractorClassName(), settings); - } - if (settings.getMappingParent() != null) { - settings.setProperty(ConstantFieldExtractor.PROPERTY, ConfigurationOptions.ES_MAPPING_PARENT); - parentExtractor = ObjectUtils. instantiate(settings.getMappingParentExtractorClassName(), - settings); - } - if (settings.getMappingRouting() != null) { - settings.setProperty(ConstantFieldExtractor.PROPERTY, ConfigurationOptions.ES_MAPPING_ROUTING); - routingExtractor = ObjectUtils. instantiate(settings.getMappingRoutingExtractorClassName(), - settings); - } - if (settings.getMappingTtl() != null) { - settings.setProperty(ConstantFieldExtractor.PROPERTY, ConfigurationOptions.ES_MAPPING_TTL); - ttlExtractor = ObjectUtils. instantiate(settings.getMappingTtlExtractorClassName(), - settings); - } - if (settings.getMappingVersion() != null) { - settings.setProperty(ConstantFieldExtractor.PROPERTY, ConfigurationOptions.ES_MAPPING_VERSION); - versionExtractor = ObjectUtils. instantiate(settings.getMappingVersionExtractorClassName(), - settings); - } - if (settings.getMappingTimestamp() != null) { - settings.setProperty(ConstantFieldExtractor.PROPERTY, ConfigurationOptions.ES_MAPPING_TIMESTAMP); - timestampExtractor = ObjectUtils. instantiate( - settings.getMappingTimestampExtractorClassName(), settings); + private void initFieldExtractors(Settings settings) { + jsonInput = settings.getInputAsJson(); + + if (jsonInput) { + if (log.isDebugEnabled()) { + log.debug("JSON input; using internal field extractor for efficient parsing..."); + } + + jsonExtractors = new JsonFieldExtractors(settings); + idExtractor = jsonExtractors.id(); + parentExtractor = jsonExtractors.parent(); + routingExtractor = jsonExtractors.routing(); + versionExtractor = jsonExtractors.version(); + ttlExtractor = jsonExtractors.ttl(); + timestampExtractor = jsonExtractors.timestamp(); } + else { - if (log.isTraceEnabled()) { - log.trace(String.format("Instantiated value writer [%s]", valueWriter)); - if (idExtractor != null) { - log.trace(String.format("Instantiated id extractor [%s]", idExtractor)); + // init extractors (if needed) + if (settings.getMappingId() != null) { + settings.setProperty(ConstantFieldExtractor.PROPERTY, ConfigurationOptions.ES_MAPPING_ID); + idExtractor = ObjectUtils. instantiate(settings.getMappingIdExtractorClassName(), settings); + } + if (settings.getMappingParent() != null) { + settings.setProperty(ConstantFieldExtractor.PROPERTY, ConfigurationOptions.ES_MAPPING_PARENT); + parentExtractor = ObjectUtils. instantiate(settings.getMappingParentExtractorClassName(), settings); } - if (parentExtractor != null) { - log.trace(String.format("Instantiated parent extractor [%s]", parentExtractor)); + if (settings.getMappingRouting() != null) { + settings.setProperty(ConstantFieldExtractor.PROPERTY, ConfigurationOptions.ES_MAPPING_ROUTING); + routingExtractor = ObjectUtils. instantiate(settings.getMappingRoutingExtractorClassName(), settings); } - if (routingExtractor != null) { - log.trace(String.format("Instantiated routing extractor [%s]", routingExtractor)); + if (settings.getMappingTtl() != null) { + settings.setProperty(ConstantFieldExtractor.PROPERTY, ConfigurationOptions.ES_MAPPING_TTL); + ttlExtractor = ObjectUtils. instantiate(settings.getMappingTtlExtractorClassName(), settings); } - if (ttlExtractor != null) { - log.trace(String.format("Instantiated ttl extractor [%s]", ttlExtractor)); + if (settings.getMappingVersion() != null) { + settings.setProperty(ConstantFieldExtractor.PROPERTY, ConfigurationOptions.ES_MAPPING_VERSION); + versionExtractor = ObjectUtils. instantiate(settings.getMappingVersionExtractorClassName(), settings); } - if (versionExtractor != null) { - log.trace(String.format("Instantiated version extractor [%s]", versionExtractor)); + if (settings.getMappingTimestamp() != null) { + settings.setProperty(ConstantFieldExtractor.PROPERTY, ConfigurationOptions.ES_MAPPING_TIMESTAMP); + timestampExtractor = ObjectUtils. instantiate( + settings.getMappingTimestampExtractorClassName(), settings); } - if (timestampExtractor != null) { - log.trace(String.format("Instantiated timestamp extractor [%s]", timestampExtractor)); + + if (log.isTraceEnabled()) { + log.trace(String.format("Instantiated value writer [%s]", valueWriter)); + if (idExtractor != null) { + log.trace(String.format("Instantiated id extractor [%s]", idExtractor)); + } + if (parentExtractor != null) { + log.trace(String.format("Instantiated parent extractor [%s]", parentExtractor)); + } + if (routingExtractor != null) { + log.trace(String.format("Instantiated routing extractor [%s]", routingExtractor)); + } + if (ttlExtractor != null) { + log.trace(String.format("Instantiated ttl extractor [%s]", ttlExtractor)); + } + if (versionExtractor != null) { + log.trace(String.format("Instantiated version extractor [%s]", versionExtractor)); + } + if (timestampExtractor != null) { + log.trace(String.format("Instantiated timestamp extractor [%s]", timestampExtractor)); + } } } } @@ -130,8 +153,11 @@ public Command createCommand() { List after = new ArrayList(); writeAfterObject(after); + before = compact(before); + after = compact(after); + // compress pieces - return new TemplatedCommand(compact(before), compact(after), valueWriter); + return (jsonInput ? new JsonTemplatedCommand(before, after, jsonExtractors) : new TemplatedCommand(before, after, valueWriter)); } protected void writeAfterObject(List after) { diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/BulkCommands.java b/src/main/java/org/elasticsearch/hadoop/serialization/command/BulkCommands.java similarity index 96% rename from src/main/java/org/elasticsearch/hadoop/serialization/BulkCommands.java rename to src/main/java/org/elasticsearch/hadoop/serialization/command/BulkCommands.java index 30913f558..f4284c131 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/BulkCommands.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/command/BulkCommands.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.command; import org.elasticsearch.hadoop.cfg.ConfigurationOptions; import org.elasticsearch.hadoop.cfg.Settings; diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/Command.java b/src/main/java/org/elasticsearch/hadoop/serialization/command/Command.java similarity index 95% rename from src/main/java/org/elasticsearch/hadoop/serialization/Command.java rename to src/main/java/org/elasticsearch/hadoop/serialization/command/Command.java index 4313ca049..c4b349da2 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/Command.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/command/Command.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.command; import org.elasticsearch.hadoop.util.BytesRef; diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/CommandFactory.java b/src/main/java/org/elasticsearch/hadoop/serialization/command/CommandFactory.java similarity index 93% rename from src/main/java/org/elasticsearch/hadoop/serialization/CommandFactory.java rename to src/main/java/org/elasticsearch/hadoop/serialization/command/CommandFactory.java index 0f1332866..bf830fd30 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/CommandFactory.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/command/CommandFactory.java @@ -16,7 +16,8 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.command; + interface CommandFactory { diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/CreateCommandFactory.java b/src/main/java/org/elasticsearch/hadoop/serialization/command/CreateCommandFactory.java similarity index 95% rename from src/main/java/org/elasticsearch/hadoop/serialization/CreateCommandFactory.java rename to src/main/java/org/elasticsearch/hadoop/serialization/command/CreateCommandFactory.java index d6a71742f..53e287258 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/CreateCommandFactory.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/command/CreateCommandFactory.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.command; import org.elasticsearch.hadoop.cfg.ConfigurationOptions; import org.elasticsearch.hadoop.cfg.Settings; diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/IndexCommandFactory.java b/src/main/java/org/elasticsearch/hadoop/serialization/command/IndexCommandFactory.java similarity index 95% rename from src/main/java/org/elasticsearch/hadoop/serialization/IndexCommandFactory.java rename to src/main/java/org/elasticsearch/hadoop/serialization/command/IndexCommandFactory.java index e07089d70..3f9b6e425 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/IndexCommandFactory.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/command/IndexCommandFactory.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.command; import org.elasticsearch.hadoop.cfg.ConfigurationOptions; import org.elasticsearch.hadoop.cfg.Settings; diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/command/JsonTemplatedCommand.java b/src/main/java/org/elasticsearch/hadoop/serialization/command/JsonTemplatedCommand.java new file mode 100644 index 000000000..2b120ea77 --- /dev/null +++ b/src/main/java/org/elasticsearch/hadoop/serialization/command/JsonTemplatedCommand.java @@ -0,0 +1,85 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.hadoop.serialization.command; + +import java.util.Collection; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; +import org.elasticsearch.hadoop.serialization.builder.NoOpValueWriter; +import org.elasticsearch.hadoop.serialization.builder.ValueWriter; +import org.elasticsearch.hadoop.serialization.field.JsonFieldExtractors; +import org.elasticsearch.hadoop.util.Assert; +import org.elasticsearch.hadoop.util.BytesArray; + +/** + * Dedicated JSON command that skips the content generation phase (since the data is already JSON). + */ +class JsonTemplatedCommand extends TemplatedCommand { + + private static Log log = LogFactory.getLog(JsonTemplatedCommand.class); + + private final JsonFieldExtractors jsonExtractors; + + public JsonTemplatedCommand(Collection beforeObject, Collection afterObject, JsonFieldExtractors jsonExtractors) { + super(beforeObject, afterObject, new NoOpValueWriter()); + this.jsonExtractors = jsonExtractors; + } + + @Override + protected Object preProcess(Object object, BytesArray storage) { + // serialize the json early on and copy it to storage + Assert.notNull(object, "Empty/null JSON document given..."); + Assert.isTrue(object instanceof Writable, + String.format("Class [%s] not supported; only Hadoop Writables", object.getClass())); + + // handle common cases + if (object instanceof Text) { + Text t = (Text) object; + storage.bytes(t.getBytes(), t.getLength()); + } + else if (object instanceof BytesWritable) { + BytesWritable b = (BytesWritable) object; + storage.bytes(b.getBytes(), b.getLength()); + } + else { + // fall-back to generic toString contract + if (log.isTraceEnabled()) { + log.trace(String.format("Unknown Writable type for object [%s], using default toString()", object)); + } + + storage.bytes(object.toString()); + } + + if (log.isTraceEnabled()) { + log.trace(String.format("About to extract information from [%s]", storage)); + } + + jsonExtractors.process(storage); + return storage; + } + + @Override + protected void doWriteObject(Object object, BytesArray storage, ValueWriter writer) { + // no-op - the object has been already serialized to storage + } +} diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/TemplatedCommand.java b/src/main/java/org/elasticsearch/hadoop/serialization/command/TemplatedCommand.java similarity index 78% rename from src/main/java/org/elasticsearch/hadoop/serialization/TemplatedCommand.java rename to src/main/java/org/elasticsearch/hadoop/serialization/command/TemplatedCommand.java index af4e115c4..1ec17ad64 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/TemplatedCommand.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/command/TemplatedCommand.java @@ -16,10 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.command; import java.util.Collection; +import org.elasticsearch.hadoop.serialization.builder.ContentBuilder; +import org.elasticsearch.hadoop.serialization.builder.ValueWriter; +import org.elasticsearch.hadoop.serialization.field.FieldExtractor; import org.elasticsearch.hadoop.util.Assert; import org.elasticsearch.hadoop.util.BytesArray; import org.elasticsearch.hadoop.util.BytesRef; @@ -65,19 +68,28 @@ BytesArray write(Object object) { @Override public BytesRef write(Object object) { ref.reset(); + scratchPad.reset(); + Object processed = preProcess(object, scratchPad); // write before object - writeTemplate(beforeObject, object); + writeTemplate(beforeObject, processed); // write object - scratchPad.reset(); - FastByteArrayOutputStream bos = new FastByteArrayOutputStream(scratchPad); - ContentBuilder.generate(bos, valueWriter).value(object).flush().close(); + doWriteObject(processed, scratchPad, valueWriter); ref.add(scratchPad); // writer after object - writeTemplate(afterObject, object); + writeTemplate(afterObject, processed); return ref; } + protected Object preProcess(Object object, BytesArray storage) { + return object; + } + + protected void doWriteObject(Object object, BytesArray storage, ValueWriter writer) { + FastByteArrayOutputStream bos = new FastByteArrayOutputStream(storage); + ContentBuilder.generate(bos, writer).value(object).flush().close(); + } + private void writeTemplate(Collection template, Object object) { for (Object item : template) { if (item instanceof byte[]) { diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/UpdateCommandFactory.java b/src/main/java/org/elasticsearch/hadoop/serialization/command/UpdateCommandFactory.java similarity index 94% rename from src/main/java/org/elasticsearch/hadoop/serialization/UpdateCommandFactory.java rename to src/main/java/org/elasticsearch/hadoop/serialization/command/UpdateCommandFactory.java index 9e880a254..35a9b6d86 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/UpdateCommandFactory.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/command/UpdateCommandFactory.java @@ -16,12 +16,13 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.command; import java.util.List; import org.elasticsearch.hadoop.cfg.ConfigurationOptions; import org.elasticsearch.hadoop.cfg.Settings; +import org.elasticsearch.hadoop.serialization.field.FieldExtractor; import org.elasticsearch.hadoop.util.Assert; class UpdateCommandFactory extends AbstractCommandFactory { diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/ConstantFieldExtractor.java b/src/main/java/org/elasticsearch/hadoop/serialization/field/ConstantFieldExtractor.java similarity index 94% rename from src/main/java/org/elasticsearch/hadoop/serialization/ConstantFieldExtractor.java rename to src/main/java/org/elasticsearch/hadoop/serialization/field/ConstantFieldExtractor.java index 99e738b1a..c8b5083b4 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/ConstantFieldExtractor.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/field/ConstantFieldExtractor.java @@ -16,9 +16,10 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.field; import org.elasticsearch.hadoop.cfg.Settings; +import org.elasticsearch.hadoop.serialization.SettingsAware; public class ConstantFieldExtractor implements FieldExtractor, SettingsAware { diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/FieldExtractor.java b/src/main/java/org/elasticsearch/hadoop/serialization/field/FieldExtractor.java similarity index 89% rename from src/main/java/org/elasticsearch/hadoop/serialization/FieldExtractor.java rename to src/main/java/org/elasticsearch/hadoop/serialization/field/FieldExtractor.java index e02e696d5..ada181ecd 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/FieldExtractor.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/field/FieldExtractor.java @@ -16,7 +16,9 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.field; + +import org.elasticsearch.hadoop.serialization.SettingsAware; /** * Basic extractor (for serialization purposes) of a field. Typically used with {@link SettingsAware} for configuration/injection purposes. diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/field/JsonFieldExtractors.java b/src/main/java/org/elasticsearch/hadoop/serialization/field/JsonFieldExtractors.java new file mode 100644 index 000000000..1b92dc758 --- /dev/null +++ b/src/main/java/org/elasticsearch/hadoop/serialization/field/JsonFieldExtractors.java @@ -0,0 +1,133 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.hadoop.serialization.field; + +import java.util.ArrayList; +import java.util.List; + +import org.elasticsearch.hadoop.cfg.Settings; +import org.elasticsearch.hadoop.serialization.ParsingUtils; +import org.elasticsearch.hadoop.serialization.json.JacksonJsonParser; +import org.elasticsearch.hadoop.util.BytesArray; +import org.elasticsearch.hadoop.util.ObjectUtils; + +public class JsonFieldExtractors { + + + private final List results = new ArrayList(6); + private int usedSlots = 0; + private String[] paths; + + private FieldExtractor id, parent, routing, ttl, version, timestamp; + + private class PrecomputedFieldExtractor implements FieldExtractor { + + private final int slot; + + public PrecomputedFieldExtractor(int slot) { + this.slot = slot; + } + + @Override + public String field(Object target) { + return results.get(slot); + } + } + + private static class FixedFieldExtractor implements FieldExtractor { + private final String value; + + public FixedFieldExtractor(String value) { + this.value = value; + } + + @Override + public String field(Object target) { + return value; + } + } + + public JsonFieldExtractors(Settings settings) { + List jsonPaths = new ArrayList(6); + + id = init(settings.getMappingId(), jsonPaths); + parent = init(settings.getMappingParent(), jsonPaths); + routing = init(settings.getMappingRouting(), jsonPaths); + ttl = init(settings.getMappingTtl(), jsonPaths); + version = init(settings.getMappingVersion(), jsonPaths); + timestamp = init(settings.getMappingTimestamp(), jsonPaths); + + paths = jsonPaths.toArray(new String[jsonPaths.size()]); + } + + private FieldExtractor init(String setting, List pathList) { + if (setting != null) { + String constant = initConstant(setting); + if (constant != null) { + return new FixedFieldExtractor(constant); + } + else { + pathList.add(setting); + return new PrecomputedFieldExtractor(usedSlots++); + } + } + return null; + } + + private String initConstant(String field) { + if (field != null && field.startsWith("<") && field.endsWith(">")) { + return field.substring(1, field.length() - 1); + } + return null; + } + + public FieldExtractor id() { + return id; + } + + public FieldExtractor parent() { + return parent; + } + + public FieldExtractor routing() { + return routing; + } + + public FieldExtractor ttl() { + return ttl; + } + + public FieldExtractor version() { + return version; + } + + public FieldExtractor timestamp() { + return timestamp; + } + + public void process(BytesArray storage) { + // no extractors, no lookups + if (ObjectUtils.isEmpty(paths)) { + return; + } + + results.clear(); + results.addAll(ParsingUtils.values(new JacksonJsonParser(storage.bytes(), 0, storage.length()), paths)); + } +} \ No newline at end of file diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/MapWritableFieldExtractor.java b/src/main/java/org/elasticsearch/hadoop/serialization/field/MapWritableFieldExtractor.java similarity index 96% rename from src/main/java/org/elasticsearch/hadoop/serialization/MapWritableFieldExtractor.java rename to src/main/java/org/elasticsearch/hadoop/serialization/field/MapWritableFieldExtractor.java index 4468ca00c..a9f186be7 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/MapWritableFieldExtractor.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/field/MapWritableFieldExtractor.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.hadoop.serialization; +package org.elasticsearch.hadoop.serialization.field; import java.util.Map; diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/field/NoOpFieldExtractor.java b/src/main/java/org/elasticsearch/hadoop/serialization/field/NoOpFieldExtractor.java new file mode 100644 index 000000000..88bdf0b2d --- /dev/null +++ b/src/main/java/org/elasticsearch/hadoop/serialization/field/NoOpFieldExtractor.java @@ -0,0 +1,27 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.hadoop.serialization.field; + +public class NoOpFieldExtractor implements FieldExtractor { + + @Override + public String field(Object target) { + throw new IllegalStateException("Should not be called - indicates a configuration issue or a bug"); + } +} diff --git a/src/main/java/org/elasticsearch/hadoop/serialization/json/JacksonJsonParser.java b/src/main/java/org/elasticsearch/hadoop/serialization/json/JacksonJsonParser.java index fe4ec4e4f..88dafd30d 100644 --- a/src/main/java/org/elasticsearch/hadoop/serialization/json/JacksonJsonParser.java +++ b/src/main/java/org/elasticsearch/hadoop/serialization/json/JacksonJsonParser.java @@ -49,8 +49,12 @@ public JacksonJsonParser(InputStream in) { } public JacksonJsonParser(byte[] content) { + this(content, 0, content.length); + } + + public JacksonJsonParser(byte[] content, int offset, int length) { try { - this.parser = JSON_FACTORY.createJsonParser(content); + this.parser = JSON_FACTORY.createJsonParser(content, offset, length); } catch (IOException ex) { throw new SerializationException(ex); } diff --git a/src/test/java/org/elasticsearch/hadoop/integration/mr/MRNewApiSaveTest.java b/src/test/java/org/elasticsearch/hadoop/integration/mr/MRNewApiSaveTest.java index 9219737fd..d9b3a22cd 100644 --- a/src/test/java/org/elasticsearch/hadoop/integration/mr/MRNewApiSaveTest.java +++ b/src/test/java/org/elasticsearch/hadoop/integration/mr/MRNewApiSaveTest.java @@ -20,12 +20,15 @@ import java.io.File; import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; import java.util.LinkedHashMap; import java.util.Map; import java.util.StringTokenizer; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Text; import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Mapper; import org.apache.hadoop.mapreduce.lib.input.TextInputFormat; @@ -38,11 +41,15 @@ import org.elasticsearch.hadoop.util.WritableUtils; import org.junit.FixMethodOrder; import org.junit.Test; +import org.junit.runner.RunWith; import org.junit.runners.MethodSorters; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; import static org.junit.Assert.*; @FixMethodOrder(MethodSorters.NAME_ASCENDING) +@RunWith(Parameterized.class) public class MRNewApiSaveTest { public static class JsonMapper extends Mapper { @@ -62,12 +69,55 @@ protected void map(Object key, Object value, Context context) throws IOException } } + @Parameters + public static Collection configs() throws IOException { + Configuration conf = HdpBootstrap.hadoopConfig(); + HadoopCfgUtils.setGenericOptions(conf); + + Job job = new Job(conf); + job.setInputFormatClass(TextInputFormat.class); + job.setOutputFormatClass(EsOutputFormat.class); + job.setMapOutputValueClass(LinkedMapWritable.class); + job.setMapperClass(JsonMapper.class); + job.setNumReduceTasks(0); + + + Job standard = new Job(job.getConfiguration()); + File fl = new File("src/test/resources/artists.dat"); + long splitSize = fl.length() / 3; + TextInputFormat.setMaxInputSplitSize(standard, splitSize); + TextInputFormat.setMinInputSplitSize(standard, 50); + + standard.setMapperClass(JsonMapper.class); + standard.setMapOutputValueClass(LinkedMapWritable.class); + TextInputFormat.addInputPath(standard, new Path("src/test/resources/artists.dat")); + + Job json = new Job(job.getConfiguration()); + json.setMapperClass(Mapper.class); + json.setMapOutputValueClass(Text.class); + json.getConfiguration().set(ConfigurationOptions.ES_INPUT_JSON, "true"); + TextInputFormat.addInputPath(json, new Path("src/test/resources/artists.json")); + + return Arrays.asList(new Object[][] { + { standard, "" }, + { json, "json-" } }); + } + + private String indexPrefix = ""; + private Configuration config; + + public MRNewApiSaveTest(Job job, String indexPrefix) { + this.indexPrefix = indexPrefix; + this.config = job.getConfiguration(); + } + + @Test public void testBasicSave() throws Exception { Configuration conf = createConf(); conf.set(ConfigurationOptions.ES_RESOURCE, "mrnewapi/save"); - new Job(conf).waitForCompletion(true); + runJob(conf); } @Test @@ -76,7 +126,7 @@ public void testSaveWithId() throws Exception { conf.set(ConfigurationOptions.ES_RESOURCE, "mrnewapi/savewithid"); conf.set(ConfigurationOptions.ES_MAPPING_ID, "number"); - new Job(conf).waitForCompletion(true); + runJob(conf); } @Test @@ -86,7 +136,7 @@ public void testCreateWithId() throws Exception { conf.set(ConfigurationOptions.ES_MAPPING_ID, "number"); conf.set(ConfigurationOptions.ES_RESOURCE, "mrnewapi/createwithid"); - new Job(conf).waitForCompletion(true); + runJob(conf); } @Test @@ -96,7 +146,7 @@ public void testCreateWithIdShouldFailOnDuplicate() throws Exception { conf.set(ConfigurationOptions.ES_MAPPING_ID, "number"); conf.set(ConfigurationOptions.ES_RESOURCE, "mrnewapi/createwithid"); - assertFalse("job should have failed", new Job(conf).waitForCompletion(true)); + assertFalse("job should have failed", runJob(conf)); } @Test(expected = IllegalArgumentException.class) @@ -105,7 +155,7 @@ public void testUpdateWithoutId() throws Exception { conf.set(ConfigurationOptions.ES_WRITE_OPERATION, "update"); conf.set(ConfigurationOptions.ES_RESOURCE, "mrnewapi/update"); - new Job(conf).waitForCompletion(true); + runJob(conf); } @Test @@ -115,7 +165,7 @@ public void testUpdateWithId() throws Exception { conf.set(ConfigurationOptions.ES_MAPPING_ID, "number"); conf.set(ConfigurationOptions.ES_RESOURCE, "mrnewapi/update"); - new Job(conf).waitForCompletion(true); + runJob(conf); } @Test @@ -126,7 +176,7 @@ public void testUpdateWithoutUpsert() throws Exception { conf.set(ConfigurationOptions.ES_RESOURCE, "mrnewapi/updatewoupsert"); conf.set(ConfigurationOptions.ES_UPSERT_DOC, "false"); - assertFalse("job should have failed", new Job(conf).waitForCompletion(true)); + assertFalse("job should have failed", runJob(conf)); } @Test(expected = IllegalArgumentException.class) @@ -135,7 +185,7 @@ public void testIndexAutoCreateDisabled() throws Exception { conf.set(ConfigurationOptions.ES_RESOURCE, "mrnewapi/non-existing"); conf.set(ConfigurationOptions.ES_INDEX_AUTO_CREATE, "no"); - new Job(conf).waitForCompletion(true); + runJob(conf); } @Test @@ -145,27 +195,17 @@ public void testParentChild() throws Exception { conf.set(ConfigurationOptions.ES_INDEX_AUTO_CREATE, "no"); conf.set(ConfigurationOptions.ES_MAPPING_PARENT, "number"); - RestUtils.putMapping("mrnewapi/child", "org/elasticsearch/hadoop/integration/mr-child.json"); + RestUtils.putMapping(indexPrefix + "mrnewapi/child", "org/elasticsearch/hadoop/integration/mr-child.json"); - new Job(conf).waitForCompletion(true); + runJob(conf); } private Configuration createConf() throws IOException { - Configuration conf = HdpBootstrap.hadoopConfig(); - HadoopCfgUtils.setGenericOptions(conf); - - Job job = new Job(conf); - job.setInputFormatClass(TextInputFormat.class); - job.setOutputFormatClass(EsOutputFormat.class); - job.setMapOutputValueClass(LinkedMapWritable.class); - job.setMapperClass(JsonMapper.class); - job.setNumReduceTasks(0); + return new Configuration(config); + } - TextInputFormat.addInputPath(job, new Path("src/test/resources/artists.dat")); - File fl = new File("src/test/resources/artists.dat"); - long splitSize = fl.length() / 3; - TextInputFormat.setMaxInputSplitSize(job, splitSize); - TextInputFormat.setMinInputSplitSize(job, 50); - return job.getConfiguration(); + private boolean runJob(Configuration conf) throws Exception { + conf.set(ConfigurationOptions.ES_RESOURCE, indexPrefix + conf.get(ConfigurationOptions.ES_RESOURCE)); + return new Job(conf).waitForCompletion(true); } } \ No newline at end of file diff --git a/src/test/java/org/elasticsearch/hadoop/integration/mr/MROldApiSaveTest.java b/src/test/java/org/elasticsearch/hadoop/integration/mr/MROldApiSaveTest.java index 22882a879..7f37e6b19 100644 --- a/src/test/java/org/elasticsearch/hadoop/integration/mr/MROldApiSaveTest.java +++ b/src/test/java/org/elasticsearch/hadoop/integration/mr/MROldApiSaveTest.java @@ -19,11 +19,14 @@ package org.elasticsearch.hadoop.integration.mr; import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; import java.util.LinkedHashMap; import java.util.Map; import java.util.StringTokenizer; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.io.Text; import org.apache.hadoop.mapred.FileInputFormat; import org.apache.hadoop.mapred.InputSplit; import org.apache.hadoop.mapred.JobClient; @@ -33,6 +36,7 @@ import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.TextInputFormat; +import org.apache.hadoop.mapred.lib.IdentityMapper; import org.apache.hadoop.mapred.lib.IdentityReducer; import org.elasticsearch.hadoop.cfg.ConfigurationOptions; import org.elasticsearch.hadoop.integration.HdpBootstrap; @@ -43,19 +47,15 @@ import org.elasticsearch.hadoop.util.WritableUtils; import org.junit.FixMethodOrder; import org.junit.Test; +import org.junit.runner.RunWith; import org.junit.runners.MethodSorters; +import org.junit.runners.Parameterized; +import org.junit.runners.Parameterized.Parameters; @FixMethodOrder(MethodSorters.NAME_ASCENDING) +@RunWith(Parameterized.class) public class MROldApiSaveTest { - public static class SplittableTextInputFormat extends TextInputFormat { - - @Override - public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { - return super.getSplits(job, job.getInt("actual.splits", 3)); - } - } - public static class JsonMapper extends MapReduceBase implements Mapper { @Override @@ -73,13 +73,60 @@ public void map(Object key, Object value, OutputCollector output, Reporter repor output.collect(key, WritableUtils.toWritable(entry)); } } + public static class SplittableTextInputFormat extends TextInputFormat { + + @Override + public InputSplit[] getSplits(JobConf job, int numSplits) throws IOException { + return super.getSplits(job, job.getInt("actual.splits", 3)); + } + } + + @Parameters + public static Collection configs() { + JobConf conf = HdpBootstrap.hadoopConfig(); + + conf.setInputFormat(SplittableTextInputFormat.class); + conf.setOutputFormat(EsOutputFormat.class); + conf.setReducerClass(IdentityReducer.class); + HadoopCfgUtils.setGenericOptions(conf); + conf.setNumMapTasks(2); + conf.setInt("actual.splits", 2); + conf.setNumReduceTasks(0); + + + JobConf standard = new JobConf(conf); + standard.setMapperClass(JsonMapper.class); + standard.setMapOutputValueClass(LinkedMapWritable.class); + standard.set(ConfigurationOptions.ES_INPUT_JSON, "false"); + FileInputFormat.setInputPaths(standard, new Path("src/test/resources/artists.dat")); + + JobConf json = new JobConf(conf); + json.setMapperClass(IdentityMapper.class); + json.setMapOutputValueClass(Text.class); + json.set(ConfigurationOptions.ES_INPUT_JSON, "true"); + FileInputFormat.setInputPaths(json, new Path("src/test/resources/artists.json")); + + return Arrays.asList(new Object[][] { + { standard, "" }, + { json, "json-" } + }); + } + + private String indexPrefix = ""; + private JobConf config; + + public MROldApiSaveTest(JobConf config, String indexPrefix) { + this.indexPrefix = indexPrefix; + this.config = config; + } + @Test public void testBasicIndex() throws Exception { JobConf conf = createJobConf(); conf.set(ConfigurationOptions.ES_RESOURCE, "mroldapi/save"); - JobClient.runJob(conf); + runJob(conf); } @Test @@ -88,7 +135,7 @@ public void testBasicIndexWithId() throws Exception { conf.set(ConfigurationOptions.ES_MAPPING_ID, "number"); conf.set(ConfigurationOptions.ES_RESOURCE, "mroldapi/savewithid"); - JobClient.runJob(conf); + runJob(conf); } @Test @@ -98,7 +145,7 @@ public void testCreateWithId() throws Exception { conf.set(ConfigurationOptions.ES_MAPPING_ID, "number"); conf.set(ConfigurationOptions.ES_RESOURCE, "mroldapi/createwithid"); - JobClient.runJob(conf); + runJob(conf); } @Test(expected = IOException.class) @@ -112,7 +159,7 @@ public void testUpdateWithoutId() throws Exception { conf.set(ConfigurationOptions.ES_WRITE_OPERATION, "update"); conf.set(ConfigurationOptions.ES_RESOURCE, "mroldapi/update"); - JobClient.runJob(conf); + runJob(conf); } @Test @@ -122,7 +169,7 @@ public void testUpdateWithId() throws Exception { conf.set(ConfigurationOptions.ES_MAPPING_ID, "number"); conf.set(ConfigurationOptions.ES_RESOURCE, "mroldapi/update"); - JobClient.runJob(conf); + runJob(conf); } @Test(expected = IOException.class) @@ -133,7 +180,7 @@ public void testUpdateWithoutUpsert() throws Exception { conf.set(ConfigurationOptions.ES_RESOURCE, "mroldapi/updatewoupsert"); conf.set(ConfigurationOptions.ES_UPSERT_DOC, "false"); - JobClient.runJob(conf); + runJob(conf); } @Test(expected = IllegalArgumentException.class) @@ -142,7 +189,7 @@ public void testIndexAutoCreateDisabled() throws Exception { conf.set(ConfigurationOptions.ES_RESOURCE, "mroldapi/non-existing"); conf.set(ConfigurationOptions.ES_INDEX_AUTO_CREATE, "no"); - JobClient.runJob(conf); + runJob(conf); } @Test @@ -152,9 +199,9 @@ public void testParentChild() throws Exception { conf.set(ConfigurationOptions.ES_INDEX_AUTO_CREATE, "no"); conf.set(ConfigurationOptions.ES_MAPPING_PARENT, "number"); - RestUtils.putMapping("mroldapi/child", "org/elasticsearch/hadoop/integration/mr-child.json"); + RestUtils.putMapping(indexPrefix + "mroldapi/child", "org/elasticsearch/hadoop/integration/mr-child.json"); - JobClient.runJob(conf); + runJob(conf); } //@Test @@ -163,25 +210,17 @@ public void testNested() throws Exception { conf.set(ConfigurationOptions.ES_RESOURCE, "mroldapi/nested"); conf.set(ConfigurationOptions.ES_INDEX_AUTO_CREATE, "no"); - RestUtils.putMapping("mroldapi/nested", "org/elasticsearch/hadoop/integration/mr-nested.json"); + RestUtils.putMapping(indexPrefix + "mroldapi/nested", "org/elasticsearch/hadoop/integration/mr-nested.json"); - JobClient.runJob(conf); + runJob(conf); } private JobConf createJobConf() { - JobConf conf = HdpBootstrap.hadoopConfig(); - - conf.setInputFormat(SplittableTextInputFormat.class); - conf.setOutputFormat(EsOutputFormat.class); - conf.setMapOutputValueClass(LinkedMapWritable.class); - conf.setMapperClass(JsonMapper.class); - conf.setReducerClass(IdentityReducer.class); - HadoopCfgUtils.setGenericOptions(conf); - conf.setNumMapTasks(2); - conf.setInt("actual.splits", 2); - conf.setNumReduceTasks(0); + return new JobConf(config); + } - FileInputFormat.setInputPaths(conf, new Path("src/test/resources/artists.dat")); - return conf; + private void runJob(JobConf conf) throws Exception { + conf.set(ConfigurationOptions.ES_RESOURCE, indexPrefix + conf.get(ConfigurationOptions.ES_RESOURCE)); + JobClient.runJob(conf); } } \ No newline at end of file diff --git a/src/test/java/org/elasticsearch/hadoop/integration/mr/MRSuite.java b/src/test/java/org/elasticsearch/hadoop/integration/mr/MRSuite.java index 24658977c..346cd0dfe 100644 --- a/src/test/java/org/elasticsearch/hadoop/integration/mr/MRSuite.java +++ b/src/test/java/org/elasticsearch/hadoop/integration/mr/MRSuite.java @@ -28,7 +28,7 @@ @RunWith(Suite.class) @Suite.SuiteClasses({ MROldApiSaveTest.class, MROldApiSearchTest.class, MRNewApiSaveTest.class, MRNewApiSearchTest.class }) -//@Suite.SuiteClasses({ MRNewApiSaveTest.class, MRNewApiSearchTest.class }) +//@Suite.SuiteClasses({ MRNewApiSaveTest.class }) public class MRSuite { @ClassRule public static ExternalResource resource = new LocalEs(); @@ -36,5 +36,6 @@ public class MRSuite { @BeforeClass public static void setupHdfs() throws Exception { HdfsUtils.copyFromLocal("src/test/resources/artists.dat"); + HdfsUtils.copyFromLocal("src/test/resources/artists.json"); } -} +} \ No newline at end of file diff --git a/src/test/java/org/elasticsearch/hadoop/integration/rest/RestQueryTest.java b/src/test/java/org/elasticsearch/hadoop/integration/rest/RestQueryTest.java index 119b6c37c..704af739a 100644 --- a/src/test/java/org/elasticsearch/hadoop/integration/rest/RestQueryTest.java +++ b/src/test/java/org/elasticsearch/hadoop/integration/rest/RestQueryTest.java @@ -30,9 +30,9 @@ import org.elasticsearch.hadoop.rest.dto.Node; import org.elasticsearch.hadoop.rest.dto.Shard; import org.elasticsearch.hadoop.rest.dto.mapping.Field; -import org.elasticsearch.hadoop.serialization.JdkValueReader; -import org.elasticsearch.hadoop.serialization.JdkValueWriter; import org.elasticsearch.hadoop.serialization.ScrollReader; +import org.elasticsearch.hadoop.serialization.builder.JdkValueReader; +import org.elasticsearch.hadoop.serialization.builder.JdkValueWriter; import org.elasticsearch.hadoop.util.TestSettings; import org.junit.After; import org.junit.Before; diff --git a/src/test/java/org/elasticsearch/hadoop/integration/rest/RestSaveTest.java b/src/test/java/org/elasticsearch/hadoop/integration/rest/RestSaveTest.java index a0b0d54f8..392238ed1 100644 --- a/src/test/java/org/elasticsearch/hadoop/integration/rest/RestSaveTest.java +++ b/src/test/java/org/elasticsearch/hadoop/integration/rest/RestSaveTest.java @@ -26,7 +26,7 @@ import org.elasticsearch.hadoop.rest.Resource; import org.elasticsearch.hadoop.rest.RestClient; import org.elasticsearch.hadoop.rest.RestRepository; -import org.elasticsearch.hadoop.serialization.JdkValueWriter; +import org.elasticsearch.hadoop.serialization.builder.JdkValueWriter; import org.elasticsearch.hadoop.util.BytesArray; import org.elasticsearch.hadoop.util.TestSettings; import org.elasticsearch.hadoop.util.TrackingBytesArray; diff --git a/src/test/java/org/elasticsearch/hadoop/rest/ScrollReaderTest.java b/src/test/java/org/elasticsearch/hadoop/rest/ScrollReaderTest.java index 202f6ec50..9253f314c 100644 --- a/src/test/java/org/elasticsearch/hadoop/rest/ScrollReaderTest.java +++ b/src/test/java/org/elasticsearch/hadoop/rest/ScrollReaderTest.java @@ -23,8 +23,8 @@ import java.util.List; import java.util.Map; -import org.elasticsearch.hadoop.serialization.JdkValueReader; import org.elasticsearch.hadoop.serialization.ScrollReader; +import org.elasticsearch.hadoop.serialization.builder.JdkValueReader; import org.junit.Test; import static org.junit.Assert.*; diff --git a/src/test/java/org/elasticsearch/hadoop/serialization/CommandTest.java b/src/test/java/org/elasticsearch/hadoop/serialization/CommandTest.java index 32218ad97..b592fdb6e 100644 --- a/src/test/java/org/elasticsearch/hadoop/serialization/CommandTest.java +++ b/src/test/java/org/elasticsearch/hadoop/serialization/CommandTest.java @@ -26,6 +26,9 @@ import org.elasticsearch.hadoop.cfg.ConfigurationOptions; import org.elasticsearch.hadoop.cfg.Settings; import org.elasticsearch.hadoop.rest.InitializationUtils; +import org.elasticsearch.hadoop.serialization.builder.JdkValueWriter; +import org.elasticsearch.hadoop.serialization.command.BulkCommands; +import org.elasticsearch.hadoop.serialization.command.Command; import org.elasticsearch.hadoop.util.BytesArray; import org.elasticsearch.hadoop.util.TestSettings; import org.junit.Before; @@ -163,7 +166,7 @@ private Command create(Settings settings) { private Settings settings() { Settings set = new TestSettings(); - SerializationUtils.setValueWriterIfNotSet(set, JdkValueWriter.class, null); + InitializationUtils.setValueWriterIfNotSet(set, JdkValueWriter.class, null); InitializationUtils.setFieldExtractorIfNotSet(set, MapFieldExtractor.class, null); if (isUpdateOp()) { set.setProperty(ConfigurationOptions.ES_MAPPING_ID, "<2>"); diff --git a/src/test/java/org/elasticsearch/hadoop/serialization/HiveTypeToJsonTest.java b/src/test/java/org/elasticsearch/hadoop/serialization/HiveTypeToJsonTest.java index 2a948a29b..d59c2c502 100644 --- a/src/test/java/org/elasticsearch/hadoop/serialization/HiveTypeToJsonTest.java +++ b/src/test/java/org/elasticsearch/hadoop/serialization/HiveTypeToJsonTest.java @@ -39,6 +39,7 @@ import org.apache.hadoop.io.Text; import org.elasticsearch.hadoop.hive.HiveType; import org.elasticsearch.hadoop.hive.HiveValueWriter; +import org.elasticsearch.hadoop.serialization.builder.ContentBuilder; import org.elasticsearch.hadoop.util.FastByteArrayOutputStream; import org.junit.After; import org.junit.AfterClass; diff --git a/src/test/java/org/elasticsearch/hadoop/serialization/JdkTypeToJsonTest.java b/src/test/java/org/elasticsearch/hadoop/serialization/JdkTypeToJsonTest.java index f724bda82..884be44fa 100644 --- a/src/test/java/org/elasticsearch/hadoop/serialization/JdkTypeToJsonTest.java +++ b/src/test/java/org/elasticsearch/hadoop/serialization/JdkTypeToJsonTest.java @@ -21,6 +21,8 @@ import java.util.Arrays; import java.util.Collections; +import org.elasticsearch.hadoop.serialization.builder.ContentBuilder; +import org.elasticsearch.hadoop.serialization.builder.JdkValueWriter; import org.elasticsearch.hadoop.util.FastByteArrayOutputStream; import org.junit.After; import org.junit.AfterClass; diff --git a/src/test/java/org/elasticsearch/hadoop/serialization/JsonValuePathTest.java b/src/test/java/org/elasticsearch/hadoop/serialization/JsonValuePathTest.java new file mode 100644 index 000000000..7f3730df9 --- /dev/null +++ b/src/test/java/org/elasticsearch/hadoop/serialization/JsonValuePathTest.java @@ -0,0 +1,64 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.hadoop.serialization; + +import java.io.InputStream; +import java.util.List; + +import org.elasticsearch.hadoop.serialization.json.JacksonJsonParser; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class JsonValuePathTest { + + private Parser parser; + + @Before + public void before() { + InputStream in = getClass().getResourceAsStream("parser-test-nested.json"); + parser = new JacksonJsonParser(in); + } + + @After + public void after() { + parser.close(); + } + + @Test + public void testFirstLevel() throws Exception { + List vals = ParsingUtils.values(parser, "firstName", "foo", "age"); + assertEquals(3, vals.size()); + assertEquals("John", vals.get(0)); + assertNull(vals.get(1)); + assertEquals("25", vals.get(2)); + } + + @Test + public void testSecondLevel() throws Exception { + List vals = ParsingUtils.values(parser, "address.state", "address.foo", "address.building.floors", "address.building.bar"); + assertEquals(4, vals.size()); + assertEquals("NY", vals.get(0)); + assertNull(vals.get(1)); + assertEquals("10", vals.get(2)); + assertNull(vals.get(3)); + } +} \ No newline at end of file diff --git a/src/test/java/org/elasticsearch/hadoop/serialization/MapFieldExtractor.java b/src/test/java/org/elasticsearch/hadoop/serialization/MapFieldExtractor.java index 90a7cf3b9..7b3a64fbd 100644 --- a/src/test/java/org/elasticsearch/hadoop/serialization/MapFieldExtractor.java +++ b/src/test/java/org/elasticsearch/hadoop/serialization/MapFieldExtractor.java @@ -21,16 +21,16 @@ import java.util.Map; import org.elasticsearch.hadoop.cfg.Settings; +import org.elasticsearch.hadoop.serialization.field.ConstantFieldExtractor; public class MapFieldExtractor extends ConstantFieldExtractor { - private String fieldName; @SuppressWarnings("rawtypes") @Override protected String extractField(Object target) { if (target instanceof Map) { Map map = (Map) target; - Object w = map.get(fieldName); + Object w = map.get(getFieldName()); // since keys are likely primitives, just do a toString return (w != null ? w.toString() : null); } @@ -40,6 +40,5 @@ protected String extractField(Object target) { @Override public void setSettings(Settings settings) { super.setSettings(settings); - fieldName = getFieldName(); } } diff --git a/src/test/java/org/elasticsearch/hadoop/serialization/PigTypeToJsonTest.java b/src/test/java/org/elasticsearch/hadoop/serialization/PigTypeToJsonTest.java index 5a9d99e4c..7e160de8b 100644 --- a/src/test/java/org/elasticsearch/hadoop/serialization/PigTypeToJsonTest.java +++ b/src/test/java/org/elasticsearch/hadoop/serialization/PigTypeToJsonTest.java @@ -31,6 +31,7 @@ import org.apache.pig.impl.util.Utils; import org.elasticsearch.hadoop.pig.PigTuple; import org.elasticsearch.hadoop.pig.PigValueWriter; +import org.elasticsearch.hadoop.serialization.builder.ContentBuilder; import org.elasticsearch.hadoop.util.FastByteArrayOutputStream; import org.junit.After; import org.junit.AfterClass; diff --git a/src/test/java/org/elasticsearch/hadoop/serialization/ValueReaderTest.java b/src/test/java/org/elasticsearch/hadoop/serialization/ValueReaderTest.java index c5d19d7e4..5324c104d 100644 --- a/src/test/java/org/elasticsearch/hadoop/serialization/ValueReaderTest.java +++ b/src/test/java/org/elasticsearch/hadoop/serialization/ValueReaderTest.java @@ -20,6 +20,7 @@ import java.io.InputStream; +import org.elasticsearch.hadoop.serialization.builder.JdkValueReader; import org.junit.After; import org.junit.Before; import org.junit.Test; diff --git a/src/test/java/org/elasticsearch/hadoop/serialization/WritableTypeFromJsonTest.java b/src/test/java/org/elasticsearch/hadoop/serialization/WritableTypeFromJsonTest.java index 18a7bc779..f7cfa4be1 100644 --- a/src/test/java/org/elasticsearch/hadoop/serialization/WritableTypeFromJsonTest.java +++ b/src/test/java/org/elasticsearch/hadoop/serialization/WritableTypeFromJsonTest.java @@ -22,6 +22,7 @@ import org.elasticsearch.hadoop.mr.WritableValueReader; import org.elasticsearch.hadoop.serialization.Parser.NumberType; import org.elasticsearch.hadoop.serialization.Parser.Token; +import org.elasticsearch.hadoop.serialization.builder.ValueReader; import org.elasticsearch.hadoop.serialization.json.JacksonJsonParser; import org.junit.Before; import org.junit.Test; diff --git a/src/test/java/org/elasticsearch/hadoop/serialization/WritableTypeToJsonTest.java b/src/test/java/org/elasticsearch/hadoop/serialization/WritableTypeToJsonTest.java index 682bc9901..15b2d1390 100644 --- a/src/test/java/org/elasticsearch/hadoop/serialization/WritableTypeToJsonTest.java +++ b/src/test/java/org/elasticsearch/hadoop/serialization/WritableTypeToJsonTest.java @@ -35,6 +35,7 @@ import org.apache.hadoop.io.Writable; import org.elasticsearch.hadoop.mr.LinkedMapWritable; import org.elasticsearch.hadoop.mr.WritableValueWriter; +import org.elasticsearch.hadoop.serialization.builder.ContentBuilder; import org.elasticsearch.hadoop.util.FastByteArrayOutputStream; import org.junit.After; import org.junit.AfterClass; diff --git a/src/test/resources/artists.json b/src/test/resources/artists.json new file mode 100644 index 000000000..3aa7674a0 --- /dev/null +++ b/src/test/resources/artists.json @@ -0,0 +1,953 @@ +{"number":"1","name":"MALICE MIZER","url":"http://www.last.fm/music/MALICE+MIZER","picture":"http://userserve-ak.last.fm/serve/252/10808.jpg"} +{"number":"2","name":"Diary of Dreams","url":"http://www.last.fm/music/Diary+of+Dreams","picture":"http://userserve-ak.last.fm/serve/252/3052066.jpg"} +{"number":"3","name":"Carpathian Forest","url":"http://www.last.fm/music/Carpathian+Forest","picture":"http://userserve-ak.last.fm/serve/252/40222717.jpg"} +{"number":"4","name":"Moi dix Mois","url":"http://www.last.fm/music/Moi+dix+Mois","picture":"http://userserve-ak.last.fm/serve/252/54697835.png"} +{"number":"5","name":"Bella Morte","url":"http://www.last.fm/music/Bella+Morte","picture":"http://userserve-ak.last.fm/serve/252/14789013.jpg"} +{"number":"6","name":"Moonspell","url":"http://www.last.fm/music/Moonspell","picture":"http://userserve-ak.last.fm/serve/252/2181591.jpg"} +{"number":"7","name":"Marilyn Manson","url":"http://www.last.fm/music/Marilyn+Manson","picture":"http://userserve-ak.last.fm/serve/252/2558217.jpg"} +{"number":"8","name":"DIR EN GREY","url":"http://www.last.fm/music/DIR+EN+GREY","picture":"http://userserve-ak.last.fm/serve/252/46968835.png"} +{"number":"9","name":"Combichrist","url":"http://www.last.fm/music/Combichrist","picture":"http://userserve-ak.last.fm/serve/252/51273485.jpg"} +{"number":"10","name":"Grendel","url":"http://www.last.fm/music/Grendel","picture":"http://userserve-ak.last.fm/serve/252/5872875.jpg"} +{"number":"11","name":"Agonoize","url":"http://www.last.fm/music/Agonoize","picture":"http://userserve-ak.last.fm/serve/252/31693309.jpg"} +{"number":"12","name":"Behemoth","url":"http://www.last.fm/music/Behemoth","picture":"http://userserve-ak.last.fm/serve/252/54196161.jpg"} +{"number":"13","name":"Hocico","url":"http://www.last.fm/music/Hocico","picture":"http://userserve-ak.last.fm/serve/252/34892635.jpg"} +{"number":"15","name":"Dimmu Borgir","url":"http://www.last.fm/music/Dimmu+Borgir","picture":"http://userserve-ak.last.fm/serve/252/52216127.png"} +{"number":"16","name":"London After Midnight","url":"http://www.last.fm/music/London+After+Midnight","picture":"http://userserve-ak.last.fm/serve/252/5364091.jpg"} +{"number":"17","name":"Psyclon Nine","url":"http://www.last.fm/music/Psyclon+Nine","picture":"http://userserve-ak.last.fm/serve/252/35246025.jpg"} +{"number":"19","name":":wumpscut:","url":"http://www.last.fm/music/%3Awumpscut%3A","picture":"http://userserve-ak.last.fm/serve/252/541326.jpg"} +{"number":"20","name":"Limbonic Art","url":"http://www.last.fm/music/Limbonic+Art","picture":"http://userserve-ak.last.fm/serve/252/29314111.jpg"} +{"number":"21","name":"Artista sconosciuto","url":"http://www.last.fm/music/Artista+sconosciuto","picture":"http://userserve-ak.last.fm/serve/252/17969009.png"} +{"number":"22","name":"xotox","url":"http://www.last.fm/music/xotox","picture":"http://userserve-ak.last.fm/serve/252/5297893.jpg"} +{"number":"23","name":"The Kovenant","url":"http://www.last.fm/music/The+Kovenant","picture":"http://userserve-ak.last.fm/serve/252/8058051.jpg"} +{"number":"24","name":"Feindflug","url":"http://www.last.fm/music/Feindflug","picture":"http://userserve-ak.last.fm/serve/252/17291471.jpg"} +{"number":"25","name":"Cradle of Filth","url":"http://www.last.fm/music/Cradle+of+Filth","picture":"http://userserve-ak.last.fm/serve/252/30799583.jpg"} +{"number":"26","name":"Tamtrum","url":"http://www.last.fm/music/Tamtrum","picture":"http://userserve-ak.last.fm/serve/252/8193963.jpg"} +{"number":"27","name":"Laibach","url":"http://www.last.fm/music/Laibach","picture":"http://userserve-ak.last.fm/serve/252/615172.jpg"} +{"number":"28","name":"Gorgoroth","url":"http://www.last.fm/music/Gorgoroth","picture":"http://userserve-ak.last.fm/serve/252/35522943.jpg"} +{"number":"30","name":"And One","url":"http://www.last.fm/music/And+One","picture":"http://userserve-ak.last.fm/serve/252/50818861.jpg"} +{"number":"31","name":"Amduscia","url":"http://www.last.fm/music/Amduscia","picture":"http://userserve-ak.last.fm/serve/252/23951523.jpg"} +{"number":"32","name":"Covenant","url":"http://www.last.fm/music/Covenant","picture":"http://userserve-ak.last.fm/serve/252/59793013.jpg"} +{"number":"34","name":"Icon of Coil","url":"http://www.last.fm/music/Icon+of+Coil","picture":"http://userserve-ak.last.fm/serve/252/371747.jpg"} +{"number":"36","name":"KMFDM","url":"http://www.last.fm/music/KMFDM","picture":"http://userserve-ak.last.fm/serve/252/19970449.jpg"} +{"number":"37","name":"Funker Vogt","url":"http://www.last.fm/music/Funker+Vogt","picture":"http://userserve-ak.last.fm/serve/252/16436553.jpg"} +{"number":"38","name":"Dawn of Ashes","url":"http://www.last.fm/music/Dawn+of+Ashes","picture":"http://userserve-ak.last.fm/serve/252/28578343.jpg"} +{"number":"39","name":"Sopor Aeternus & The Ensemble of Shadows","url":"http://www.last.fm/music/Sopor%2BAeternus%2B%2526%2BThe%2BEnsemble%2Bof%2BShadows","picture":"http://userserve-ak.last.fm/serve/252/260221.jpg"} +{"number":"40","name":"The Faint","url":"http://www.last.fm/music/The+Faint","picture":"http://userserve-ak.last.fm/serve/252/1985.jpg"} +{"number":"41","name":"God Module","url":"http://www.last.fm/music/God+Module","picture":"http://userserve-ak.last.fm/serve/252/608782.jpg"} +{"number":"43","name":"Noisuf-X","url":"http://www.last.fm/music/Noisuf-X","picture":"http://userserve-ak.last.fm/serve/252/27456325.jpg"} +{"number":"44","name":"Das Ich","url":"http://www.last.fm/music/Das+Ich","picture":"http://userserve-ak.last.fm/serve/252/46246053.jpg"} +{"number":"45","name":"Mindless Self Indulgence","url":"http://www.last.fm/music/Mindless+Self+Indulgence","picture":"http://userserve-ak.last.fm/serve/252/360590.jpg"} +{"number":"46","name":"Pyotr Ilyich Tchaikovsky","url":"http://www.last.fm/music/Pyotr+Ilyich+Tchaikovsky","picture":"http://userserve-ak.last.fm/serve/252/44326361.png"} +{"number":"47","name":"Emperor","url":"http://www.last.fm/music/Emperor","picture":"http://userserve-ak.last.fm/serve/252/12767259.jpg"} +{"number":"48","name":"Anorexia Nervosa","url":"http://www.last.fm/music/Anorexia+Nervosa","picture":"http://userserve-ak.last.fm/serve/252/396508.jpg"} +{"number":"49","name":"Reaper","url":"http://www.last.fm/music/Reaper","picture":"http://userserve-ak.last.fm/serve/252/36414477.png"} +{"number":"50","name":"Gothminister","url":"http://www.last.fm/music/Gothminister","picture":"http://userserve-ak.last.fm/serve/252/23255695.jpg"} +{"number":"51","name":"Duran Duran","url":"http://www.last.fm/music/Duran+Duran","picture":"http://userserve-ak.last.fm/serve/252/155668.jpg"} +{"number":"52","name":"Morcheeba","url":"http://www.last.fm/music/Morcheeba","picture":"http://userserve-ak.last.fm/serve/252/46005111.png"} +{"number":"53","name":"Air","url":"http://www.last.fm/music/Air","picture":"http://userserve-ak.last.fm/serve/252/251119.jpg"} +{"number":"54","name":"Hooverphonic","url":"http://www.last.fm/music/Hooverphonic","picture":"http://userserve-ak.last.fm/serve/252/40553471.png"} +{"number":"55","name":"Kylie Minogue","url":"http://www.last.fm/music/Kylie+Minogue","picture":"http://userserve-ak.last.fm/serve/252/12740835.jpg"} +{"number":"56","name":"Daft Punk","url":"http://www.last.fm/music/Daft+Punk","picture":"http://userserve-ak.last.fm/serve/252/10923145.png"} +{"number":"57","name":"Thievery Corporation","url":"http://www.last.fm/music/Thievery+Corporation","picture":"http://userserve-ak.last.fm/serve/252/50633735.png"} +{"number":"58","name":"Goldfrapp","url":"http://www.last.fm/music/Goldfrapp","picture":"http://userserve-ak.last.fm/serve/252/36460899.png"} +{"number":"59","name":"New Order","url":"http://www.last.fm/music/New+Order","picture":"http://userserve-ak.last.fm/serve/252/6650979.jpg"} +{"number":"60","name":"Matt Bianco","url":"http://www.last.fm/music/Matt+Bianco","picture":"http://userserve-ak.last.fm/serve/252/369674.jpg"} +{"number":"61","name":"Talk Talk","url":"http://www.last.fm/music/Talk+Talk","picture":"http://userserve-ak.last.fm/serve/252/48047875.jpg"} +{"number":"62","name":"Prefab Sprout","url":"http://www.last.fm/music/Prefab+Sprout","picture":"http://userserve-ak.last.fm/serve/252/15600007.png"} +{"number":"63","name":"Enigma","url":"http://www.last.fm/music/Enigma","picture":"http://userserve-ak.last.fm/serve/252/36180069.png"} +{"number":"65","name":"Coldplay","url":"http://www.last.fm/music/Coldplay","picture":"http://userserve-ak.last.fm/serve/252/67770.jpg"} +{"number":"66","name":"Faithless","url":"http://www.last.fm/music/Faithless","picture":"http://userserve-ak.last.fm/serve/252/42558111.jpg"} +{"number":"67","name":"Madonna","url":"http://www.last.fm/music/Madonna","picture":"http://userserve-ak.last.fm/serve/252/340387.jpg"} +{"number":"68","name":"Icehouse","url":"http://www.last.fm/music/Icehouse","picture":"http://userserve-ak.last.fm/serve/252/189704.jpg"} +{"number":"69","name":"Sade","url":"http://www.last.fm/music/Sade","picture":"http://userserve-ak.last.fm/serve/252/61389233.png"} +{"number":"70","name":"Moby","url":"http://www.last.fm/music/Moby","picture":"http://userserve-ak.last.fm/serve/252/130045.jpg"} +{"number":"71","name":"Dido","url":"http://www.last.fm/music/Dido","picture":"http://userserve-ak.last.fm/serve/252/21564957.jpg"} +{"number":"72","name":"Depeche Mode","url":"http://www.last.fm/music/Depeche+Mode","picture":"http://userserve-ak.last.fm/serve/252/75022.jpg"} +{"number":"74","name":"Basia","url":"http://www.last.fm/music/Basia","picture":"http://userserve-ak.last.fm/serve/252/28540541.png"} +{"number":"75","name":"Camouflage","url":"http://www.last.fm/music/Camouflage","picture":"http://userserve-ak.last.fm/serve/252/4183502.jpg"} +{"number":"76","name":"Electronic","url":"http://www.last.fm/music/Electronic","picture":"http://userserve-ak.last.fm/serve/252/11464257.jpg"} +{"number":"77","name":"George Michael","url":"http://www.last.fm/music/George+Michael","picture":"http://userserve-ak.last.fm/serve/252/39849449.png"} +{"number":"78","name":"The Adventures","url":"http://www.last.fm/music/The+Adventures","picture":"http://userserve-ak.last.fm/serve/252/462539.jpg"} +{"number":"79","name":"Fiction Factory","url":"http://www.last.fm/music/Fiction+Factory","picture":"http://userserve-ak.last.fm/serve/252/365892.jpg"} +{"number":"80","name":"Groove Armada","url":"http://www.last.fm/music/Groove+Armada","picture":"http://userserve-ak.last.fm/serve/252/7025387.jpg"} +{"number":"81","name":"Portishead","url":"http://www.last.fm/music/Portishead","picture":"http://userserve-ak.last.fm/serve/252/11272935.jpg"} +{"number":"82","name":"Marc Almond","url":"http://www.last.fm/music/Marc+Almond","picture":"http://userserve-ak.last.fm/serve/252/50997309.png"} +{"number":"83","name":"Cock Robin","url":"http://www.last.fm/music/Cock+Robin","picture":"http://userserve-ak.last.fm/serve/252/287993.jpg"} +{"number":"84","name":"Cut Copy","url":"http://www.last.fm/music/Cut+Copy","picture":"http://userserve-ak.last.fm/serve/252/9539517.jpg"} +{"number":"85","name":"Spandau Ballet","url":"http://www.last.fm/music/Spandau+Ballet","picture":"http://userserve-ak.last.fm/serve/252/354540.jpg"} +{"number":"86","name":"Katie Melua","url":"http://www.last.fm/music/Katie+Melua","picture":"http://userserve-ak.last.fm/serve/252/38702721.png"} +{"number":"87","name":"Deacon Blue","url":"http://www.last.fm/music/Deacon+Blue","picture":"http://userserve-ak.last.fm/serve/252/439471.jpg"} +{"number":"88","name":"Gorillaz","url":"http://www.last.fm/music/Gorillaz","picture":"http://userserve-ak.last.fm/serve/252/45388159.png"} +{"number":"89","name":"Lady Gaga","url":"http://www.last.fm/music/Lady+Gaga","picture":"http://userserve-ak.last.fm/serve/252/47390093.png"} +{"number":"90","name":"Kosheen","url":"http://www.last.fm/music/Kosheen","picture":"http://userserve-ak.last.fm/serve/252/3309551.jpg"} +{"number":"91","name":"Nik Kershaw","url":"http://www.last.fm/music/Nik+Kershaw","picture":"http://userserve-ak.last.fm/serve/252/4204557.jpg"} +{"number":"92","name":"Vitamin Z","url":"http://www.last.fm/music/Vitamin+Z","picture":"http://userserve-ak.last.fm/serve/252/197506.jpg"} +{"number":"93","name":"Jean-Michel Jarre","url":"http://www.last.fm/music/Jean-Michel+Jarre","picture":"http://userserve-ak.last.fm/serve/252/242240.jpg"} +{"number":"94","name":"Ministry of Sound","url":"http://www.last.fm/music/Ministry+of+Sound","picture":"http://userserve-ak.last.fm/serve/252/13305931.jpg"} +{"number":"95","name":"Simply Red","url":"http://www.last.fm/music/Simply+Red","picture":"http://userserve-ak.last.fm/serve/252/50997673.png"} +{"number":"96","name":"Fleetwood Mac","url":"http://www.last.fm/music/Fleetwood+Mac","picture":"http://userserve-ak.last.fm/serve/252/337211.jpg"} +{"number":"97","name":"Duffy","url":"http://www.last.fm/music/Duffy","picture":"http://userserve-ak.last.fm/serve/252/55142075.png"} +{"number":"98","name":"Japan","url":"http://www.last.fm/music/Japan","picture":"http://userserve-ak.last.fm/serve/252/34822153.jpg"} +{"number":"99","name":"INXS","url":"http://www.last.fm/music/INXS","picture":"http://userserve-ak.last.fm/serve/252/26433237.jpg"} +{"number":"100","name":"ABC","url":"http://www.last.fm/music/ABC","picture":"http://userserve-ak.last.fm/serve/252/41596559.png"} +{"number":"101","name":"Pleq","url":"http://www.last.fm/music/Pleq","picture":"http://userserve-ak.last.fm/serve/252/58414481.jpg"} +{"number":"102","name":"Segue","url":"http://www.last.fm/music/Segue","picture":"http://userserve-ak.last.fm/serve/252/38568681.jpg"} +{"number":"103","name":"Max Richter","url":"http://www.last.fm/music/Max+Richter","picture":"http://userserve-ak.last.fm/serve/252/51974589.png"} +{"number":"104","name":"Celer","url":"http://www.last.fm/music/Celer","picture":"http://userserve-ak.last.fm/serve/252/43677713.jpg"} +{"number":"105","name":"Pjusk","url":"http://www.last.fm/music/Pjusk","picture":"http://userserve-ak.last.fm/serve/252/43518367.jpg"} +{"number":"106","name":"Pleq & Segue","url":"http://www.last.fm/music/Pleq%2B%2526%2BSegue","picture":"http://userserve-ak.last.fm/serve/252/58293455.jpg"} +{"number":"107","name":"Burial","url":"http://www.last.fm/music/Burial","picture":"http://userserve-ak.last.fm/serve/252/60689649.png"} +{"number":"108","name":"Magnitophono","url":"http://www.last.fm/music/Magnitophono","picture":"http://userserve-ak.last.fm/serve/252/24388525.jpg"} +{"number":"109","name":"andy graydon","url":"http://www.last.fm/music/andy+graydon","picture":"http://userserve-ak.last.fm/serve/252/26569717.jpg"} +{"number":"110","name":"ddekombinacja","url":"http://www.last.fm/music/ddekombinacja","picture":"http://userserve-ak.last.fm/serve/252/31731637.jpg"} +{"number":"111","name":"Pleq & Anna Rose Carter","url":"http://www.last.fm/music/+noredirect/Pleq%2B%2526%2BAnna%2BRose%2BCarter","picture":""} +{"number":"112","name":"Go Koyashiki","url":"http://www.last.fm/music/Go+Koyashiki","picture":"http://userserve-ak.last.fm/serve/252/58277441.jpg"} +{"number":"113","name":"Dustin O'Halloran","url":"http://www.last.fm/music/Dustin+O%27Halloran","picture":"http://userserve-ak.last.fm/serve/252/28429321.jpg"} +{"number":"114","name":"offthesky","url":"http://www.last.fm/music/offthesky","picture":"http://userserve-ak.last.fm/serve/252/35022765.png"} +{"number":"115","name":"Nebulo","url":"http://www.last.fm/music/Nebulo","picture":"http://userserve-ak.last.fm/serve/252/14070783.jpg"} +{"number":"116","name":"Fjordne","url":"http://www.last.fm/music/Fjordne","picture":"http://userserve-ak.last.fm/serve/252/43583903.jpg"} +{"number":"117","name":"Pleq & Chihiro","url":"http://www.last.fm/music/Pleq%2B%2526%2BChihiro","picture":"http://userserve-ak.last.fm/serve/252/24610863.jpg"} +{"number":"118","name":"Spyweirdos","url":"http://www.last.fm/music/Spyweirdos","picture":"http://userserve-ak.last.fm/serve/252/59489573.png"} +{"number":"119","name":"billy gomberg + offthesky","url":"http://www.last.fm/music/+noredirect/billy%2Bgomberg%2B%252B%2Boffthesky","picture":""} +{"number":"120","name":"Deru","url":"http://www.last.fm/music/Deru","picture":"http://userserve-ak.last.fm/serve/252/27329185.jpg"} +{"number":"121","name":"Lukid","url":"http://www.last.fm/music/Lukid","picture":"http://userserve-ak.last.fm/serve/252/47883977.jpg"} +{"number":"122","name":"Manekinekod","url":"http://www.last.fm/music/Manekinekod","picture":"http://userserve-ak.last.fm/serve/252/39374677.jpg"} +{"number":"123","name":"Danny Norbury","url":"http://www.last.fm/music/Danny+Norbury","picture":"http://userserve-ak.last.fm/serve/252/4964091.jpg"} +{"number":"124","name":"Shinkei & Mise En Scene","url":"http://www.last.fm/music/Shinkei%2B%2526%2BMise%2BEn%2BScene","picture":""} +{"number":"125","name":"aslope","url":"http://www.last.fm/music/aslope","picture":"http://userserve-ak.last.fm/serve/252/39251237.jpg"} +{"number":"126","name":"The Notwist","url":"http://www.last.fm/music/The+Notwist","picture":"http://userserve-ak.last.fm/serve/252/13533259.jpg"} +{"number":"127","name":"Off the Sky","url":"http://www.last.fm/music/Off+the+Sky","picture":"http://userserve-ak.last.fm/serve/252/421091.jpg"} +{"number":"128","name":"strom noir","url":"http://www.last.fm/music/strom+noir","picture":"http://userserve-ak.last.fm/serve/252/5519362.jpg"} +{"number":"129","name":"Aless","url":"http://www.last.fm/music/Aless","picture":"http://userserve-ak.last.fm/serve/252/44521979.jpg"} +{"number":"130","name":"Philippe Lamy","url":"http://www.last.fm/music/Philippe+Lamy","picture":"http://userserve-ak.last.fm/serve/252/34563725.jpg"} +{"number":"131","name":"Part Timer","url":"http://www.last.fm/music/Part+Timer","picture":"http://userserve-ak.last.fm/serve/252/26514979.jpg"} +{"number":"132","name":"Deaf Center","url":"http://www.last.fm/music/Deaf+Center","picture":"http://userserve-ak.last.fm/serve/252/179552.jpg"} +{"number":"133","name":"Lauki","url":"http://www.last.fm/music/Lauki","picture":""} +{"number":"134","name":"Big Brotherz","url":"http://www.last.fm/music/Big+Brotherz","picture":"http://userserve-ak.last.fm/serve/252/48444347.jpg"} +{"number":"135","name":"cokiyu","url":"http://www.last.fm/music/cokiyu","picture":"http://userserve-ak.last.fm/serve/252/41954375.jpg"} +{"number":"136","name":"Verbose","url":"http://www.last.fm/music/Verbose","picture":"http://userserve-ak.last.fm/serve/252/3406085.jpg"} +{"number":"137","name":"Worm Is Green","url":"http://www.last.fm/music/Worm+Is+Green","picture":"http://userserve-ak.last.fm/serve/252/613837.jpg"} +{"number":"138","name":"Library Tapes","url":"http://www.last.fm/music/Library+Tapes","picture":"http://userserve-ak.last.fm/serve/252/2247728.jpg"} +{"number":"139","name":"Richard Skelton","url":"http://www.last.fm/music/Richard+Skelton","picture":"http://userserve-ak.last.fm/serve/252/43254939.png"} +{"number":"140","name":"Matryoshka","url":"http://www.last.fm/music/Matryoshka","picture":"http://userserve-ak.last.fm/serve/252/42836783.jpg"} +{"number":"141","name":"Aidan Baker","url":"http://www.last.fm/music/Aidan+Baker","picture":"http://userserve-ak.last.fm/serve/252/60671123.jpg"} +{"number":"142","name":"Ylid","url":"http://www.last.fm/music/Ylid","picture":"http://userserve-ak.last.fm/serve/252/381247.jpg"} +{"number":"143","name":"Daisuke Miyatani","url":"http://www.last.fm/music/Daisuke+Miyatani","picture":"http://userserve-ak.last.fm/serve/252/28588821.jpg"} +{"number":"144","name":"Dakota Suite","url":"http://www.last.fm/music/Dakota+Suite","picture":"http://userserve-ak.last.fm/serve/252/34844669.jpg"} +{"number":"145","name":"Ryonkt","url":"http://www.last.fm/music/Ryonkt","picture":"http://userserve-ak.last.fm/serve/252/3400979.jpg"} +{"number":"146","name":"Hird","url":"http://www.last.fm/music/Hird","picture":"http://userserve-ak.last.fm/serve/252/317707.jpg"} +{"number":"147","name":"Svarte Greiner","url":"http://www.last.fm/music/Svarte+Greiner","picture":"http://userserve-ak.last.fm/serve/252/29675371.jpg"} +{"number":"148","name":"The Boats","url":"http://www.last.fm/music/The+Boats","picture":"http://userserve-ak.last.fm/serve/252/12285119.jpg"} +{"number":"149","name":"The Sound Of Lucrecia","url":"http://www.last.fm/music/The+Sound+Of+Lucrecia","picture":"http://userserve-ak.last.fm/serve/252/37565391.jpg"} +{"number":"150","name":"Porzellan","url":"http://www.last.fm/music/Porzellan","picture":"http://userserve-ak.last.fm/serve/252/36392371.jpg"} +{"number":"151","name":"Deep Forest","url":"http://www.last.fm/music/Deep+Forest","picture":"http://userserve-ak.last.fm/serve/252/347830.jpg"} +{"number":"152","name":"Porcupine Tree","url":"http://www.last.fm/music/Porcupine+Tree","picture":"http://userserve-ak.last.fm/serve/252/13065279.jpg"} +{"number":"153","name":"De/Vision","url":"http://www.last.fm/music/De%252FVision","picture":"http://userserve-ak.last.fm/serve/252/14554013.jpg"} +{"number":"154","name":"Radiohead","url":"http://www.last.fm/music/Radiohead","picture":"http://userserve-ak.last.fm/serve/252/8461967.jpg"} +{"number":"155","name":"Robbie Williams","url":"http://www.last.fm/music/Robbie+Williams","picture":"http://userserve-ak.last.fm/serve/252/35492555.png"} +{"number":"156","name":"VAST","url":"http://www.last.fm/music/VAST","picture":"http://userserve-ak.last.fm/serve/252/3516839.jpg"} +{"number":"157","name":"Michael Jackson","url":"http://www.last.fm/music/Michael+Jackson","picture":"http://userserve-ak.last.fm/serve/252/30501527.png"} +{"number":"159","name":"The Cure","url":"http://www.last.fm/music/The+Cure","picture":"http://userserve-ak.last.fm/serve/252/9151219.jpg"} +{"number":"161","name":"Enrique Iglesias","url":"http://www.last.fm/music/Enrique+Iglesias","picture":"http://userserve-ak.last.fm/serve/252/46065447.png"} +{"number":"162","name":"God Is an Astronaut","url":"http://www.last.fm/music/God+Is+an+Astronaut","picture":"http://userserve-ak.last.fm/serve/252/47057207.jpg"} +{"number":"163","name":"Pink Floyd","url":"http://www.last.fm/music/Pink+Floyd","picture":"http://userserve-ak.last.fm/serve/252/39219129.jpg"} +{"number":"165","name":"Planet Funk","url":"http://www.last.fm/music/Planet+Funk","picture":"http://userserve-ak.last.fm/serve/252/30909637.jpg"} +{"number":"166","name":"The Cranberries","url":"http://www.last.fm/music/The+Cranberries","picture":"http://userserve-ak.last.fm/serve/252/8407.jpg"} +{"number":"167","name":"Scissor Sisters","url":"http://www.last.fm/music/Scissor+Sisters","picture":"http://userserve-ak.last.fm/serve/252/52406253.png"} +{"number":"170","name":"Mew","url":"http://www.last.fm/music/Mew","picture":"http://userserve-ak.last.fm/serve/252/42247291.jpg"} +{"number":"171","name":"Stereophonics","url":"http://www.last.fm/music/Stereophonics","picture":"http://userserve-ak.last.fm/serve/252/248095.jpg"} +{"number":"172","name":"Garbage","url":"http://www.last.fm/music/Garbage","picture":"http://userserve-ak.last.fm/serve/252/143186.jpg"} +{"number":"173","name":"Placebo","url":"http://www.last.fm/music/Placebo","picture":"http://userserve-ak.last.fm/serve/252/300983.jpg"} +{"number":"174","name":"Infected Mushroom","url":"http://www.last.fm/music/Infected+Mushroom","picture":"http://userserve-ak.last.fm/serve/252/6628687.jpg"} +{"number":"175","name":"Delerium","url":"http://www.last.fm/music/Delerium","picture":"http://userserve-ak.last.fm/serve/252/7342823.jpg"} +{"number":"176","name":"Scorpions","url":"http://www.last.fm/music/Scorpions","picture":"http://userserve-ak.last.fm/serve/252/3616479.png"} +{"number":"177","name":"Rock Star Supernova","url":"http://www.last.fm/music/Rock+Star+Supernova","picture":"http://userserve-ak.last.fm/serve/252/139182.jpg"} +{"number":"178","name":"Dolores O'Riordan","url":"http://www.last.fm/music/Dolores+O%27Riordan","picture":"http://userserve-ak.last.fm/serve/252/5515307.jpg"} +{"number":"179","name":"Tommy Lee","url":"http://www.last.fm/music/Tommy+Lee","picture":"http://userserve-ak.last.fm/serve/252/17693251.jpg"} +{"number":"180","name":"Roxette","url":"http://www.last.fm/music/Roxette","picture":"http://userserve-ak.last.fm/serve/252/2298997.jpg"} +{"number":"181","name":"Paradise Lost","url":"http://www.last.fm/music/Paradise+Lost","picture":"http://userserve-ak.last.fm/serve/252/35325935.jpg"} +{"number":"182","name":"Keane","url":"http://www.last.fm/music/Keane","picture":"http://userserve-ak.last.fm/serve/252/8891209.jpg"} +{"number":"183","name":"Jamiroquai","url":"http://www.last.fm/music/Jamiroquai","picture":"http://userserve-ak.last.fm/serve/252/17958209.jpg"} +{"number":"184","name":"James Blunt","url":"http://www.last.fm/music/James+Blunt","picture":"http://userserve-ak.last.fm/serve/252/2862243.jpg"} +{"number":"185","name":"Reamonn","url":"http://www.last.fm/music/Reamonn","picture":"http://userserve-ak.last.fm/serve/252/45094837.png"} +{"number":"186","name":"Blackfield","url":"http://www.last.fm/music/Blackfield","picture":"http://userserve-ak.last.fm/serve/252/55835355.png"} +{"number":"187","name":"a-ha","url":"http://www.last.fm/music/a-ha","picture":"http://userserve-ak.last.fm/serve/252/44717589.png"} +{"number":"188","name":"Opeth","url":"http://www.last.fm/music/Opeth","picture":"http://userserve-ak.last.fm/serve/252/45325391.png"} +{"number":"189","name":"Poets of the Fall","url":"http://www.last.fm/music/Poets+of+the+Fall","picture":"http://userserve-ak.last.fm/serve/252/60552447.jpg"} +{"number":"190","name":"Muse","url":"http://www.last.fm/music/Muse","picture":"http://userserve-ak.last.fm/serve/252/416514.jpg"} +{"number":"191","name":"OneRepublic","url":"http://www.last.fm/music/OneRepublic","picture":"http://userserve-ak.last.fm/serve/252/38037651.png"} +{"number":"192","name":"Elton John","url":"http://www.last.fm/music/Elton+John","picture":"http://userserve-ak.last.fm/serve/252/17079375.jpg"} +{"number":"193","name":"Tears for Fears","url":"http://www.last.fm/music/Tears+for+Fears","picture":"http://userserve-ak.last.fm/serve/252/42813709.png"} +{"number":"194","name":"The Tiger Lillies","url":"http://www.last.fm/music/The+Tiger+Lillies","picture":"http://userserve-ak.last.fm/serve/252/5921990.png"} +{"number":"195","name":"Bright Eyes","url":"http://www.last.fm/music/Bright+Eyes","picture":"http://userserve-ak.last.fm/serve/252/26273.jpg"} +{"number":"196","name":"Beirut","url":"http://www.last.fm/music/Beirut","picture":"http://userserve-ak.last.fm/serve/252/51274303.png"} +{"number":"197","name":"Tom Waits","url":"http://www.last.fm/music/Tom+Waits","picture":"http://userserve-ak.last.fm/serve/252/207221.jpg"} +{"number":"198","name":"System of a Down","url":"http://www.last.fm/music/System+of+a+Down","picture":"http://userserve-ak.last.fm/serve/252/363017.jpg"} +{"number":"199","name":"Arcade Fire","url":"http://www.last.fm/music/Arcade+Fire","picture":"http://userserve-ak.last.fm/serve/252/18697365.jpg"} +{"number":"200","name":"Gogol Bordello","url":"http://www.last.fm/music/Gogol+Bordello","picture":"http://userserve-ak.last.fm/serve/252/35336869.jpg"} +{"number":"202","name":"CAKE","url":"http://www.last.fm/music/CAKE","picture":"http://userserve-ak.last.fm/serve/252/12825895.gif"} +{"number":"203","name":"Blur","url":"http://www.last.fm/music/Blur","picture":"http://userserve-ak.last.fm/serve/252/269394.jpg"} +{"number":"204","name":"Damien Rice","url":"http://www.last.fm/music/Damien+Rice","picture":"http://userserve-ak.last.fm/serve/252/292534.png"} +{"number":"205","name":"Andrew Bird","url":"http://www.last.fm/music/Andrew+Bird","picture":"http://userserve-ak.last.fm/serve/252/951.jpg"} +{"number":"206","name":"Beck","url":"http://www.last.fm/music/Beck","picture":"http://userserve-ak.last.fm/serve/252/7497941.jpg"} +{"number":"207","name":"Arctic Monkeys","url":"http://www.last.fm/music/Arctic+Monkeys","picture":"http://userserve-ak.last.fm/serve/252/33550647.jpg"} +{"number":"208","name":"Babyshambles","url":"http://www.last.fm/music/Babyshambles","picture":"http://userserve-ak.last.fm/serve/252/371350.jpg"} +{"number":"209","name":"My Chemical Romance","url":"http://www.last.fm/music/My+Chemical+Romance","picture":"http://userserve-ak.last.fm/serve/252/411086.jpg"} +{"number":"210","name":"Bloc Party","url":"http://www.last.fm/music/Bloc+Party","picture":"http://userserve-ak.last.fm/serve/252/780386.jpg"} +{"number":"211","name":"The Decemberists","url":"http://www.last.fm/music/The+Decemberists","picture":"http://userserve-ak.last.fm/serve/252/886900.jpg"} +{"number":"212","name":"Bob Dylan","url":"http://www.last.fm/music/Bob+Dylan","picture":"http://userserve-ak.last.fm/serve/252/13979651.jpg"} +{"number":"213","name":"Jacques Brel","url":"http://www.last.fm/music/Jacques+Brel","picture":"http://userserve-ak.last.fm/serve/252/23711613.jpg"} +{"number":"214","name":"DeVotchKa","url":"http://www.last.fm/music/DeVotchKa","picture":"http://userserve-ak.last.fm/serve/252/200510.jpg"} +{"number":"215","name":"Jeff Buckley","url":"http://www.last.fm/music/Jeff+Buckley","picture":"http://userserve-ak.last.fm/serve/252/49531.jpg"} +{"number":"216","name":"Sparklehorse","url":"http://www.last.fm/music/Sparklehorse","picture":"http://userserve-ak.last.fm/serve/252/43593291.png"} +{"number":"217","name":"Death Cab for Cutie","url":"http://www.last.fm/music/Death+Cab+for+Cutie","picture":"http://userserve-ak.last.fm/serve/252/60010487.jpg"} +{"number":"218","name":"Man Man","url":"http://www.last.fm/music/Man+Man","picture":"http://userserve-ak.last.fm/serve/252/5366105.jpg"} +{"number":"219","name":"M. Ward","url":"http://www.last.fm/music/M.+Ward","picture":"http://userserve-ak.last.fm/serve/252/27105637.jpg"} +{"number":"220","name":"Red Hot Chili Peppers","url":"http://www.last.fm/music/Red+Hot+Chili+Peppers","picture":"http://userserve-ak.last.fm/serve/252/155038.jpg"} +{"number":"221","name":"Travis","url":"http://www.last.fm/music/Travis","picture":"http://userserve-ak.last.fm/serve/252/137315.jpg"} +{"number":"222","name":"Modest Mouse","url":"http://www.last.fm/music/Modest+Mouse","picture":"http://userserve-ak.last.fm/serve/252/886281.jpg"} +{"number":"223","name":"Band of Horses","url":"http://www.last.fm/music/Band+of+Horses","picture":"http://userserve-ak.last.fm/serve/252/299960.jpg"} +{"number":"224","name":"Yeong-wook Jo","url":"http://www.last.fm/music/Yeong-wook+Jo","picture":"http://userserve-ak.last.fm/serve/252/33177663.jpg"} +{"number":"225","name":"Interpol","url":"http://www.last.fm/music/Interpol","picture":"http://userserve-ak.last.fm/serve/252/50656.jpg"} +{"number":"226","name":"Queens of the Stone Age","url":"http://www.last.fm/music/Queens+of+the+Stone+Age","picture":"http://userserve-ak.last.fm/serve/252/270457.jpg"} +{"number":"227","name":"The Beatles","url":"http://www.last.fm/music/The+Beatles","picture":"http://userserve-ak.last.fm/serve/252/2588646.jpg"} +{"number":"228","name":"Kings of Leon","url":"http://www.last.fm/music/Kings+of+Leon","picture":"http://userserve-ak.last.fm/serve/252/2739814.jpg"} +{"number":"229","name":"The Killers","url":"http://www.last.fm/music/The+Killers","picture":"http://userserve-ak.last.fm/serve/252/250099.jpg"} +{"number":"230","name":"Green Day","url":"http://www.last.fm/music/Green+Day","picture":"http://userserve-ak.last.fm/serve/252/15291249.jpg"} +{"number":"231","name":"Devendra Banhart","url":"http://www.last.fm/music/Devendra+Banhart","picture":"http://userserve-ak.last.fm/serve/252/22080795.jpg"} +{"number":"232","name":"Sunset Rubdown","url":"http://www.last.fm/music/Sunset+Rubdown","picture":"http://userserve-ak.last.fm/serve/252/41876349.png"} +{"number":"233","name":"Nine Inch Nails","url":"http://www.last.fm/music/Nine+Inch+Nails","picture":"http://userserve-ak.last.fm/serve/252/29014025.jpg"} +{"number":"234","name":"Nirvana","url":"http://www.last.fm/music/Nirvana","picture":"http://userserve-ak.last.fm/serve/252/380269.jpg"} +{"number":"235","name":"Eels","url":"http://www.last.fm/music/Eels","picture":"http://userserve-ak.last.fm/serve/252/33367415.jpg"} +{"number":"236","name":"Clap Your Hands Say Yeah","url":"http://www.last.fm/music/Clap+Your+Hands+Say+Yeah","picture":"http://userserve-ak.last.fm/serve/252/765229.jpg"} +{"number":"237","name":"Thom Yorke","url":"http://www.last.fm/music/Thom+Yorke","picture":"http://userserve-ak.last.fm/serve/252/29289641.jpg"} +{"number":"238","name":"Massive Attack","url":"http://www.last.fm/music/Massive+Attack","picture":"http://userserve-ak.last.fm/serve/252/41438057.jpg"} +{"number":"239","name":"Keyshia Cole","url":"http://www.last.fm/music/Keyshia+Cole","picture":"http://userserve-ak.last.fm/serve/252/61549437.png"} +{"number":"240","name":"Racionais MC's","url":"http://www.last.fm/music/Racionais+MC%27s","picture":"http://userserve-ak.last.fm/serve/252/3000654.jpg"} +{"number":"241","name":"Toni Braxton","url":"http://www.last.fm/music/Toni+Braxton","picture":"http://userserve-ak.last.fm/serve/252/46200685.png"} +{"number":"242","name":"Elo da Corrente","url":"http://www.last.fm/music/Elo+da+Corrente","picture":"http://userserve-ak.last.fm/serve/252/54216039.jpg"} +{"number":"243","name":"Teta Lando","url":"http://www.last.fm/music/Teta+Lando","picture":"http://userserve-ak.last.fm/serve/252/57587349.jpg"} +{"number":"244","name":"Flagelo Urbano","url":"http://www.last.fm/music/Flagelo+Urbano","picture":"http://userserve-ak.last.fm/serve/252/57586835.jpg"} +{"number":"245","name":"Bob Marley & The Wailers","url":"http://www.last.fm/music/Bob%2BMarley%2B%2526%2BThe%2BWailers","picture":"http://userserve-ak.last.fm/serve/252/208159.jpg"} +{"number":"246","name":"Adriana Calcanhotto","url":"http://www.last.fm/music/Adriana+Calcanhotto","picture":"http://userserve-ak.last.fm/serve/252/31094941.jpg"} +{"number":"247","name":"Maxwell","url":"http://www.last.fm/music/Maxwell","picture":"http://userserve-ak.last.fm/serve/252/47848615.png"} +{"number":"248","name":"Pharoahe Monch","url":"http://www.last.fm/music/Pharoahe+Monch","picture":"http://userserve-ak.last.fm/serve/252/54010599.jpg"} +{"number":"249","name":"Common","url":"http://www.last.fm/music/Common","picture":"http://userserve-ak.last.fm/serve/252/10623523.jpg"} +{"number":"250","name":"Bob Marley","url":"http://www.last.fm/music/Bob+Marley","picture":"http://userserve-ak.last.fm/serve/252/77849.jpg"} +{"number":"251","name":"Whitney Houston","url":"http://www.last.fm/music/Whitney+Houston","picture":"http://userserve-ak.last.fm/serve/252/58879935.png"} +{"number":"252","name":"Tamia","url":"http://www.last.fm/music/Tamia","picture":"http://userserve-ak.last.fm/serve/252/26185453.png"} +{"number":"253","name":"Bryan Adams","url":"http://www.last.fm/music/Bryan+Adams","picture":"http://userserve-ak.last.fm/serve/252/40597201.png"} +{"number":"254","name":"Lokua Kanza","url":"http://www.last.fm/music/Lokua+Kanza","picture":"http://userserve-ak.last.fm/serve/252/2908571.jpg"} +{"number":"255","name":"Nach","url":"http://www.last.fm/music/Nach","picture":"http://userserve-ak.last.fm/serve/252/2510461.jpg"} +{"number":"256","name":"Kid Abelha","url":"http://www.last.fm/music/Kid+Abelha","picture":"http://userserve-ak.last.fm/serve/252/61705261.jpg"} +{"number":"257","name":"Mariah Carey","url":"http://www.last.fm/music/Mariah+Carey","picture":"http://userserve-ak.last.fm/serve/252/4230813.jpg"} +{"number":"258","name":"Tina Turner","url":"http://www.last.fm/music/Tina+Turner","picture":"http://userserve-ak.last.fm/serve/252/260723.jpg"} +{"number":"259","name":"9th Wonder","url":"http://www.last.fm/music/9th+Wonder","picture":"http://userserve-ak.last.fm/serve/252/5915045.jpg"} +{"number":"260","name":"Floetry","url":"http://www.last.fm/music/Floetry","picture":"http://userserve-ak.last.fm/serve/252/7196965.jpg"} +{"number":"261","name":"Marisa Monte","url":"http://www.last.fm/music/Marisa+Monte","picture":"http://userserve-ak.last.fm/serve/252/20654981.jpg"} +{"number":"262","name":"Mary J. Blige","url":"http://www.last.fm/music/Mary+J.+Blige","picture":"http://userserve-ak.last.fm/serve/252/60234673.png"} +{"number":"263","name":"Jill Scott","url":"http://www.last.fm/music/Jill+Scott","picture":"http://userserve-ak.last.fm/serve/252/61389167.png"} +{"number":"264","name":"Sara Tavares","url":"http://www.last.fm/music/Sara+Tavares","picture":"http://userserve-ak.last.fm/serve/252/44745251.png"} +{"number":"266","name":"Erykah Badu","url":"http://www.last.fm/music/Erykah+Badu","picture":"http://userserve-ak.last.fm/serve/252/44415509.jpg"} +{"number":"267","name":"Nas","url":"http://www.last.fm/music/Nas","picture":"http://userserve-ak.last.fm/serve/252/46358275.png"} +{"number":"268","name":"Richard Marx","url":"http://www.last.fm/music/Richard+Marx","picture":"http://userserve-ak.last.fm/serve/252/7905405.jpg"} +{"number":"269","name":"Dru Hill","url":"http://www.last.fm/music/Dru+Hill","picture":"http://userserve-ak.last.fm/serve/252/52209529.png"} +{"number":"270","name":"Maria Bethânia","url":"http://www.last.fm/music/Maria+Beth%C3%A2nia","picture":"http://userserve-ak.last.fm/serve/252/15719805.jpg"} +{"number":"271","name":"Mos Def","url":"http://www.last.fm/music/Mos+Def","picture":"http://userserve-ak.last.fm/serve/252/28041091.jpg"} +{"number":"272","name":"Faith Evans","url":"http://www.last.fm/music/Faith+Evans","picture":"http://userserve-ak.last.fm/serve/252/10418795.jpg"} +{"number":"273","name":"Avant","url":"http://www.last.fm/music/Avant","picture":"http://userserve-ak.last.fm/serve/252/135431.jpg"} +{"number":"274","name":"Bob da Rage Sense","url":"http://www.last.fm/music/Bob+da+Rage+Sense","picture":"http://userserve-ak.last.fm/serve/252/4355.jpg"} +{"number":"275","name":"Angie Stone","url":"http://www.last.fm/music/Angie+Stone","picture":"http://userserve-ak.last.fm/serve/252/5417218.jpg"} +{"number":"276","name":"Gang Starr","url":"http://www.last.fm/music/Gang+Starr","picture":"http://userserve-ak.last.fm/serve/252/43178391.jpg"} +{"number":"277","name":"Papa Wemba","url":"http://www.last.fm/music/Papa+Wemba","picture":"http://userserve-ak.last.fm/serve/252/307662.jpg"} +{"number":"278","name":"2Pac","url":"http://www.last.fm/music/2Pac","picture":"http://userserve-ak.last.fm/serve/252/17814269.jpg"} +{"number":"279","name":"Brandy","url":"http://www.last.fm/music/Brandy","picture":"http://userserve-ak.last.fm/serve/252/61865299.png"} +{"number":"280","name":"Sam the Kid","url":"http://www.last.fm/music/Sam+the+Kid","picture":"http://userserve-ak.last.fm/serve/252/35581.jpg"} +{"number":"281","name":"Dealema","url":"http://www.last.fm/music/Dealema","picture":"http://userserve-ak.last.fm/serve/252/317653.jpg"} +{"number":"282","name":"Boss AC","url":"http://www.last.fm/music/Boss+AC","picture":"http://userserve-ak.last.fm/serve/252/46755589.jpg"} +{"number":"283","name":"Ana Carolina","url":"http://www.last.fm/music/Ana+Carolina","picture":"http://userserve-ak.last.fm/serve/252/48498003.jpg"} +{"number":"284","name":"Dilated Peoples","url":"http://www.last.fm/music/Dilated+Peoples","picture":"http://userserve-ak.last.fm/serve/252/47091215.png"} +{"number":"285","name":"Janet Jackson","url":"http://www.last.fm/music/Janet+Jackson","picture":"http://userserve-ak.last.fm/serve/252/4245358.jpg"} +{"number":"286","name":"Peter Tosh","url":"http://www.last.fm/music/Peter+Tosh","picture":"http://userserve-ak.last.fm/serve/252/851498.jpg"} +{"number":"287","name":"Monica","url":"http://www.last.fm/music/Monica","picture":"http://userserve-ak.last.fm/serve/252/43749117.png"} +{"number":"288","name":"Rihanna","url":"http://www.last.fm/music/Rihanna","picture":"http://userserve-ak.last.fm/serve/252/53023109.png"} +{"number":"289","name":"Britney Spears","url":"http://www.last.fm/music/Britney+Spears","picture":"http://userserve-ak.last.fm/serve/252/60126439.png"} +{"number":"290","name":"Jordin Sparks","url":"http://www.last.fm/music/Jordin+Sparks","picture":"http://userserve-ak.last.fm/serve/252/3143050.png"} +{"number":"291","name":"Kelly Clarkson","url":"http://www.last.fm/music/Kelly+Clarkson","picture":"http://userserve-ak.last.fm/serve/252/28236733.png"} +{"number":"292","name":"Christina Aguilera","url":"http://www.last.fm/music/Christina+Aguilera","picture":"http://userserve-ak.last.fm/serve/252/47363849.png"} +{"number":"293","name":"Ashlee Simpson","url":"http://www.last.fm/music/Ashlee+Simpson","picture":"http://userserve-ak.last.fm/serve/252/33543587.png"} +{"number":"294","name":"Leona Lewis","url":"http://www.last.fm/music/Leona+Lewis","picture":"http://userserve-ak.last.fm/serve/252/45104681.png"} +{"number":"296","name":"Sugababes","url":"http://www.last.fm/music/Sugababes","picture":"http://userserve-ak.last.fm/serve/252/12353825.jpg"} +{"number":"297","name":"David Cook","url":"http://www.last.fm/music/David+Cook","picture":"http://userserve-ak.last.fm/serve/252/61825565.png"} +{"number":"298","name":"Lily Allen","url":"http://www.last.fm/music/Lily+Allen","picture":"http://userserve-ak.last.fm/serve/252/21181805.png"} +{"number":"299","name":"Jennifer Lopez","url":"http://www.last.fm/music/Jennifer+Lopez","picture":"http://userserve-ak.last.fm/serve/252/49596953.png"} +{"number":"300","name":"Katy Perry","url":"http://www.last.fm/music/Katy+Perry","picture":"http://userserve-ak.last.fm/serve/252/42128121.png"} +{"number":"301","name":"Alicia Keys","url":"http://www.last.fm/music/Alicia+Keys","picture":"http://userserve-ak.last.fm/serve/252/40295231.jpg"} +{"number":"302","name":"P!nk","url":"http://www.last.fm/music/P%21nk","picture":"http://userserve-ak.last.fm/serve/252/56011579.png"} +{"number":"303","name":"Panic at the Disco","url":"http://www.last.fm/music/+noredirect/Panic+at+the+Disco","picture":"http://userserve-ak.last.fm/serve/252/34502133.png"} +{"number":"304","name":"David Archuleta","url":"http://www.last.fm/music/David+Archuleta","picture":"http://userserve-ak.last.fm/serve/252/57042871.png"} +{"number":"305","name":"Katharine McPhee","url":"http://www.last.fm/music/Katharine+McPhee","picture":"http://userserve-ak.last.fm/serve/252/55177859.png"} +{"number":"306","name":"Black Eyed Peas","url":"http://www.last.fm/music/Black+Eyed+Peas","picture":"http://userserve-ak.last.fm/serve/252/53698409.png"} +{"number":"307","name":"Kate Voegele","url":"http://www.last.fm/music/Kate+Voegele","picture":"http://userserve-ak.last.fm/serve/252/60041035.png"} +{"number":"308","name":"Kat DeLuna","url":"http://www.last.fm/music/Kat+DeLuna","picture":"http://userserve-ak.last.fm/serve/252/50935719.png"} +{"number":"309","name":"Danity Kane","url":"http://www.last.fm/music/Danity+Kane","picture":"http://userserve-ak.last.fm/serve/252/4707257.png"} +{"number":"310","name":"Nelly Furtado","url":"http://www.last.fm/music/Nelly+Furtado","picture":"http://userserve-ak.last.fm/serve/252/327744.jpg"} +{"number":"311","name":"Natasha Bedingfield","url":"http://www.last.fm/music/Natasha+Bedingfield","picture":"http://userserve-ak.last.fm/serve/252/49228915.png"} +{"number":"312","name":"Cassie","url":"http://www.last.fm/music/Cassie","picture":"http://userserve-ak.last.fm/serve/252/55559609.png"} +{"number":"313","name":"Allison Iraheta","url":"http://www.last.fm/music/Allison+Iraheta","picture":"http://userserve-ak.last.fm/serve/252/58025983.png"} +{"number":"314","name":"Ciara","url":"http://www.last.fm/music/Ciara","picture":"http://userserve-ak.last.fm/serve/252/41425039.png"} +{"number":"315","name":"Matt Pokora","url":"http://www.last.fm/music/+noredirect/Matt+Pokora","picture":"http://userserve-ak.last.fm/serve/252/5067334.jpg"} +{"number":"316","name":"Alanis Morissette","url":"http://www.last.fm/music/Alanis+Morissette","picture":"http://userserve-ak.last.fm/serve/252/6797661.jpg"} +{"number":"317","name":"Keri Hilson","url":"http://www.last.fm/music/Keri+Hilson","picture":"http://userserve-ak.last.fm/serve/252/61266477.png"} +{"number":"318","name":"Hilary Duff","url":"http://www.last.fm/music/Hilary+Duff","picture":"http://userserve-ak.last.fm/serve/252/8182513.jpg"} +{"number":"319","name":"Haley James Scott","url":"http://www.last.fm/music/Haley+James+Scott","picture":"http://userserve-ak.last.fm/serve/252/422138.jpg"} +{"number":"320","name":"Fergie","url":"http://www.last.fm/music/Fergie","picture":"http://userserve-ak.last.fm/serve/252/39646735.png"} +{"number":"321","name":"Jonas Brothers","url":"http://www.last.fm/music/Jonas+Brothers","picture":"http://userserve-ak.last.fm/serve/252/48619535.png"} +{"number":"322","name":"Ashanti","url":"http://www.last.fm/music/Ashanti","picture":"http://userserve-ak.last.fm/serve/252/59555091.png"} +{"number":"323","name":"The Saturdays","url":"http://www.last.fm/music/The+Saturdays","picture":"http://userserve-ak.last.fm/serve/252/28722161.png"} +{"number":"324","name":"Cobra Starship","url":"http://www.last.fm/music/Cobra+Starship","picture":"http://userserve-ak.last.fm/serve/252/47227651.jpg"} +{"number":"325","name":"Ashley Tisdale","url":"http://www.last.fm/music/Ashley+Tisdale","picture":"http://userserve-ak.last.fm/serve/252/62194153.png"} +{"number":"326","name":"JoJo","url":"http://www.last.fm/music/JoJo","picture":"http://userserve-ak.last.fm/serve/252/58611683.png"} +{"number":"327","name":"Chris Brown","url":"http://www.last.fm/music/Chris+Brown","picture":"http://userserve-ak.last.fm/serve/252/60726327.png"} +{"number":"328","name":"David Guetta","url":"http://www.last.fm/music/David+Guetta","picture":"http://userserve-ak.last.fm/serve/252/43155097.png"} +{"number":"329","name":"Justin Timberlake","url":"http://www.last.fm/music/Justin+Timberlake","picture":"http://userserve-ak.last.fm/serve/252/10351161.png"} +{"number":"330","name":"T.I.","url":"http://www.last.fm/music/T.I.","picture":"http://userserve-ak.last.fm/serve/252/402488.jpg"} +{"number":"331","name":"Kanye West","url":"http://www.last.fm/music/Kanye+West","picture":"http://userserve-ak.last.fm/serve/252/8942513.jpg"} +{"number":"332","name":"Kelly Rowland","url":"http://www.last.fm/music/Kelly+Rowland","picture":"http://userserve-ak.last.fm/serve/252/49377641.png"} +{"number":"333","name":"Avril Lavigne","url":"http://www.last.fm/music/Avril+Lavigne","picture":"http://userserve-ak.last.fm/serve/252/59708309.png"} +{"number":"334","name":"No Angels","url":"http://www.last.fm/music/No+Angels","picture":"http://userserve-ak.last.fm/serve/252/61126961.png"} +{"number":"335","name":"Sandy","url":"http://www.last.fm/music/Sandy","picture":"http://userserve-ak.last.fm/serve/252/61835829.png"} +{"number":"336","name":"Vanessa Petruo","url":"http://www.last.fm/music/Vanessa+Petruo","picture":"http://userserve-ak.last.fm/serve/252/57670865.png"} +{"number":"337","name":"Alexander","url":"http://www.last.fm/music/Alexander","picture":"http://userserve-ak.last.fm/serve/252/56191985.jpg"} +{"number":"338","name":"Queensberry","url":"http://www.last.fm/music/Queensberry","picture":"http://userserve-ak.last.fm/serve/252/49220935.png"} +{"number":"339","name":"Fady Maalouf","url":"http://www.last.fm/music/Fady+Maalouf","picture":"http://userserve-ak.last.fm/serve/252/10946343.jpg"} +{"number":"340","name":"Cher","url":"http://www.last.fm/music/Cher","picture":"http://userserve-ak.last.fm/serve/252/49248521.png"} +{"number":"342","name":"Colbie Caillat","url":"http://www.last.fm/music/Colbie+Caillat","picture":"http://userserve-ak.last.fm/serve/252/42119399.png"} +{"number":"343","name":"Annett Louisan","url":"http://www.last.fm/music/Annett+Louisan","picture":"http://userserve-ak.last.fm/serve/252/61639707.png"} +{"number":"344","name":"Taylor Swift","url":"http://www.last.fm/music/Taylor+Swift","picture":"http://userserve-ak.last.fm/serve/252/45447333.png"} +{"number":"345","name":"Nadja Benaissa","url":"http://www.last.fm/music/Nadja+Benaissa","picture":"http://userserve-ak.last.fm/serve/252/10990083.jpg"} +{"number":"346","name":"Lena","url":"http://www.last.fm/music/Lena","picture":"http://userserve-ak.last.fm/serve/252/60923125.png"} +{"number":"347","name":"Cassandra Steen","url":"http://www.last.fm/music/Cassandra+Steen","picture":"http://userserve-ak.last.fm/serve/252/24346503.jpg"} +{"number":"348","name":"Wir sind Helden","url":"http://www.last.fm/music/Wir+sind+Helden","picture":"http://userserve-ak.last.fm/serve/252/264776.jpg"} +{"number":"349","name":"The Pussycat Dolls","url":"http://www.last.fm/music/The+Pussycat+Dolls","picture":"http://userserve-ak.last.fm/serve/252/31329805.png"} +{"number":"350","name":"Monrose","url":"http://www.last.fm/music/Monrose","picture":"http://userserve-ak.last.fm/serve/252/46455007.png"} +{"number":"351","name":"Gabrielle","url":"http://www.last.fm/music/Gabrielle","picture":"http://userserve-ak.last.fm/serve/252/37483335.png"} +{"number":"352","name":"Cheryl Cole","url":"http://www.last.fm/music/Cheryl+Cole","picture":"http://userserve-ak.last.fm/serve/252/58844383.png"} +{"number":"353","name":"Milow","url":"http://www.last.fm/music/Milow","picture":"http://userserve-ak.last.fm/serve/252/3024096.jpg"} +{"number":"354","name":"Sarah Connor","url":"http://www.last.fm/music/Sarah+Connor","picture":"http://userserve-ak.last.fm/serve/252/58208833.png"} +{"number":"355","name":"Jason Mraz","url":"http://www.last.fm/music/Jason+Mraz","picture":"http://userserve-ak.last.fm/serve/252/37271.jpg"} +{"number":"356","name":"Brad Paisley","url":"http://www.last.fm/music/Brad+Paisley","picture":"http://userserve-ak.last.fm/serve/252/56984919.png"} +{"number":"357","name":"Silbermond","url":"http://www.last.fm/music/Silbermond","picture":"http://userserve-ak.last.fm/serve/252/29939807.jpg"} +{"number":"358","name":"Diyana","url":"http://www.last.fm/music/Diyana","picture":"http://userserve-ak.last.fm/serve/252/60882489.jpg"} +{"number":"359","name":"Maria Mena","url":"http://www.last.fm/music/Maria+Mena","picture":"http://userserve-ak.last.fm/serve/252/13556587.jpg"} +{"number":"360","name":"Shania Twain","url":"http://www.last.fm/music/Shania+Twain","picture":"http://userserve-ak.last.fm/serve/252/54699627.png"} +{"number":"361","name":"Jennifer Chung","url":"http://www.last.fm/music/Jennifer+Chung","picture":"http://userserve-ak.last.fm/serve/252/43498741.png"} +{"number":"362","name":"A Fine Frenzy","url":"http://www.last.fm/music/A+Fine+Frenzy","picture":"http://userserve-ak.last.fm/serve/252/42549881.png"} +{"number":"363","name":"Lucylicious","url":"http://www.last.fm/music/Lucylicious","picture":"http://userserve-ak.last.fm/serve/252/216583.jpg"} +{"number":"364","name":"Sugarland","url":"http://www.last.fm/music/Sugarland","picture":"http://userserve-ak.last.fm/serve/252/52586511.png"} +{"number":"365","name":"Christina Stürmer","url":"http://www.last.fm/music/Christina+St%C3%BCrmer","picture":"http://userserve-ak.last.fm/serve/252/60968461.png"} +{"number":"366","name":"Jack Johnson","url":"http://www.last.fm/music/Jack+Johnson","picture":"http://userserve-ak.last.fm/serve/252/3148499.jpg"} +{"number":"367","name":"BIG BANG","url":"http://www.last.fm/music/BIG+BANG","picture":"http://userserve-ak.last.fm/serve/252/61388019.png"} +{"number":"368","name":"김종국","url":"http://www.last.fm/music/%EA%B9%80%EC%A2%85%EA%B5%AD","picture":"http://userserve-ak.last.fm/serve/252/41376289.png"} +{"number":"369","name":"2PM","url":"http://www.last.fm/music/2PM","picture":"http://userserve-ak.last.fm/serve/252/47622441.png"} +{"number":"370","name":"Katatonia","url":"http://www.last.fm/music/Katatonia","picture":"http://userserve-ak.last.fm/serve/252/43161889.png"} +{"number":"371","name":"Skillet","url":"http://www.last.fm/music/Skillet","picture":"http://userserve-ak.last.fm/serve/252/34285029.png"} +{"number":"373","name":"Jay Park","url":"http://www.last.fm/music/Jay+Park","picture":"http://userserve-ak.last.fm/serve/252/54124413.png"} +{"number":"376","name":"UVERworld","url":"http://www.last.fm/music/UVERworld","picture":"http://userserve-ak.last.fm/serve/252/54744933.png"} +{"number":"377","name":"Linkin Park","url":"http://www.last.fm/music/Linkin+Park","picture":"http://userserve-ak.last.fm/serve/252/50034821.png"} +{"number":"378","name":"Evanescence","url":"http://www.last.fm/music/Evanescence","picture":"http://userserve-ak.last.fm/serve/252/8403975.jpg"} +{"number":"380","name":"Blackmore's Night","url":"http://www.last.fm/music/Blackmore%27s+Night","picture":"http://userserve-ak.last.fm/serve/252/8525311.jpg"} +{"number":"381","name":"C.N.BLUE","url":"http://www.last.fm/music/C.N.BLUE","picture":"http://userserve-ak.last.fm/serve/252/40767839.png"} +{"number":"382","name":"BUCK-TICK","url":"http://www.last.fm/music/BUCK-TICK","picture":"http://userserve-ak.last.fm/serve/252/47324357.png"} +{"number":"383","name":"Bad Religion","url":"http://www.last.fm/music/Bad+Religion","picture":"http://userserve-ak.last.fm/serve/252/311705.jpg"} +{"number":"384","name":"Fairuz","url":"http://www.last.fm/music/Fairuz","picture":"http://userserve-ak.last.fm/serve/252/43357523.jpg"} +{"number":"385","name":"T.M.Revolution","url":"http://www.last.fm/music/T.M.Revolution","picture":"http://userserve-ak.last.fm/serve/252/45550921.png"} +{"number":"386","name":"Cascada","url":"http://www.last.fm/music/Cascada","picture":"http://userserve-ak.last.fm/serve/252/58352129.png"} +{"number":"387","name":"TRAX","url":"http://www.last.fm/music/TRAX","picture":"http://userserve-ak.last.fm/serve/252/51409555.png"} +{"number":"388","name":"B2ST","url":"http://www.last.fm/music/B2ST","picture":"http://userserve-ak.last.fm/serve/252/55077729.png"} +{"number":"389","name":"Enya","url":"http://www.last.fm/music/Enya","picture":"http://userserve-ak.last.fm/serve/252/41797263.png"} +{"number":"390","name":"GACKT","url":"http://www.last.fm/music/GACKT","picture":"http://userserve-ak.last.fm/serve/252/38755357.jpg"} +{"number":"391","name":"Sinamore","url":"http://www.last.fm/music/Sinamore","picture":"http://userserve-ak.last.fm/serve/252/2838771.jpg"} +{"number":"392","name":"Remi Bandali","url":"http://www.last.fm/music/Remi+Bandali","picture":"http://userserve-ak.last.fm/serve/252/3424688.jpg"} +{"number":"393","name":"Super Junior","url":"http://www.last.fm/music/Super+Junior","picture":"http://userserve-ak.last.fm/serve/252/47844309.png"} +{"number":"395","name":"NEWS","url":"http://www.last.fm/music/NEWS","picture":"http://userserve-ak.last.fm/serve/252/45232739.png"} +{"number":"398","name":"www.DjFahad.com","url":"http://www.last.fm/music/www.DjFahad.com","picture":""} +{"number":"400","name":"2NE1","url":"http://www.last.fm/music/2NE1","picture":"http://userserve-ak.last.fm/serve/252/59385911.png"} +{"number":"401","name":"RENTRER EN SOI","url":"http://www.last.fm/music/RENTRER+EN+SOI","picture":"http://userserve-ak.last.fm/serve/252/5167122.jpg"} +{"number":"402","name":"3mmey","url":"http://www.last.fm/music/3mmey","picture":"http://userserve-ak.last.fm/serve/252/45502773.jpg"} +{"number":"403","name":"The All-American Rejects","url":"http://www.last.fm/music/The+All-American+Rejects","picture":"http://userserve-ak.last.fm/serve/252/3863300.jpg"} +{"number":"405","name":"The Rasmus","url":"http://www.last.fm/music/The+Rasmus","picture":"http://userserve-ak.last.fm/serve/252/20912833.jpg"} +{"number":"406","name":"HYDE","url":"http://www.last.fm/music/HYDE","picture":"http://userserve-ak.last.fm/serve/252/13666789.jpg"} +{"number":"407","name":"Akira Yamaoka","url":"http://www.last.fm/music/Akira+Yamaoka","picture":"http://userserve-ak.last.fm/serve/252/57243037.png"} +{"number":"408","name":"Crossfade","url":"http://www.last.fm/music/Crossfade","picture":"http://userserve-ak.last.fm/serve/252/3540719.jpg"} +{"number":"409","name":"G-Dragon","url":"http://www.last.fm/music/G-Dragon","picture":"http://userserve-ak.last.fm/serve/252/55710257.png"} +{"number":"410","name":"GD&TOP","url":"http://www.last.fm/music/GD%2526TOP","picture":"http://userserve-ak.last.fm/serve/252/56029661.png"} +{"number":"411","name":"Chinese mix","url":"http://www.last.fm/music/Chinese+mix","picture":""} +{"number":"412","name":"blink-182","url":"http://www.last.fm/music/blink-182","picture":"http://userserve-ak.last.fm/serve/252/28749561.jpg"} +{"number":"413","name":"After School","url":"http://www.last.fm/music/After+School","picture":"http://userserve-ak.last.fm/serve/252/62199081.png"} +{"number":"414","name":"Wasted Land","url":"http://www.last.fm/music/Wasted+Land","picture":"http://userserve-ak.last.fm/serve/252/184971.jpg"} +{"number":"415","name":"Laith Al-Deen","url":"http://www.last.fm/music/Laith+Al-Deen","picture":"http://userserve-ak.last.fm/serve/252/293017.jpg"} +{"number":"416","name":"The National","url":"http://www.last.fm/music/The+National","picture":"http://userserve-ak.last.fm/serve/252/51042665.jpg"} +{"number":"417","name":"Cordel do Fogo Encantado","url":"http://www.last.fm/music/Cordel+do+Fogo+Encantado","picture":"http://userserve-ak.last.fm/serve/252/10362443.jpg"} +{"number":"418","name":"Sigur Rós","url":"http://www.last.fm/music/Sigur+R%C3%B3s","picture":"http://userserve-ak.last.fm/serve/252/43524777.png"} +{"number":"419","name":"Tokyo Police Club","url":"http://www.last.fm/music/Tokyo+Police+Club","picture":"http://userserve-ak.last.fm/serve/252/13641441.jpg"} +{"number":"420","name":"The Kills","url":"http://www.last.fm/music/The+Kills","picture":"http://userserve-ak.last.fm/serve/252/37894401.png"} +{"number":"423","name":"Bat for Lashes","url":"http://www.last.fm/music/Bat+for+Lashes","picture":"http://userserve-ak.last.fm/serve/252/58240359.png"} +{"number":"424","name":"The Strokes","url":"http://www.last.fm/music/The+Strokes","picture":"http://userserve-ak.last.fm/serve/252/850741.jpg"} +{"number":"425","name":"Yeah Yeah Yeahs","url":"http://www.last.fm/music/Yeah+Yeah+Yeahs","picture":"http://userserve-ak.last.fm/serve/252/927.jpg"} +{"number":"426","name":"Rilo Kiley","url":"http://www.last.fm/music/Rilo+Kiley","picture":"http://userserve-ak.last.fm/serve/252/5029.jpg"} +{"number":"427","name":"The Cribs","url":"http://www.last.fm/music/The+Cribs","picture":"http://userserve-ak.last.fm/serve/252/7720457.jpg"} +{"number":"428","name":"The Libertines","url":"http://www.last.fm/music/The+Libertines","picture":"http://userserve-ak.last.fm/serve/252/40041379.jpg"} +{"number":"429","name":"The Smiths","url":"http://www.last.fm/music/The+Smiths","picture":"http://userserve-ak.last.fm/serve/252/49596745.png"} +{"number":"430","name":"Kate Nash","url":"http://www.last.fm/music/Kate+Nash","picture":"http://userserve-ak.last.fm/serve/252/5999731.jpg"} +{"number":"431","name":"Wolf Parade","url":"http://www.last.fm/music/Wolf+Parade","picture":"http://userserve-ak.last.fm/serve/252/301105.jpg"} +{"number":"432","name":"Klaxons","url":"http://www.last.fm/music/Klaxons","picture":"http://userserve-ak.last.fm/serve/252/53401675.png"} +{"number":"433","name":"Broken Social Scene","url":"http://www.last.fm/music/Broken+Social+Scene","picture":"http://userserve-ak.last.fm/serve/252/25629877.jpg"} +{"number":"434","name":"Spoon","url":"http://www.last.fm/music/Spoon","picture":"http://userserve-ak.last.fm/serve/252/32916.jpg"} +{"number":"435","name":"Black Lips","url":"http://www.last.fm/music/Black+Lips","picture":"http://userserve-ak.last.fm/serve/252/21449125.jpg"} +{"number":"436","name":"The Smashing Pumpkins","url":"http://www.last.fm/music/The+Smashing+Pumpkins","picture":"http://userserve-ak.last.fm/serve/252/887234.jpg"} +{"number":"437","name":"She & Him","url":"http://www.last.fm/music/She%2B%2526%2BHim","picture":"http://userserve-ak.last.fm/serve/252/57197011.png"} +{"number":"438","name":"O Teatro Mágico","url":"http://www.last.fm/music/O+Teatro+M%C3%A1gico","picture":"http://userserve-ak.last.fm/serve/252/18060895.gif"} +{"number":"439","name":"Los Hermanos","url":"http://www.last.fm/music/Los+Hermanos","picture":"http://userserve-ak.last.fm/serve/252/22264369.jpg"} +{"number":"440","name":"Animal Collective","url":"http://www.last.fm/music/Animal+Collective","picture":"http://userserve-ak.last.fm/serve/252/53139735.png"} +{"number":"441","name":"Joy Division","url":"http://www.last.fm/music/Joy+Division","picture":"http://userserve-ak.last.fm/serve/252/4227094.jpg"} +{"number":"442","name":"The Tallest Man on Earth","url":"http://www.last.fm/music/The+Tallest+Man+on+Earth","picture":"http://userserve-ak.last.fm/serve/252/55778717.png"} +{"number":"443","name":"The Rakes","url":"http://www.last.fm/music/The+Rakes","picture":"http://userserve-ak.last.fm/serve/252/416213.jpg"} +{"number":"445","name":"Bon Iver","url":"http://www.last.fm/music/Bon+Iver","picture":"http://userserve-ak.last.fm/serve/252/4406986.jpg"} +{"number":"446","name":"Ra Ra Riot","url":"http://www.last.fm/music/Ra+Ra+Riot","picture":"http://userserve-ak.last.fm/serve/252/45624701.png"} +{"number":"447","name":"Manu Chao","url":"http://www.last.fm/music/Manu+Chao","picture":"http://userserve-ak.last.fm/serve/252/22560345.jpg"} +{"number":"448","name":"Art Brut","url":"http://www.last.fm/music/Art+Brut","picture":"http://userserve-ak.last.fm/serve/252/266856.jpg"} +{"number":"449","name":"Poney Express","url":"http://www.last.fm/music/Poney+Express","picture":"http://userserve-ak.last.fm/serve/252/5258513.jpg"} +{"number":"450","name":"She Wants Revenge","url":"http://www.last.fm/music/She+Wants+Revenge","picture":"http://userserve-ak.last.fm/serve/252/4967577.jpg"} +{"number":"451","name":"Local Natives","url":"http://www.last.fm/music/Local+Natives","picture":"http://userserve-ak.last.fm/serve/252/40466355.jpg"} +{"number":"452","name":"Mundo Livre S/A","url":"http://www.last.fm/music/Mundo%2BLivre%2BS%252FA","picture":"http://userserve-ak.last.fm/serve/252/56573.jpg"} +{"number":"453","name":"Silverchair","url":"http://www.last.fm/music/Silverchair","picture":"http://userserve-ak.last.fm/serve/252/20540173.jpg"} +{"number":"455","name":"Backstreet Boys","url":"http://www.last.fm/music/Backstreet+Boys","picture":"http://userserve-ak.last.fm/serve/252/35322101.png"} +{"number":"456","name":"Jesse McCartney","url":"http://www.last.fm/music/Jesse+McCartney","picture":"http://userserve-ak.last.fm/serve/252/54340985.png"} +{"number":"457","name":"Aaron Carter","url":"http://www.last.fm/music/Aaron+Carter","picture":"http://userserve-ak.last.fm/serve/252/41791623.png"} +{"number":"458","name":"Justin Bieber","url":"http://www.last.fm/music/Justin+Bieber","picture":"http://userserve-ak.last.fm/serve/252/61499269.png"} +{"number":"459","name":"*NSYNC","url":"http://www.last.fm/music/%2ANSYNC","picture":"http://userserve-ak.last.fm/serve/252/51725803.png"} +{"number":"460","name":"Noize MC","url":"http://www.last.fm/music/Noize+MC","picture":"http://userserve-ak.last.fm/serve/252/45141167.jpg"} +{"number":"461","name":"Miley Cyrus","url":"http://www.last.fm/music/Miley+Cyrus","picture":"http://userserve-ak.last.fm/serve/252/58452885.png"} +{"number":"462","name":"Blue","url":"http://www.last.fm/music/Blue","picture":"http://userserve-ak.last.fm/serve/252/60025255.jpg"} +{"number":"463","name":"Savage Garden","url":"http://www.last.fm/music/Savage+Garden","picture":"http://userserve-ak.last.fm/serve/252/45487441.png"} +{"number":"464","name":"3OH!3","url":"http://www.last.fm/music/3OH%213","picture":"http://userserve-ak.last.fm/serve/252/46792355.png"} +{"number":"465","name":"Jessica Simpson","url":"http://www.last.fm/music/Jessica+Simpson","picture":"http://userserve-ak.last.fm/serve/252/44836897.png"} +{"number":"466","name":"Ke$ha","url":"http://www.last.fm/music/Ke%24ha","picture":"http://userserve-ak.last.fm/serve/252/47829587.png"} +{"number":"467","name":"Darren Hayes","url":"http://www.last.fm/music/Darren+Hayes","picture":"http://userserve-ak.last.fm/serve/252/57535553.jpg"} +{"number":"468","name":"Usher","url":"http://www.last.fm/music/Usher","picture":"http://userserve-ak.last.fm/serve/252/12220759.jpg"} +{"number":"469","name":"Nick Carter","url":"http://www.last.fm/music/Nick+Carter","picture":"http://userserve-ak.last.fm/serve/252/58260599.png"} +{"number":"470","name":"nevershoutnever!","url":"http://www.last.fm/music/nevershoutnever%21","picture":"http://userserve-ak.last.fm/serve/252/58915783.png"} +{"number":"471","name":"Metro Station","url":"http://www.last.fm/music/Metro+Station","picture":"http://userserve-ak.last.fm/serve/252/8127003.jpg"} +{"number":"472","name":"3 Doors Down","url":"http://www.last.fm/music/3+Doors+Down","picture":"http://userserve-ak.last.fm/serve/252/19984097.jpg"} +{"number":"473","name":"Dream Street","url":"http://www.last.fm/music/Dream+Street","picture":"http://userserve-ak.last.fm/serve/252/39024277.png"} +{"number":"474","name":"Craig David","url":"http://www.last.fm/music/Craig+David","picture":"http://userserve-ak.last.fm/serve/252/46140519.jpg"} +{"number":"475","name":"Eminem","url":"http://www.last.fm/music/Eminem","picture":"http://userserve-ak.last.fm/serve/252/29959269.jpg"} +{"number":"476","name":"M. Pokora","url":"http://www.last.fm/music/M.+Pokora","picture":"http://userserve-ak.last.fm/serve/252/52020305.png"} +{"number":"477","name":"Daniel Merriweather","url":"http://www.last.fm/music/Daniel+Merriweather","picture":"http://userserve-ak.last.fm/serve/252/53480041.png"} +{"number":"479","name":"Westlife","url":"http://www.last.fm/music/Westlife","picture":"http://userserve-ak.last.fm/serve/252/47700103.png"} +{"number":"480","name":"Guf","url":"http://www.last.fm/music/Guf","picture":"http://userserve-ak.last.fm/serve/252/60174913.jpg"} +{"number":"481","name":"Simple Plan","url":"http://www.last.fm/music/Simple+Plan","picture":"http://userserve-ak.last.fm/serve/252/61544537.png"} +{"number":"482","name":"Jeremih","url":"http://www.last.fm/music/Jeremih","picture":"http://userserve-ak.last.fm/serve/252/48134259.png"} +{"number":"483","name":"Centr","url":"http://www.last.fm/music/Centr","picture":"http://userserve-ak.last.fm/serve/252/33306611.jpg"} +{"number":"484","name":"Ne-Yo","url":"http://www.last.fm/music/Ne-Yo","picture":"http://userserve-ak.last.fm/serve/252/8565803.jpg"} +{"number":"485","name":"Kelis","url":"http://www.last.fm/music/Kelis","picture":"http://userserve-ak.last.fm/serve/252/51593689.png"} +{"number":"486","name":"30 Seconds to Mars","url":"http://www.last.fm/music/30+Seconds+to+Mars","picture":"http://userserve-ak.last.fm/serve/252/47014513.png"} +{"number":"487","name":"The Birthday Massacre","url":"http://www.last.fm/music/The+Birthday+Massacre","picture":"http://userserve-ak.last.fm/serve/252/51539353.png"} +{"number":"488","name":"Three Days Grace","url":"http://www.last.fm/music/Three+Days+Grace","picture":"http://userserve-ak.last.fm/serve/252/34286677.jpg"} +{"number":"489","name":"Limp Bizkit","url":"http://www.last.fm/music/Limp+Bizkit","picture":"http://userserve-ak.last.fm/serve/252/3044858.jpg"} +{"number":"490","name":"Seether","url":"http://www.last.fm/music/Seether","picture":"http://userserve-ak.last.fm/serve/252/59999971.jpg"} +{"number":"491","name":"Fort Minor","url":"http://www.last.fm/music/Fort+Minor","picture":"http://userserve-ak.last.fm/serve/252/142200.jpg"} +{"number":"492","name":"Funeral for a Friend","url":"http://www.last.fm/music/Funeral+for+a+Friend","picture":"http://userserve-ak.last.fm/serve/252/40840695.jpg"} +{"number":"493","name":"Lostprophets","url":"http://www.last.fm/music/Lostprophets","picture":"http://userserve-ak.last.fm/serve/252/208529.jpg"} +{"number":"494","name":"Finch","url":"http://www.last.fm/music/Finch","picture":"http://userserve-ak.last.fm/serve/252/8905303.jpg"} +{"number":"495","name":"This Beautiful Republic","url":"http://www.last.fm/music/This+Beautiful+Republic","picture":"http://userserve-ak.last.fm/serve/252/585346.jpg"} +{"number":"496","name":"Cold","url":"http://www.last.fm/music/Cold","picture":"http://userserve-ak.last.fm/serve/252/38920225.png"} +{"number":"497","name":"Papa Roach","url":"http://www.last.fm/music/Papa+Roach","picture":"http://userserve-ak.last.fm/serve/252/15383743.jpg"} +{"number":"498","name":"Paramore","url":"http://www.last.fm/music/Paramore","picture":"http://userserve-ak.last.fm/serve/252/35837991.png"} +{"number":"499","name":"Flyleaf","url":"http://www.last.fm/music/Flyleaf","picture":"http://userserve-ak.last.fm/serve/252/44511321.png"} +{"number":"500","name":"Bullet for My Valentine","url":"http://www.last.fm/music/Bullet+for+My+Valentine","picture":"http://userserve-ak.last.fm/serve/252/45613985.jpg"} +{"number":"501","name":"Angels & Airwaves","url":"http://www.last.fm/music/Angels%2B%2526%2BAirwaves","picture":"http://userserve-ak.last.fm/serve/252/2455791.jpg"} +{"number":"502","name":"LaFee","url":"http://www.last.fm/music/LaFee","picture":"http://userserve-ak.last.fm/serve/252/18427817.jpg"} +{"number":"503","name":"In Flames","url":"http://www.last.fm/music/In+Flames","picture":"http://userserve-ak.last.fm/serve/252/12732413.jpg"} +{"number":"504","name":"HIM","url":"http://www.last.fm/music/HIM","picture":"http://userserve-ak.last.fm/serve/252/148293.jpg"} +{"number":"505","name":"Digital Summer","url":"http://www.last.fm/music/Digital+Summer","picture":"http://userserve-ak.last.fm/serve/252/23301839.jpg"} +{"number":"506","name":"Anathema","url":"http://www.last.fm/music/Anathema","picture":"http://userserve-ak.last.fm/serve/252/45858121.png"} +{"number":"507","name":"Saosin","url":"http://www.last.fm/music/Saosin","picture":"http://userserve-ak.last.fm/serve/252/35608193.jpg"} +{"number":"508","name":"Tracktor Bowling","url":"http://www.last.fm/music/Tracktor+Bowling","picture":"http://userserve-ak.last.fm/serve/252/35671525.jpg"} +{"number":"509","name":"Pitchshifter","url":"http://www.last.fm/music/Pitchshifter","picture":"http://userserve-ak.last.fm/serve/252/29307365.jpg"} +{"number":"510","name":"P.O.D.","url":"http://www.last.fm/music/P.O.D.","picture":"http://userserve-ak.last.fm/serve/252/8949521.jpg"} +{"number":"511","name":"U2","url":"http://www.last.fm/music/U2","picture":"http://userserve-ak.last.fm/serve/252/4404698.jpg"} +{"number":"512","name":"Deftones","url":"http://www.last.fm/music/Deftones","picture":"http://userserve-ak.last.fm/serve/252/8057433.jpg"} +{"number":"513","name":"Traff!c","url":"http://www.last.fm/music/Traff%21c","picture":"http://userserve-ak.last.fm/serve/252/45754471.jpg"} +{"number":"515","name":"Clan of Xymox","url":"http://www.last.fm/music/Clan+of+Xymox","picture":"http://userserve-ak.last.fm/serve/252/35122187.jpg"} +{"number":"516","name":"Celldweller","url":"http://www.last.fm/music/Celldweller","picture":"http://userserve-ak.last.fm/serve/252/57774913.jpg"} +{"number":"517","name":"Korn","url":"http://www.last.fm/music/Korn","picture":"http://userserve-ak.last.fm/serve/252/30098555.jpg"} +{"number":"518","name":"Within Temptation","url":"http://www.last.fm/music/Within+Temptation","picture":"http://userserve-ak.last.fm/serve/252/45507049.png"} +{"number":"519","name":"Static-X","url":"http://www.last.fm/music/Static-X","picture":"http://userserve-ak.last.fm/serve/252/2272261.jpg"} +{"number":"520","name":"Fluke","url":"http://www.last.fm/music/Fluke","picture":"http://userserve-ak.last.fm/serve/252/36255881.jpg"} +{"number":"521","name":"happysad","url":"http://www.last.fm/music/happysad","picture":"http://userserve-ak.last.fm/serve/252/39558579.jpg"} +{"number":"522","name":"Lacrimosa","url":"http://www.last.fm/music/Lacrimosa","picture":"http://userserve-ak.last.fm/serve/252/70298.jpg"} +{"number":"523","name":"Lindsay Lohan","url":"http://www.last.fm/music/Lindsay+Lohan","picture":"http://userserve-ak.last.fm/serve/252/49266505.png"} +{"number":"524","name":"Jeffree Star","url":"http://www.last.fm/music/Jeffree+Star","picture":"http://userserve-ak.last.fm/serve/252/49725955.jpg"} +{"number":"525","name":"Gwen Stefani","url":"http://www.last.fm/music/Gwen+Stefani","picture":"http://userserve-ak.last.fm/serve/252/8516439.jpg"} +{"number":"526","name":"La Roux","url":"http://www.last.fm/music/La+Roux","picture":"http://userserve-ak.last.fm/serve/252/30382691.png"} +{"number":"527","name":"Lil' Wayne","url":"http://www.last.fm/music/Lil%27+Wayne","picture":"http://userserve-ak.last.fm/serve/252/42945879.jpg"} +{"number":"528","name":"Pixie Lott","url":"http://www.last.fm/music/Pixie+Lott","picture":"http://userserve-ak.last.fm/serve/252/40989569.png"} +{"number":"529","name":"Paul van Dyk","url":"http://www.last.fm/music/Paul+van+Dyk","picture":"http://userserve-ak.last.fm/serve/252/11178015.jpg"} +{"number":"530","name":"Alphabeat","url":"http://www.last.fm/music/Alphabeat","picture":"http://userserve-ak.last.fm/serve/252/22573401.jpg"} +{"number":"531","name":"Taio Cruz","url":"http://www.last.fm/music/Taio+Cruz","picture":"http://userserve-ak.last.fm/serve/252/43067677.png"} +{"number":"532","name":"Little Boots","url":"http://www.last.fm/music/Little+Boots","picture":"http://userserve-ak.last.fm/serve/252/30669737.png"} +{"number":"533","name":"Oasis","url":"http://www.last.fm/music/Oasis","picture":"http://userserve-ak.last.fm/serve/252/44937531.jpg"} +{"number":"534","name":"No Doubt","url":"http://www.last.fm/music/No+Doubt","picture":"http://userserve-ak.last.fm/serve/252/42409677.png"} +{"number":"535","name":"Sophie Ellis-Bextor","url":"http://www.last.fm/music/Sophie+Ellis-Bextor","picture":"http://userserve-ak.last.fm/serve/252/61811325.png"} +{"number":"536","name":"Good Charlotte","url":"http://www.last.fm/music/Good+Charlotte","picture":"http://userserve-ak.last.fm/serve/252/55721971.jpg"} +{"number":"537","name":"The Used","url":"http://www.last.fm/music/The+Used","picture":"http://userserve-ak.last.fm/serve/252/28575803.jpg"} +{"number":"538","name":"Maroon 5","url":"http://www.last.fm/music/Maroon+5","picture":"http://userserve-ak.last.fm/serve/252/364476.jpg"} +{"number":"539","name":"Gossip","url":"http://www.last.fm/music/Gossip","picture":"http://userserve-ak.last.fm/serve/252/33742155.png"} +{"number":"540","name":"Fall Out Boy","url":"http://www.last.fm/music/Fall+Out+Boy","picture":"http://userserve-ak.last.fm/serve/252/20181975.jpg"} +{"number":"541","name":"Space Cowboy","url":"http://www.last.fm/music/Space+Cowboy","picture":"http://userserve-ak.last.fm/serve/252/41873663.jpg"} +{"number":"542","name":"Pitty","url":"http://www.last.fm/music/Pitty","picture":"http://userserve-ak.last.fm/serve/252/61839067.png"} +{"number":"543","name":"Nicole Scherzinger","url":"http://www.last.fm/music/Nicole+Scherzinger","picture":"http://userserve-ak.last.fm/serve/252/57666385.png"} +{"number":"544","name":"Adam Lambert","url":"http://www.last.fm/music/Adam+Lambert","picture":"http://userserve-ak.last.fm/serve/252/56430315.png"} +{"number":"545","name":"Flo Rida","url":"http://www.last.fm/music/Flo+Rida","picture":"http://userserve-ak.last.fm/serve/252/52067183.png"} +{"number":"546","name":"The Ting Tings","url":"http://www.last.fm/music/The+Ting+Tings","picture":"http://userserve-ak.last.fm/serve/252/39614369.jpg"} +{"number":"547","name":"Kid Cudi","url":"http://www.last.fm/music/Kid+Cudi","picture":"http://userserve-ak.last.fm/serve/252/33794067.png"} +{"number":"548","name":"Ellie Goulding","url":"http://www.last.fm/music/Ellie+Goulding","picture":"http://userserve-ak.last.fm/serve/252/62251875.png"} +{"number":"549","name":"The Sonics","url":"http://www.last.fm/music/The+Sonics","picture":"http://userserve-ak.last.fm/serve/252/324895.jpg"} +{"number":"550","name":"The Fleshtones","url":"http://www.last.fm/music/The+Fleshtones","picture":"http://userserve-ak.last.fm/serve/252/448042.jpg"} +{"number":"551","name":"The Satelliters","url":"http://www.last.fm/music/The+Satelliters","picture":"http://userserve-ak.last.fm/serve/252/59240681.jpg"} +{"number":"552","name":"The Barracudas","url":"http://www.last.fm/music/The+Barracudas","picture":"http://userserve-ak.last.fm/serve/252/515299.jpg"} +{"number":"553","name":"The Staggers","url":"http://www.last.fm/music/The+Staggers","picture":"http://userserve-ak.last.fm/serve/252/587037.jpg"} +{"number":"554","name":"The Kinks","url":"http://www.last.fm/music/The+Kinks","picture":"http://userserve-ak.last.fm/serve/252/295886.jpg"} +{"number":"555","name":"The Swingin' Neckbreakers","url":"http://www.last.fm/music/The+Swingin%27+Neckbreakers","picture":"http://userserve-ak.last.fm/serve/252/29491.jpg"} +{"number":"556","name":"The Strollers","url":"http://www.last.fm/music/The+Strollers","picture":"http://userserve-ak.last.fm/serve/252/359302.jpg"} +{"number":"557","name":"The Fuzztones","url":"http://www.last.fm/music/The+Fuzztones","picture":"http://userserve-ak.last.fm/serve/252/24413387.jpg"} +{"number":"558","name":"The Maharajas","url":"http://www.last.fm/music/The+Maharajas","picture":"http://userserve-ak.last.fm/serve/252/443985.jpg"} +{"number":"559","name":"Southern Culture on the Skids","url":"http://www.last.fm/music/Southern+Culture+on+the+Skids","picture":"http://userserve-ak.last.fm/serve/252/101211.jpg"} +{"number":"560","name":"The Woggles","url":"http://www.last.fm/music/The+Woggles","picture":"http://userserve-ak.last.fm/serve/252/471025.jpg"} +{"number":"561","name":"The Untamed Youth","url":"http://www.last.fm/music/The+Untamed+Youth","picture":"http://userserve-ak.last.fm/serve/252/127867.jpg"} +{"number":"562","name":"The Who","url":"http://www.last.fm/music/The+Who","picture":"http://userserve-ak.last.fm/serve/252/361952.jpg"} +{"number":"563","name":"Small Faces","url":"http://www.last.fm/music/Small+Faces","picture":"http://userserve-ak.last.fm/serve/252/24083505.jpg"} +{"number":"564","name":"Mahalia Jackson","url":"http://www.last.fm/music/Mahalia+Jackson","picture":"http://userserve-ak.last.fm/serve/252/296730.jpg"} +{"number":"565","name":"Howlin' Wolf","url":"http://www.last.fm/music/Howlin%27+Wolf","picture":"http://userserve-ak.last.fm/serve/252/28517223.jpg"} +{"number":"566","name":"Jackie Wilson","url":"http://www.last.fm/music/Jackie+Wilson","picture":"http://userserve-ak.last.fm/serve/252/10248203.jpg"} +{"number":"567","name":"Sam Cooke","url":"http://www.last.fm/music/Sam+Cooke","picture":"http://userserve-ak.last.fm/serve/252/39227301.png"} +{"number":"568","name":"The Vice Barons","url":"http://www.last.fm/music/The+Vice+Barons","picture":"http://userserve-ak.last.fm/serve/252/49196981.png"} +{"number":"569","name":"Otis Redding","url":"http://www.last.fm/music/Otis+Redding","picture":"http://userserve-ak.last.fm/serve/252/51686497.png"} +{"number":"570","name":"Little Richard","url":"http://www.last.fm/music/Little+Richard","picture":"http://userserve-ak.last.fm/serve/252/17766921.jpg"} +{"number":"571","name":"LaVern Baker","url":"http://www.last.fm/music/LaVern+Baker","picture":"http://userserve-ak.last.fm/serve/252/151159.jpg"} +{"number":"572","name":"Ray Charles","url":"http://www.last.fm/music/Ray+Charles","picture":"http://userserve-ak.last.fm/serve/252/4766.jpg"} +{"number":"573","name":"The Detroit Cobras","url":"http://www.last.fm/music/The+Detroit+Cobras","picture":"http://userserve-ak.last.fm/serve/252/380643.jpg"} +{"number":"574","name":"Big Mama Thornton","url":"http://www.last.fm/music/Big+Mama+Thornton","picture":"http://userserve-ak.last.fm/serve/252/98773.jpg"} +{"number":"575","name":"Aretha Franklin","url":"http://www.last.fm/music/Aretha+Franklin","picture":"http://userserve-ak.last.fm/serve/252/12028673.jpg"} +{"number":"576","name":"Marvin Gaye","url":"http://www.last.fm/music/Marvin+Gaye","picture":"http://userserve-ak.last.fm/serve/252/526662.jpg"} +{"number":"577","name":"Ruth Brown","url":"http://www.last.fm/music/Ruth+Brown","picture":"http://userserve-ak.last.fm/serve/252/28488275.jpg"} +{"number":"578","name":"James Brown","url":"http://www.last.fm/music/James+Brown","picture":"http://userserve-ak.last.fm/serve/252/28911695.jpg"} +{"number":"579","name":"Big Maybelle","url":"http://www.last.fm/music/Big+Maybelle","picture":"http://userserve-ak.last.fm/serve/252/8896.jpg"} +{"number":"580","name":"Muddy Waters","url":"http://www.last.fm/music/Muddy+Waters","picture":"http://userserve-ak.last.fm/serve/252/60495943.png"} +{"number":"581","name":"Elmore James","url":"http://www.last.fm/music/Elmore+James","picture":"http://userserve-ak.last.fm/serve/252/768357.jpg"} +{"number":"582","name":"Sonny Boy Williamson","url":"http://www.last.fm/music/Sonny+Boy+Williamson","picture":"http://userserve-ak.last.fm/serve/252/52908121.jpg"} +{"number":"583","name":"John Lee Hooker","url":"http://www.last.fm/music/John+Lee+Hooker","picture":"http://userserve-ak.last.fm/serve/252/16210.jpg"} +{"number":"584","name":"The Isley Brothers","url":"http://www.last.fm/music/The+Isley+Brothers","picture":"http://userserve-ak.last.fm/serve/252/38349487.jpg"} +{"number":"585","name":"Chuck Berry","url":"http://www.last.fm/music/Chuck+Berry","picture":"http://userserve-ak.last.fm/serve/252/53033665.jpg"} +{"number":"586","name":"Professor Longhair","url":"http://www.last.fm/music/Professor+Longhair","picture":"http://userserve-ak.last.fm/serve/252/23486483.jpg"} +{"number":"587","name":"Sister Rosetta Tharpe","url":"http://www.last.fm/music/Sister+Rosetta+Tharpe","picture":"http://userserve-ak.last.fm/serve/252/4311743.jpg"} +{"number":"588","name":"Big Joe Turner","url":"http://www.last.fm/music/Big+Joe+Turner","picture":"http://userserve-ak.last.fm/serve/252/6338289.jpg"} +{"number":"589","name":"Andre Williams","url":"http://www.last.fm/music/Andre+Williams","picture":"http://userserve-ak.last.fm/serve/252/560208.gif"} +{"number":"590","name":"Louis Prima","url":"http://www.last.fm/music/Louis+Prima","picture":"http://userserve-ak.last.fm/serve/252/10434899.jpg"} +{"number":"591","name":"Charles Aznavour","url":"http://www.last.fm/music/Charles+Aznavour","picture":"http://userserve-ak.last.fm/serve/252/4530052.jpg"} +{"number":"592","name":"Ennio Morricone","url":"http://www.last.fm/music/Ennio+Morricone","picture":"http://userserve-ak.last.fm/serve/252/9254923.jpg"} +{"number":"593","name":"Renato Carosone","url":"http://www.last.fm/music/Renato+Carosone","picture":"http://userserve-ak.last.fm/serve/252/256914.jpg"} +{"number":"594","name":"The Pretty Things","url":"http://www.last.fm/music/The+Pretty+Things","picture":"http://userserve-ak.last.fm/serve/252/444502.jpg"} +{"number":"595","name":"The Titty Twisters Orchestra","url":"http://www.last.fm/music/The+Titty+Twisters+Orchestra","picture":"http://userserve-ak.last.fm/serve/252/4157715.jpg"} +{"number":"596","name":"The Montesas","url":"http://www.last.fm/music/The+Montesas","picture":"http://userserve-ak.last.fm/serve/252/147639.gif"} +{"number":"597","name":"The Easybeats","url":"http://www.last.fm/music/The+Easybeats","picture":"http://userserve-ak.last.fm/serve/252/32824491.jpg"} +{"number":"598","name":"The Gruesomes","url":"http://www.last.fm/music/The+Gruesomes","picture":"http://userserve-ak.last.fm/serve/252/232873.jpg"} +{"number":"599","name":"David Bowie","url":"http://www.last.fm/music/David+Bowie","picture":"http://userserve-ak.last.fm/serve/252/13533737.jpg"} +{"number":"600","name":"le bowl","url":"http://www.last.fm/music/le+bowl","picture":"http://userserve-ak.last.fm/serve/252/23244153.jpg"} +{"number":"601","name":"Prince","url":"http://www.last.fm/music/Prince","picture":"http://userserve-ak.last.fm/serve/252/3031469.jpg"} +{"number":"602","name":"Iron & Wine","url":"http://www.last.fm/music/Iron%2B%2526%2BWine","picture":"http://userserve-ak.last.fm/serve/252/11865571.jpg"} +{"number":"603","name":"Aphex Twin","url":"http://www.last.fm/music/Aphex+Twin","picture":"http://userserve-ak.last.fm/serve/252/43221327.jpg"} +{"number":"604","name":"Kings of Convenience","url":"http://www.last.fm/music/Kings+of+Convenience","picture":"http://userserve-ak.last.fm/serve/252/39430071.png"} +{"number":"605","name":"Beastie Boys","url":"http://www.last.fm/music/Beastie+Boys","picture":"http://userserve-ak.last.fm/serve/252/62238015.jpg"} +{"number":"607","name":"Jethro Tull","url":"http://www.last.fm/music/Jethro+Tull","picture":"http://userserve-ak.last.fm/serve/252/30573677.png"} +{"number":"608","name":"Tortoise","url":"http://www.last.fm/music/Tortoise","picture":"http://userserve-ak.last.fm/serve/252/298071.jpg"} +{"number":"609","name":"Nick Drake","url":"http://www.last.fm/music/Nick+Drake","picture":"http://userserve-ak.last.fm/serve/252/106225.jpg"} +{"number":"610","name":"Miles Davis","url":"http://www.last.fm/music/Miles+Davis","picture":"http://userserve-ak.last.fm/serve/252/48180827.png"} +{"number":"611","name":"The Sea and Cake","url":"http://www.last.fm/music/The+Sea+and+Cake","picture":"http://userserve-ak.last.fm/serve/252/26682329.jpg"} +{"number":"612","name":"Talking Heads","url":"http://www.last.fm/music/Talking+Heads","picture":"http://userserve-ak.last.fm/serve/252/20121955.jpg"} +{"number":"613","name":"John Coltrane","url":"http://www.last.fm/music/John+Coltrane","picture":"http://userserve-ak.last.fm/serve/252/55149557.png"} +{"number":"614","name":"Pearl Jam","url":"http://www.last.fm/music/Pearl+Jam","picture":"http://userserve-ak.last.fm/serve/252/3019178.jpg"} +{"number":"615","name":"Elliott Smith","url":"http://www.last.fm/music/Elliott+Smith","picture":"http://userserve-ak.last.fm/serve/252/1088.jpg"} +{"number":"616","name":"Squarepusher","url":"http://www.last.fm/music/Squarepusher","picture":"http://userserve-ak.last.fm/serve/252/42035371.png"} +{"number":"617","name":"The Roots","url":"http://www.last.fm/music/The+Roots","picture":"http://userserve-ak.last.fm/serve/252/8526611.jpg"} +{"number":"618","name":"Belle and Sebastian","url":"http://www.last.fm/music/Belle+and+Sebastian","picture":"http://userserve-ak.last.fm/serve/252/309385.jpg"} +{"number":"619","name":"A Tribe Called Quest","url":"http://www.last.fm/music/A+Tribe+Called+Quest","picture":"http://userserve-ak.last.fm/serve/252/49444149.jpg"} +{"number":"620","name":"Pulp","url":"http://www.last.fm/music/Pulp","picture":"http://userserve-ak.last.fm/serve/252/5881671.jpg"} +{"number":"621","name":"The Album Leaf","url":"http://www.last.fm/music/The+Album+Leaf","picture":"http://userserve-ak.last.fm/serve/252/34666505.jpg"} +{"number":"622","name":"Sufjan Stevens","url":"http://www.last.fm/music/Sufjan+Stevens","picture":"http://userserve-ak.last.fm/serve/252/2277029.jpg"} +{"number":"623","name":"Clogs","url":"http://www.last.fm/music/Clogs","picture":"http://userserve-ak.last.fm/serve/252/33322.jpg"} +{"number":"624","name":"Great Lake Swimmers","url":"http://www.last.fm/music/Great+Lake+Swimmers","picture":"http://userserve-ak.last.fm/serve/252/6441071.jpg"} +{"number":"625","name":"Thelonious Monk","url":"http://www.last.fm/music/Thelonious+Monk","picture":"http://userserve-ak.last.fm/serve/252/49613369.png"} +{"number":"626","name":"A.A. Bondy","url":"http://www.last.fm/music/A.A.+Bondy","picture":"http://userserve-ak.last.fm/serve/252/27676573.jpg"} +{"number":"627","name":"Saint Etienne","url":"http://www.last.fm/music/Saint+Etienne","picture":"http://userserve-ak.last.fm/serve/252/605045.jpg"} +{"number":"628","name":"Andrew York","url":"http://www.last.fm/music/Andrew+York","picture":"http://userserve-ak.last.fm/serve/252/34994.jpg"} +{"number":"629","name":"of Montreal","url":"http://www.last.fm/music/of+Montreal","picture":"http://userserve-ak.last.fm/serve/252/49302801.jpg"} +{"number":"630","name":"Neil Young","url":"http://www.last.fm/music/Neil+Young","picture":"http://userserve-ak.last.fm/serve/252/50538451.jpg"} +{"number":"632","name":"Explosions in the Sky","url":"http://www.last.fm/music/Explosions+in+the+Sky","picture":"http://userserve-ak.last.fm/serve/252/33593019.jpg"} +{"number":"633","name":"Ornette Coleman","url":"http://www.last.fm/music/Ornette+Coleman","picture":"http://userserve-ak.last.fm/serve/252/119080.jpg"} +{"number":"635","name":"Teen Angels","url":"http://www.last.fm/music/Teen+Angels","picture":"http://userserve-ak.last.fm/serve/252/57784433.png"} +{"number":"636","name":"Ivi Adamou","url":"http://www.last.fm/music/Ivi+Adamou","picture":"http://userserve-ak.last.fm/serve/252/61695741.png"} +{"number":"637","name":"Helena Paparizou","url":"http://www.last.fm/music/Helena+Paparizou","picture":"http://userserve-ak.last.fm/serve/252/38768601.png"} +{"number":"638","name":"Kalomoira","url":"http://www.last.fm/music/Kalomoira","picture":"http://userserve-ak.last.fm/serve/252/37288727.png"} +{"number":"641","name":"Laura Pausini","url":"http://www.last.fm/music/Laura+Pausini","picture":"http://userserve-ak.last.fm/serve/252/44209239.png"} +{"number":"642","name":"Banghra","url":"http://www.last.fm/music/Banghra","picture":"http://userserve-ak.last.fm/serve/252/30425439.png"} +{"number":"643","name":"Nikolas Metaxas","url":"http://www.last.fm/music/Nikolas+Metaxas","picture":"http://userserve-ak.last.fm/serve/252/61486893.png"} +{"number":"644","name":"Inna","url":"http://www.last.fm/music/Inna","picture":"http://userserve-ak.last.fm/serve/252/49314569.png"} +{"number":"645","name":"Stavento","url":"http://www.last.fm/music/Stavento","picture":"http://userserve-ak.last.fm/serve/252/57055801.png"} +{"number":"646","name":"RBD","url":"http://www.last.fm/music/RBD","picture":"http://userserve-ak.last.fm/serve/252/59211131.png"} +{"number":"647","name":"Eleftheria Eleftheriou","url":"http://www.last.fm/music/Eleftheria+Eleftheriou","picture":"http://userserve-ak.last.fm/serve/252/55899357.png"} +{"number":"648","name":"Antonis Remos","url":"http://www.last.fm/music/Antonis+Remos","picture":"http://userserve-ak.last.fm/serve/252/61150255.png"} +{"number":"649","name":"D'Nash","url":"http://www.last.fm/music/D%27Nash","picture":"http://userserve-ak.last.fm/serve/252/47079175.png"} +{"number":"650","name":"Giorgos Papadopoulos","url":"http://www.last.fm/music/Giorgos+Papadopoulos","picture":"http://userserve-ak.last.fm/serve/252/61754203.png"} +{"number":"651","name":"Funky G","url":"http://www.last.fm/music/Funky+G","picture":"http://userserve-ak.last.fm/serve/252/24092475.jpg"} +{"number":"652","name":"Hadise","url":"http://www.last.fm/music/Hadise","picture":"http://userserve-ak.last.fm/serve/252/61615633.png"} +{"number":"653","name":"Watch Tower Bible and Tract Society of PA","url":"http://www.last.fm/music/Watch+Tower+Bible+and+Tract+Society+of+PA","picture":"http://userserve-ak.last.fm/serve/252/35165855.png"} +{"number":"654","name":"Onirama","url":"http://www.last.fm/music/Onirama","picture":"http://userserve-ak.last.fm/serve/252/53933367.png"} +{"number":"655","name":"Jessy Matador","url":"http://www.last.fm/music/Jessy+Matador","picture":"http://userserve-ak.last.fm/serve/252/50127217.png"} +{"number":"656","name":"Master Tempo","url":"http://www.last.fm/music/Master+Tempo","picture":"http://userserve-ak.last.fm/serve/252/54849743.png"} +{"number":"657","name":"Paolo Meneguzzi","url":"http://www.last.fm/music/Paolo+Meneguzzi","picture":"http://userserve-ak.last.fm/serve/252/8575439.jpg"} +{"number":"658","name":"Floricienta","url":"http://www.last.fm/music/Floricienta","picture":"http://userserve-ak.last.fm/serve/252/2562485.jpg"} +{"number":"659","name":"Erreway","url":"http://www.last.fm/music/Erreway","picture":"http://userserve-ak.last.fm/serve/252/22940439.jpg"} +{"number":"660","name":"Dima Bilan","url":"http://www.last.fm/music/Dima+Bilan","picture":"http://userserve-ak.last.fm/serve/252/37546937.jpg"} +{"number":"661","name":"Stromae","url":"http://www.last.fm/music/Stromae","picture":"http://userserve-ak.last.fm/serve/252/46338833.png"} +{"number":"662","name":"Tamta","url":"http://www.last.fm/music/Tamta","picture":"http://userserve-ak.last.fm/serve/252/50110069.png"} +{"number":"663","name":"Sakis Rouvas","url":"http://www.last.fm/music/Sakis+Rouvas","picture":"http://userserve-ak.last.fm/serve/252/50928899.png"} +{"number":"664","name":"Karolina","url":"http://www.last.fm/music/Karolina","picture":"http://userserve-ak.last.fm/serve/252/52773619.png"} +{"number":"665","name":"Giannis Ploutarxos","url":"http://www.last.fm/music/Giannis+Ploutarxos","picture":"http://userserve-ak.last.fm/serve/252/459005.jpg"} +{"number":"667","name":"Evridiki","url":"http://www.last.fm/music/Evridiki","picture":"http://userserve-ak.last.fm/serve/252/48092681.png"} +{"number":"668","name":"Mando","url":"http://www.last.fm/music/Mando","picture":"http://userserve-ak.last.fm/serve/252/47218609.png"} +{"number":"669","name":"Eric Saade","url":"http://www.last.fm/music/Eric+Saade","picture":"http://userserve-ak.last.fm/serve/252/61631947.png"} +{"number":"670","name":"Safura","url":"http://www.last.fm/music/Safura","picture":"http://userserve-ak.last.fm/serve/252/47307721.png"} +{"number":"671","name":"Alkistis Protopsalti","url":"http://www.last.fm/music/Alkistis+Protopsalti","picture":"http://userserve-ak.last.fm/serve/252/2982211.jpg"} +{"number":"672","name":"Denise Rosenthal","url":"http://www.last.fm/music/Denise+Rosenthal","picture":"http://userserve-ak.last.fm/serve/252/55452329.png"} +{"number":"673","name":"Christos Dantis","url":"http://www.last.fm/music/Christos+Dantis","picture":"http://userserve-ak.last.fm/serve/252/18888225.png"} +{"number":"674","name":"Darin","url":"http://www.last.fm/music/Darin","picture":"http://userserve-ak.last.fm/serve/252/27202053.png"} +{"number":"675","name":"Despoina Olympiou","url":"http://www.last.fm/music/Despoina+Olympiou","picture":"http://userserve-ak.last.fm/serve/252/318158.jpg"} +{"number":"676","name":"Yohanna","url":"http://www.last.fm/music/Yohanna","picture":"http://userserve-ak.last.fm/serve/252/36308011.png"} +{"number":"677","name":"Kostas Martakis","url":"http://www.last.fm/music/Kostas+Martakis","picture":"http://userserve-ak.last.fm/serve/252/53202135.png"} +{"number":"678","name":"Soulthrow","url":"http://www.last.fm/music/Soulthrow","picture":"http://userserve-ak.last.fm/serve/252/41885015.jpg"} +{"number":"679","name":"Glee Cast","url":"http://www.last.fm/music/Glee+Cast","picture":"http://userserve-ak.last.fm/serve/252/44433851.png"} +{"number":"680","name":"The Veronicas","url":"http://www.last.fm/music/The+Veronicas","picture":"http://userserve-ak.last.fm/serve/252/31471795.png"} +{"number":"681","name":"Demi Lovato","url":"http://www.last.fm/music/Demi+Lovato","picture":"http://userserve-ak.last.fm/serve/252/41305109.png"} +{"number":"682","name":"Lights","url":"http://www.last.fm/music/Lights","picture":"http://userserve-ak.last.fm/serve/252/40210269.png"} +{"number":"683","name":"John Mayer","url":"http://www.last.fm/music/John+Mayer","picture":"http://userserve-ak.last.fm/serve/252/385672.jpg"} +{"number":"684","name":"American Idiot Cast","url":"http://www.last.fm/music/American+Idiot+Cast","picture":"http://userserve-ak.last.fm/serve/252/53866529.jpg"} +{"number":"685","name":"Nick Jonas & The Administration","url":"http://www.last.fm/music/Nick%2BJonas%2B%2526%2BThe%2BAdministration","picture":"http://userserve-ak.last.fm/serve/252/49612995.png"} +{"number":"686","name":"Selena Gomez & the Scene","url":"http://www.last.fm/music/Selena%2BGomez%2B%2526%2Bthe%2BScene","picture":"http://userserve-ak.last.fm/serve/252/51703695.png"} +{"number":"687","name":"All Time Low","url":"http://www.last.fm/music/All+Time+Low","picture":"http://userserve-ak.last.fm/serve/252/5543227.jpg"} +{"number":"688","name":"Honor Society","url":"http://www.last.fm/music/Honor+Society","picture":"http://userserve-ak.last.fm/serve/252/33974249.png"} +{"number":"689","name":"Florence + the Machine","url":"http://www.last.fm/music/Florence%2B%252B%2Bthe%2BMachine","picture":"http://userserve-ak.last.fm/serve/252/32428857.png"} +{"number":"690","name":"Marina & the Diamonds","url":"http://www.last.fm/music/Marina%2B%2526%2Bthe%2BDiamonds","picture":"http://userserve-ak.last.fm/serve/252/55003233.png"} +{"number":"691","name":"Owl City","url":"http://www.last.fm/music/Owl+City","picture":"http://userserve-ak.last.fm/serve/252/41332441.png"} +{"number":"692","name":"Mitchel Musso","url":"http://www.last.fm/music/Mitchel+Musso","picture":"http://userserve-ak.last.fm/serve/252/40032879.png"} +{"number":"693","name":"Hey Monday","url":"http://www.last.fm/music/Hey+Monday","picture":"http://userserve-ak.last.fm/serve/252/29450373.png"} +{"number":"694","name":"Darren Criss","url":"http://www.last.fm/music/Darren+Criss","picture":"http://userserve-ak.last.fm/serve/252/58635635.png"} +{"number":"695","name":"Push Play","url":"http://www.last.fm/music/Push+Play","picture":"http://userserve-ak.last.fm/serve/252/20383095.jpg"} +{"number":"696","name":"The Pierces","url":"http://www.last.fm/music/The+Pierces","picture":"http://userserve-ak.last.fm/serve/252/4185252.jpg"} +{"number":"697","name":"Sia","url":"http://www.last.fm/music/Sia","picture":"http://userserve-ak.last.fm/serve/252/45945079.png"} +{"number":"698","name":"Duncan Sheik & Steven Sater","url":"http://www.last.fm/music/Duncan%2BSheik%2B%2526%2BSteven%2BSater","picture":"http://userserve-ak.last.fm/serve/252/60484503.png"} +{"number":"699","name":"Esmée Denters","url":"http://www.last.fm/music/Esm%C3%A9e+Denters","picture":"http://userserve-ak.last.fm/serve/252/29374099.png"} +{"number":"700","name":"Boyce Avenue","url":"http://www.last.fm/music/Boyce+Avenue","picture":"http://userserve-ak.last.fm/serve/252/36672787.jpg"} +{"number":"701","name":"Shakira","url":"http://www.last.fm/music/Shakira","picture":"http://userserve-ak.last.fm/serve/252/52116105.png"} +{"number":"702","name":"Ester Dean","url":"http://www.last.fm/music/Ester+Dean","picture":"http://userserve-ak.last.fm/serve/252/61470597.png"} +{"number":"703","name":"The xx","url":"http://www.last.fm/music/The+xx","picture":"http://userserve-ak.last.fm/serve/252/61553347.png"} +{"number":"704","name":"The Pretty Reckless","url":"http://www.last.fm/music/The+Pretty+Reckless","picture":"http://userserve-ak.last.fm/serve/252/51058255.png"} +{"number":"705","name":"Engenheiros do Hawaii","url":"http://www.last.fm/music/Engenheiros+do+Hawaii","picture":"http://userserve-ak.last.fm/serve/252/76501.jpg"} +{"number":"706","name":"AC/DC","url":"http://www.last.fm/music/AC%252FDC","picture":"http://userserve-ak.last.fm/serve/252/35683175.jpg"} +{"number":"707","name":"Metallica","url":"http://www.last.fm/music/Metallica","picture":"http://userserve-ak.last.fm/serve/252/7560709.jpg"} +{"number":"708","name":"Capital Inicial","url":"http://www.last.fm/music/Capital+Inicial","picture":"http://userserve-ak.last.fm/serve/252/4599198.jpg"} +{"number":"709","name":"The White Stripes","url":"http://www.last.fm/music/The+White+Stripes","picture":"http://userserve-ak.last.fm/serve/252/27279811.jpg"} +{"number":"710","name":"Black Label Society","url":"http://www.last.fm/music/Black+Label+Society","picture":"http://userserve-ak.last.fm/serve/252/28926377.jpg"} +{"number":"711","name":"Mark Knopfler","url":"http://www.last.fm/music/Mark+Knopfler","picture":"http://userserve-ak.last.fm/serve/252/267493.jpg"} +{"number":"712","name":"Dire Straits","url":"http://www.last.fm/music/Dire+Straits","picture":"http://userserve-ak.last.fm/serve/252/12048127.jpg"} +{"number":"713","name":"Gotthard","url":"http://www.last.fm/music/Gotthard","picture":"http://userserve-ak.last.fm/serve/252/22507223.jpg"} +{"number":"715","name":"R.E.M.","url":"http://www.last.fm/music/R.E.M.","picture":"http://userserve-ak.last.fm/serve/252/2227819.jpg"} +{"number":"716","name":"Kaiser Chiefs","url":"http://www.last.fm/music/Kaiser+Chiefs","picture":"http://userserve-ak.last.fm/serve/252/2681098.jpg"} +{"number":"717","name":"Electric Light Orchestra","url":"http://www.last.fm/music/Electric+Light+Orchestra","picture":"http://userserve-ak.last.fm/serve/252/58911519.png"} +{"number":"718","name":"Johnny Cash","url":"http://www.last.fm/music/Johnny+Cash","picture":"http://userserve-ak.last.fm/serve/252/167556.jpg"} +{"number":"719","name":"Orishas","url":"http://www.last.fm/music/Orishas","picture":"http://userserve-ak.last.fm/serve/252/366524.jpg"} +{"number":"720","name":"Social Distortion","url":"http://www.last.fm/music/Social+Distortion","picture":"http://userserve-ak.last.fm/serve/252/340965.jpg"} +{"number":"721","name":"Wim Mertens","url":"http://www.last.fm/music/Wim+Mertens","picture":"http://userserve-ak.last.fm/serve/252/35625237.png"} +{"number":"722","name":"ATB","url":"http://www.last.fm/music/ATB","picture":"http://userserve-ak.last.fm/serve/252/20334901.jpg"} +{"number":"723","name":"Buena Vista Social Club","url":"http://www.last.fm/music/Buena+Vista+Social+Club","picture":"http://userserve-ak.last.fm/serve/252/2591611.jpg"} +{"number":"724","name":"Ozzy Osbourne","url":"http://www.last.fm/music/Ozzy+Osbourne","picture":"http://userserve-ak.last.fm/serve/252/91580.jpg"} +{"number":"725","name":"Titãs","url":"http://www.last.fm/music/Tit%C3%A3s","picture":"http://userserve-ak.last.fm/serve/252/3974965.jpg"} +{"number":"726","name":"Apocalyptica","url":"http://www.last.fm/music/Apocalyptica","picture":"http://userserve-ak.last.fm/serve/252/3767394.jpg"} +{"number":"727","name":"Era","url":"http://www.last.fm/music/Era","picture":"http://userserve-ak.last.fm/serve/252/23477579.png"} +{"number":"728","name":"Eric Clapton","url":"http://www.last.fm/music/Eric+Clapton","picture":"http://userserve-ak.last.fm/serve/252/2366379.jpg"} +{"number":"729","name":"Sacred Spirit","url":"http://www.last.fm/music/Sacred+Spirit","picture":"http://userserve-ak.last.fm/serve/252/16872539.jpg"} +{"number":"730","name":"Eagles","url":"http://www.last.fm/music/Eagles","picture":"http://userserve-ak.last.fm/serve/252/3241683.jpg"} +{"number":"731","name":"1200 Micrograms","url":"http://www.last.fm/music/1200+Micrograms","picture":"http://userserve-ak.last.fm/serve/252/24923253.jpg"} +{"number":"732","name":"Alice Cooper","url":"http://www.last.fm/music/Alice+Cooper","picture":"http://userserve-ak.last.fm/serve/252/391580.jpg"} +{"number":"733","name":"John Lennon","url":"http://www.last.fm/music/John+Lennon","picture":"http://userserve-ak.last.fm/serve/252/4799.jpg"} +{"number":"734","name":"Bruce Springsteen","url":"http://www.last.fm/music/Bruce+Springsteen","picture":"http://userserve-ak.last.fm/serve/252/2406762.jpg"} +{"number":"735","name":"The Rolling Stones","url":"http://www.last.fm/music/The+Rolling+Stones","picture":"http://userserve-ak.last.fm/serve/252/321677.jpg"} +{"number":"736","name":"Eliades Ochoa","url":"http://www.last.fm/music/Eliades+Ochoa","picture":"http://userserve-ak.last.fm/serve/252/20344269.jpg"} +{"number":"737","name":"Trans-Siberian Orchestra","url":"http://www.last.fm/music/Trans-Siberian+Orchestra","picture":"http://userserve-ak.last.fm/serve/252/35775099.jpg"} +{"number":"738","name":"DJ Tiësto","url":"http://www.last.fm/music/+noredirect/DJ+Ti%C3%ABsto","picture":"http://userserve-ak.last.fm/serve/252/216937.jpg"} +{"number":"739","name":"Lulu Santos","url":"http://www.last.fm/music/Lulu+Santos","picture":"http://userserve-ak.last.fm/serve/252/50732577.png"} +{"number":"743","name":"Nadja","url":"http://www.last.fm/music/Nadja","picture":"http://userserve-ak.last.fm/serve/252/60671097.jpg"} +{"number":"744","name":"Autechre","url":"http://www.last.fm/music/Autechre","picture":"http://userserve-ak.last.fm/serve/252/7725479.jpg"} +{"number":"745","name":"Aidan Baker & thisquietarmy","url":"http://www.last.fm/music/Aidan%2BBaker%2B%2526%2Bthisquietarmy","picture":"http://userserve-ak.last.fm/serve/252/25599307.jpg"} +{"number":"746","name":"Salem","url":"http://www.last.fm/music/Salem","picture":"http://userserve-ak.last.fm/serve/252/51239985.png"} +{"number":"747","name":"Carbon Based Lifeforms","url":"http://www.last.fm/music/Carbon+Based+Lifeforms","picture":"http://userserve-ak.last.fm/serve/252/37190753.jpg"} +{"number":"748","name":"Boards of Canada","url":"http://www.last.fm/music/Boards+of+Canada","picture":"http://userserve-ak.last.fm/serve/252/37989495.png"} +{"number":"749","name":"Casino Versus Japan","url":"http://www.last.fm/music/Casino+Versus+Japan","picture":"http://userserve-ak.last.fm/serve/252/47234815.jpg"} +{"number":"750","name":"Bill Withers","url":"http://www.last.fm/music/Bill+Withers","picture":"http://userserve-ak.last.fm/serve/252/41351783.jpg"} +{"number":"751","name":"Jesu","url":"http://www.last.fm/music/Jesu","picture":"http://userserve-ak.last.fm/serve/252/2285101.jpg"} +{"number":"752","name":"Boris","url":"http://www.last.fm/music/Boris","picture":"http://userserve-ak.last.fm/serve/252/43934699.jpg"} +{"number":"753","name":"Secede","url":"http://www.last.fm/music/Secede","picture":"http://userserve-ak.last.fm/serve/252/8259947.jpg"} +{"number":"754","name":"Maximum the Hormone","url":"http://www.last.fm/music/Maximum+the+Hormone","picture":"http://userserve-ak.last.fm/serve/252/4850296.jpg"} +{"number":"755","name":"65daysofstatic","url":"http://www.last.fm/music/65daysofstatic","picture":"http://userserve-ak.last.fm/serve/252/46404681.jpg"} +{"number":"756","name":"Arovane","url":"http://www.last.fm/music/Arovane","picture":"http://userserve-ak.last.fm/serve/252/98377.jpg"} +{"number":"757","name":"Crystal Castles","url":"http://www.last.fm/music/Crystal+Castles","picture":"http://userserve-ak.last.fm/serve/252/55303001.png"} +{"number":"759","name":"#####","url":"http://www.last.fm/music/%2523%2523%2523%2523%2523","picture":"http://userserve-ak.last.fm/serve/252/6999205.jpg"} +{"number":"760","name":"LAKE R▲DIO","url":"http://www.last.fm/music/LAKE+R%E2%96%B2DIO","picture":"http://userserve-ak.last.fm/serve/252/55577293.jpg"} +{"number":"761","name":"Asian Dub Foundation","url":"http://www.last.fm/music/Asian+Dub+Foundation","picture":"http://userserve-ak.last.fm/serve/252/25468299.jpg"} +{"number":"762","name":"the Chemodan","url":"http://www.last.fm/music/the+Chemodan","picture":"http://userserve-ak.last.fm/serve/252/46122543.jpg"} +{"number":"763","name":"Mono","url":"http://www.last.fm/music/Mono","picture":"http://userserve-ak.last.fm/serve/252/3323896.jpg"} +{"number":"764","name":"16 Horsepower","url":"http://www.last.fm/music/16+Horsepower","picture":"http://userserve-ak.last.fm/serve/252/150264.jpg"} +{"number":"765","name":"Multiplex","url":"http://www.last.fm/music/Multiplex","picture":"http://userserve-ak.last.fm/serve/252/63363.jpg"} +{"number":"766","name":"Ben Frost","url":"http://www.last.fm/music/Ben+Frost","picture":"http://userserve-ak.last.fm/serve/252/611694.jpg"} +{"number":"767","name":"Gridlock","url":"http://www.last.fm/music/Gridlock","picture":"http://userserve-ak.last.fm/serve/252/768016.jpg"} +{"number":"768","name":"Enter Shikari","url":"http://www.last.fm/music/Enter+Shikari","picture":"http://userserve-ak.last.fm/serve/252/40834195.jpg"} +{"number":"769","name":"Isis","url":"http://www.last.fm/music/Isis","picture":"http://userserve-ak.last.fm/serve/252/48408283.jpg"} +{"number":"770","name":"HORSE the band","url":"http://www.last.fm/music/HORSE+the+band","picture":"http://userserve-ak.last.fm/serve/252/2699284.jpg"} +{"number":"771","name":"Lazzich","url":"http://www.last.fm/music/Lazzich","picture":"http://userserve-ak.last.fm/serve/252/389205.jpg"} +{"number":"772","name":"Proem","url":"http://www.last.fm/music/Proem","picture":"http://userserve-ak.last.fm/serve/252/231962.jpg"} +{"number":"773","name":"electronicmicroanimal","url":"http://www.last.fm/music/electronicmicroanimal","picture":"http://userserve-ak.last.fm/serve/252/31333409.jpg"} +{"number":"774","name":"Mooncake","url":"http://www.last.fm/music/Mooncake","picture":"http://userserve-ak.last.fm/serve/252/6010612.jpg"} +{"number":"776","name":"sokpb avabodha","url":"http://www.last.fm/music/sokpb+avabodha","picture":"http://userserve-ak.last.fm/serve/252/34563595.jpg"} +{"number":"777","name":"The Crystal Method","url":"http://www.last.fm/music/The+Crystal+Method","picture":"http://userserve-ak.last.fm/serve/252/26115391.jpg"} +{"number":"778","name":"Ochre","url":"http://www.last.fm/music/Ochre","picture":"http://userserve-ak.last.fm/serve/252/166324.jpg"} +{"number":"779","name":"Bring Me The Horizon","url":"http://www.last.fm/music/Bring+Me+The+Horizon","picture":"http://userserve-ak.last.fm/serve/252/51720179.jpg"} +{"number":"780","name":"Gimmik","url":"http://www.last.fm/music/Gimmik","picture":"http://userserve-ak.last.fm/serve/252/6698687.jpg"} +{"number":"781","name":"Fail Emotions","url":"http://www.last.fm/music/Fail+Emotions","picture":"http://userserve-ak.last.fm/serve/252/58696327.jpg"} +{"number":"782","name":"Solar Fields","url":"http://www.last.fm/music/Solar+Fields","picture":"http://userserve-ak.last.fm/serve/252/38876027.jpg"} +{"number":"783","name":"Digitonal","url":"http://www.last.fm/music/Digitonal","picture":"http://userserve-ak.last.fm/serve/252/65311.jpg"} +{"number":"784","name":"Jackson and His Computer Band","url":"http://www.last.fm/music/Jackson+and+His+Computer+Band","picture":"http://userserve-ak.last.fm/serve/252/30555.jpg"} +{"number":"785","name":"Fear Factory","url":"http://www.last.fm/music/Fear+Factory","picture":"http://userserve-ak.last.fm/serve/252/2970763.jpg"} +{"number":"786","name":"Mandy Moore","url":"http://www.last.fm/music/Mandy+Moore","picture":"http://userserve-ak.last.fm/serve/252/61721401.png"} +{"number":"787","name":"Paulina Rubio","url":"http://www.last.fm/music/Paulina+Rubio","picture":"http://userserve-ak.last.fm/serve/252/44311983.png"} +{"number":"788","name":"Fey","url":"http://www.last.fm/music/Fey","picture":"http://userserve-ak.last.fm/serve/252/11999337.jpg"} +{"number":"789","name":"Dannii Minogue","url":"http://www.last.fm/music/Dannii+Minogue","picture":"http://userserve-ak.last.fm/serve/252/61744951.png"} +{"number":"790","name":"Belinda","url":"http://www.last.fm/music/Belinda","picture":"http://userserve-ak.last.fm/serve/252/62111231.png"} +{"number":"791","name":"Marie Serneholt","url":"http://www.last.fm/music/Marie+Serneholt","picture":"http://userserve-ak.last.fm/serve/252/38583095.png"} +{"number":"793","name":"Hannah Montana","url":"http://www.last.fm/music/Hannah+Montana","picture":"http://userserve-ak.last.fm/serve/252/5892662.png"} +{"number":"794","name":"A*Teens","url":"http://www.last.fm/music/A%2ATeens","picture":"http://userserve-ak.last.fm/serve/252/46212953.png"} +{"number":"795","name":"Gloria Trevi","url":"http://www.last.fm/music/Gloria+Trevi","picture":"http://userserve-ak.last.fm/serve/252/60658809.png"} +{"number":"796","name":"Anna Carina","url":"http://www.last.fm/music/Anna+Carina","picture":"http://userserve-ak.last.fm/serve/252/109705.jpg"} +{"number":"797","name":"Emily Osment","url":"http://www.last.fm/music/Emily+Osment","picture":"http://userserve-ak.last.fm/serve/252/52100869.png"} +{"number":"798","name":"Geri Halliwell","url":"http://www.last.fm/music/Geri+Halliwell","picture":"http://userserve-ak.last.fm/serve/252/51629225.png"} +{"number":"799","name":"Junior Caldera feat Sophie Ellis Bextor","url":"http://www.last.fm/music/Junior+Caldera+feat+Sophie+Ellis+Bextor","picture":"http://userserve-ak.last.fm/serve/252/45164301.png"} +{"number":"800","name":"Atomic Kitten","url":"http://www.last.fm/music/Atomic+Kitten","picture":"http://userserve-ak.last.fm/serve/252/37632331.png"} +{"number":"801","name":"5ive","url":"http://www.last.fm/music/5ive","picture":"http://userserve-ak.last.fm/serve/252/64390.gif"} +{"number":"803","name":"Five","url":"http://www.last.fm/music/Five","picture":"http://userserve-ak.last.fm/serve/252/40209857.png"} +{"number":"804","name":"Darkest Hour","url":"http://www.last.fm/music/Darkest+Hour","picture":"http://userserve-ak.last.fm/serve/252/57313101.jpg"} +{"number":"805","name":"Parkway Drive","url":"http://www.last.fm/music/Parkway+Drive","picture":"http://userserve-ak.last.fm/serve/252/48529665.jpg"} +{"number":"806","name":"Soilwork","url":"http://www.last.fm/music/Soilwork","picture":"http://userserve-ak.last.fm/serve/252/2510536.jpg"} +{"number":"807","name":"Crematory","url":"http://www.last.fm/music/Crematory","picture":"http://userserve-ak.last.fm/serve/252/549509.jpg"} +{"number":"808","name":"Rise Against","url":"http://www.last.fm/music/Rise+Against","picture":"http://userserve-ak.last.fm/serve/252/8698619.jpg"} +{"number":"809","name":"The Black Dahlia Murder","url":"http://www.last.fm/music/The+Black+Dahlia+Murder","picture":"http://userserve-ak.last.fm/serve/252/48224017.png"} +{"number":"810","name":"All That Remains","url":"http://www.last.fm/music/All+That+Remains","picture":"http://userserve-ak.last.fm/serve/252/177714.jpg"} +{"number":"811","name":"Sonic Syndicate","url":"http://www.last.fm/music/Sonic+Syndicate","picture":"http://userserve-ak.last.fm/serve/252/4859016.jpg"} +{"number":"812","name":"Dark Tranquillity","url":"http://www.last.fm/music/Dark+Tranquillity","picture":"http://userserve-ak.last.fm/serve/252/19741.jpg"} +{"number":"813","name":"As I Lay Dying","url":"http://www.last.fm/music/As+I+Lay+Dying","picture":"http://userserve-ak.last.fm/serve/252/43950585.png"} +{"number":"814","name":"Caliban","url":"http://www.last.fm/music/Caliban","picture":"http://userserve-ak.last.fm/serve/252/2467804.jpg"} +{"number":"815","name":"Killswitch Engage","url":"http://www.last.fm/music/Killswitch+Engage","picture":"http://userserve-ak.last.fm/serve/252/108045.jpg"} +{"number":"816","name":"A Day to Remember","url":"http://www.last.fm/music/A+Day+to+Remember","picture":"http://userserve-ak.last.fm/serve/252/54498549.png"} +{"number":"817","name":"Corpus Christi","url":"http://www.last.fm/music/Corpus+Christi","picture":"http://userserve-ak.last.fm/serve/252/23912437.jpg"} +{"number":"818","name":"36 Crazyfists","url":"http://www.last.fm/music/36+Crazyfists","picture":"http://userserve-ak.last.fm/serve/252/22598.jpg"} +{"number":"819","name":"Avatar","url":"http://www.last.fm/music/Avatar","picture":"http://userserve-ak.last.fm/serve/252/37587989.jpg"} +{"number":"820","name":"At the Gates","url":"http://www.last.fm/music/At+the+Gates","picture":"http://userserve-ak.last.fm/serve/252/229553.gif"} +{"number":"821","name":"Amon Amarth","url":"http://www.last.fm/music/Amon+Amarth","picture":"http://userserve-ak.last.fm/serve/252/22183005.gif"} +{"number":"822","name":"Evading Downfall","url":"http://www.last.fm/music/Evading+Downfall","picture":"http://userserve-ak.last.fm/serve/252/29830359.jpg"} +{"number":"823","name":"August Burns Red","url":"http://www.last.fm/music/August+Burns+Red","picture":"http://userserve-ak.last.fm/serve/252/62176629.jpg"} +{"number":"824","name":"Ensiferum","url":"http://www.last.fm/music/Ensiferum","picture":"http://userserve-ak.last.fm/serve/252/267212.jpg"} +{"number":"825","name":"Whitechapel","url":"http://www.last.fm/music/Whitechapel","picture":"http://userserve-ak.last.fm/serve/252/43512781.jpg"} +{"number":"826","name":"Divine Heresy","url":"http://www.last.fm/music/Divine+Heresy","picture":"http://userserve-ak.last.fm/serve/252/23959579.jpg"} +{"number":"827","name":"Ignite","url":"http://www.last.fm/music/Ignite","picture":"http://userserve-ak.last.fm/serve/252/23103241.jpg"} +{"number":"828","name":"From a Second Story Window","url":"http://www.last.fm/music/From+a+Second+Story+Window","picture":"http://userserve-ak.last.fm/serve/252/6004816.jpg"} +{"number":"829","name":"It Prevails","url":"http://www.last.fm/music/It+Prevails","picture":"http://userserve-ak.last.fm/serve/252/56599537.png"} +{"number":"830","name":"Atreyu","url":"http://www.last.fm/music/Atreyu","picture":"http://userserve-ak.last.fm/serve/252/7415709.jpg"} +{"number":"831","name":"Emmure","url":"http://www.last.fm/music/Emmure","picture":"http://userserve-ak.last.fm/serve/252/30039941.jpg"} +{"number":"832","name":"I Killed the Prom Queen","url":"http://www.last.fm/music/I+Killed+the+Prom+Queen","picture":"http://userserve-ak.last.fm/serve/252/60008937.jpg"} +{"number":"833","name":"Suffokate","url":"http://www.last.fm/music/Suffokate","picture":"http://userserve-ak.last.fm/serve/252/52608565.jpg"} +{"number":"834","name":"Five Finger Death Punch","url":"http://www.last.fm/music/Five+Finger+Death+Punch","picture":"http://userserve-ak.last.fm/serve/252/25011321.jpg"} +{"number":"835","name":"Scar Symmetry","url":"http://www.last.fm/music/Scar+Symmetry","picture":"http://userserve-ak.last.fm/serve/252/33787733.jpg"} +{"number":"836","name":"Butterfly Coma","url":"http://www.last.fm/music/Butterfly+Coma","picture":"http://userserve-ak.last.fm/serve/252/24369885.jpg"} +{"number":"837","name":"Eluveitie","url":"http://www.last.fm/music/Eluveitie","picture":"http://userserve-ak.last.fm/serve/252/23089827.jpg"} +{"number":"838","name":"Debeli Precjednik","url":"http://www.last.fm/music/Debeli+Precjednik","picture":"http://userserve-ak.last.fm/serve/252/2880105.jpg"} +{"number":"839","name":"All Shall Perish","url":"http://www.last.fm/music/All+Shall+Perish","picture":"http://userserve-ak.last.fm/serve/252/53246221.jpg"} +{"number":"840","name":"Millencolin","url":"http://www.last.fm/music/Millencolin","picture":"http://userserve-ak.last.fm/serve/252/6008803.jpg"} +{"number":"841","name":"Slayer","url":"http://www.last.fm/music/Slayer","picture":"http://userserve-ak.last.fm/serve/252/10447745.jpg"} +{"number":"842","name":"Despised Icon","url":"http://www.last.fm/music/Despised+Icon","picture":"http://userserve-ak.last.fm/serve/252/38109119.jpg"} +{"number":"843","name":"Pantera","url":"http://www.last.fm/music/Pantera","picture":"http://userserve-ak.last.fm/serve/252/11838735.jpg"} +{"number":"844","name":"Pain Confessor","url":"http://www.last.fm/music/Pain+Confessor","picture":"http://userserve-ak.last.fm/serve/252/7726867.jpg"} +{"number":"845","name":"The Dillinger Escape Plan","url":"http://www.last.fm/music/The+Dillinger+Escape+Plan","picture":"http://userserve-ak.last.fm/serve/252/60807625.png"} +{"number":"846","name":"Lamb of God","url":"http://www.last.fm/music/Lamb+of+God","picture":"http://userserve-ak.last.fm/serve/252/39060809.jpg"} +{"number":"847","name":"The Mercury Arc","url":"http://www.last.fm/music/The+Mercury+Arc","picture":"http://userserve-ak.last.fm/serve/252/39053993.jpg"} +{"number":"848","name":"Ego Fall","url":"http://www.last.fm/music/Ego+Fall","picture":"http://userserve-ak.last.fm/serve/252/18048023.jpg"} +{"number":"849","name":"Feed Her To The Sharks","url":"http://www.last.fm/music/Feed+Her+To+The+Sharks","picture":"http://userserve-ak.last.fm/serve/252/21744695.jpg"} +{"number":"850","name":"Haste the Day","url":"http://www.last.fm/music/Haste+the+Day","picture":"http://userserve-ak.last.fm/serve/252/18495765.jpg"} +{"number":"851","name":"Manic Street Preachers","url":"http://www.last.fm/music/Manic+Street+Preachers","picture":"http://userserve-ak.last.fm/serve/252/2314375.jpg"} +{"number":"852","name":"The Stranglers","url":"http://www.last.fm/music/The+Stranglers","picture":"http://userserve-ak.last.fm/serve/252/13188829.jpg"} +{"number":"853","name":"Therapy?","url":"http://www.last.fm/music/Therapy%3F","picture":"http://userserve-ak.last.fm/serve/252/247865.jpg"} +{"number":"854","name":"Alice in Chains","url":"http://www.last.fm/music/Alice+in+Chains","picture":"http://userserve-ak.last.fm/serve/252/3373415.jpg"} +{"number":"855","name":"Jane's Addiction","url":"http://www.last.fm/music/Jane%27s+Addiction","picture":"http://userserve-ak.last.fm/serve/252/28726593.jpg"} +{"number":"856","name":"Soundgarden","url":"http://www.last.fm/music/Soundgarden","picture":"http://userserve-ak.last.fm/serve/252/4376762.jpg"} +{"number":"857","name":"The Clash","url":"http://www.last.fm/music/The+Clash","picture":"http://userserve-ak.last.fm/serve/252/36023147.jpg"} +{"number":"858","name":"PJ Harvey","url":"http://www.last.fm/music/PJ+Harvey","picture":"http://userserve-ak.last.fm/serve/252/59003457.jpg"} +{"number":"859","name":"Pixies","url":"http://www.last.fm/music/Pixies","picture":"http://userserve-ak.last.fm/serve/252/52526.jpg"} +{"number":"860","name":"Stone Temple Pilots","url":"http://www.last.fm/music/Stone+Temple+Pilots","picture":"http://userserve-ak.last.fm/serve/252/4872267.jpg"} +{"number":"861","name":"Killing Joke","url":"http://www.last.fm/music/Killing+Joke","picture":"http://userserve-ak.last.fm/serve/252/36042595.jpg"} +{"number":"862","name":"Primal Scream","url":"http://www.last.fm/music/Primal+Scream","picture":"http://userserve-ak.last.fm/serve/252/3845.jpg"} +{"number":"863","name":"The Velvet Underground","url":"http://www.last.fm/music/The+Velvet+Underground","picture":"http://userserve-ak.last.fm/serve/252/281077.jpg"} +{"number":"864","name":"Ministry","url":"http://www.last.fm/music/Ministry","picture":"http://userserve-ak.last.fm/serve/252/21791341.jpg"} +{"number":"865","name":"Republika","url":"http://www.last.fm/music/Republika","picture":"http://userserve-ak.last.fm/serve/252/12085709.jpg"} +{"number":"866","name":"Virgin Prunes","url":"http://www.last.fm/music/Virgin+Prunes","picture":"http://userserve-ak.last.fm/serve/252/80625.jpg"} +{"number":"867","name":"Nick Cave and the Bad Seeds","url":"http://www.last.fm/music/Nick+Cave+and+the+Bad+Seeds","picture":"http://userserve-ak.last.fm/serve/252/53360969.jpg"} +{"number":"868","name":"The Doors","url":"http://www.last.fm/music/The+Doors","picture":"http://userserve-ak.last.fm/serve/252/3871341.jpg"} +{"number":"869","name":"Afghan Whigs","url":"http://www.last.fm/music/Afghan+Whigs","picture":"http://userserve-ak.last.fm/serve/252/65473.jpg"} +{"number":"870","name":"Skeletal Family","url":"http://www.last.fm/music/Skeletal+Family","picture":"http://userserve-ak.last.fm/serve/252/43096733.jpg"} +{"number":"871","name":"New Model Army","url":"http://www.last.fm/music/New+Model+Army","picture":"http://userserve-ak.last.fm/serve/252/30786759.png"} +{"number":"872","name":"Closterkeller","url":"http://www.last.fm/music/Closterkeller","picture":"http://userserve-ak.last.fm/serve/252/47373491.png"} +{"number":"873","name":"Porno for Pyros","url":"http://www.last.fm/music/Porno+for+Pyros","picture":"http://userserve-ak.last.fm/serve/252/16820597.jpg"} +{"number":"874","name":"Roxy Music","url":"http://www.last.fm/music/Roxy+Music","picture":"http://userserve-ak.last.fm/serve/252/519416.jpg"} +{"number":"875","name":"Bauhaus","url":"http://www.last.fm/music/Bauhaus","picture":"http://userserve-ak.last.fm/serve/252/17796755.jpg"} +{"number":"876","name":"Revolting Cocks","url":"http://www.last.fm/music/Revolting+Cocks","picture":"http://userserve-ak.last.fm/serve/252/3056879.jpg"} +{"number":"877","name":"Alphaville","url":"http://www.last.fm/music/Alphaville","picture":"http://userserve-ak.last.fm/serve/252/519364.jpg"} +{"number":"878","name":"Wolfsheim","url":"http://www.last.fm/music/Wolfsheim","picture":"http://userserve-ak.last.fm/serve/252/2428.jpg"} +{"number":"879","name":"Can","url":"http://www.last.fm/music/Can","picture":"http://userserve-ak.last.fm/serve/252/55893641.png"} +{"number":"880","name":"Nitzer Ebb","url":"http://www.last.fm/music/Nitzer+Ebb","picture":"http://userserve-ak.last.fm/serve/252/13182349.jpg"} +{"number":"881","name":"Kyuss","url":"http://www.last.fm/music/Kyuss","picture":"http://userserve-ak.last.fm/serve/252/60258675.jpg"} +{"number":"882","name":"The Jesus Lizard","url":"http://www.last.fm/music/The+Jesus+Lizard","picture":"http://userserve-ak.last.fm/serve/252/10324043.jpg"} +{"number":"883","name":"Tiamat","url":"http://www.last.fm/music/Tiamat","picture":"http://userserve-ak.last.fm/serve/252/36047401.jpg"} +{"number":"884","name":"Brygada Kryzys","url":"http://www.last.fm/music/Brygada+Kryzys","picture":"http://userserve-ak.last.fm/serve/252/276221.jpg"} +{"number":"885","name":"Tracy Chapman","url":"http://www.last.fm/music/Tracy+Chapman","picture":"http://userserve-ak.last.fm/serve/252/24031497.jpg"} +{"number":"886","name":"Dragonette","url":"http://www.last.fm/music/Dragonette","picture":"http://userserve-ak.last.fm/serve/252/44153343.png"} +{"number":"887","name":"New Young Pony Club","url":"http://www.last.fm/music/New+Young+Pony+Club","picture":"http://userserve-ak.last.fm/serve/252/7262.png"} +{"number":"888","name":"Siobhan Donaghy","url":"http://www.last.fm/music/Siobhan+Donaghy","picture":"http://userserve-ak.last.fm/serve/252/41695421.png"} +{"number":"889","name":"Cyndi Lauper","url":"http://www.last.fm/music/Cyndi+Lauper","picture":"http://userserve-ak.last.fm/serve/252/6967243.jpg"} +{"number":"890","name":"Kelly Osbourne","url":"http://www.last.fm/music/Kelly+Osbourne","picture":"http://userserve-ak.last.fm/serve/252/110159.jpg"} +{"number":"891","name":"Jewel","url":"http://www.last.fm/music/Jewel","picture":"http://userserve-ak.last.fm/serve/252/40676239.png"} +{"number":"892","name":"Dixie Chicks","url":"http://www.last.fm/music/Dixie+Chicks","picture":"http://userserve-ak.last.fm/serve/252/198972.jpg"} +{"number":"893","name":"Kate Miller-Heidke","url":"http://www.last.fm/music/Kate+Miller-Heidke","picture":"http://userserve-ak.last.fm/serve/252/7232413.jpg"} +{"number":"894","name":"Sneaky Sound System","url":"http://www.last.fm/music/Sneaky+Sound+System","picture":"http://userserve-ak.last.fm/serve/252/39498.gif"} +{"number":"895","name":"Chew Lips","url":"http://www.last.fm/music/Chew+Lips","picture":"http://userserve-ak.last.fm/serve/252/36275117.png"} +{"number":"896","name":"Noisettes","url":"http://www.last.fm/music/Noisettes","picture":"http://userserve-ak.last.fm/serve/252/42098211.png"} +{"number":"897","name":"Shakespear's Sister","url":"http://www.last.fm/music/Shakespear%27s+Sister","picture":"http://userserve-ak.last.fm/serve/252/36478479.jpg"} +{"number":"898","name":"Tegan and Sara","url":"http://www.last.fm/music/Tegan+and+Sara","picture":"http://userserve-ak.last.fm/serve/252/36243721.png"} +{"number":"899","name":"Missy Higgins","url":"http://www.last.fm/music/Missy+Higgins","picture":"http://userserve-ak.last.fm/serve/252/23635043.jpg"} +{"number":"900","name":"Annie Lennox","url":"http://www.last.fm/music/Annie+Lennox","picture":"http://userserve-ak.last.fm/serve/252/2669950.jpg"} +{"number":"901","name":"VV Brown","url":"http://www.last.fm/music/VV+Brown","picture":"http://userserve-ak.last.fm/serve/252/52352633.png"} +{"number":"902","name":"The Cast Of Buffy The Vampire Slayer","url":"http://www.last.fm/music/The+Cast+Of+Buffy+The+Vampire+Slayer","picture":"http://userserve-ak.last.fm/serve/252/2820789.jpg"} +{"number":"903","name":"Amy Winehouse","url":"http://www.last.fm/music/Amy+Winehouse","picture":"http://userserve-ak.last.fm/serve/252/366300.jpg"} +{"number":"904","name":"Rogue Traders","url":"http://www.last.fm/music/Rogue+Traders","picture":"http://userserve-ak.last.fm/serve/252/3881876.png"} +{"number":"905","name":"Crowded House","url":"http://www.last.fm/music/Crowded+House","picture":"http://userserve-ak.last.fm/serve/252/7245347.jpg"} +{"number":"906","name":"Girls Aloud","url":"http://www.last.fm/music/Girls+Aloud","picture":"http://userserve-ak.last.fm/serve/252/46994061.png"} +{"number":"907","name":"Timbaland","url":"http://www.last.fm/music/Timbaland","picture":"http://userserve-ak.last.fm/serve/252/40255265.png"} +{"number":"908","name":"Vanessa Hudgens","url":"http://www.last.fm/music/Vanessa+Hudgens","picture":"http://userserve-ak.last.fm/serve/252/57822071.png"} +{"number":"909","name":"Delta Goodrem","url":"http://www.last.fm/music/Delta+Goodrem","picture":"http://userserve-ak.last.fm/serve/252/58609541.png"} +{"number":"910","name":"John Williams","url":"http://www.last.fm/music/John+Williams","picture":"http://userserve-ak.last.fm/serve/252/59954101.jpg"} +{"number":"911","name":"Rachel Stevens","url":"http://www.last.fm/music/Rachel+Stevens","picture":"http://userserve-ak.last.fm/serve/252/40015573.png"} +{"number":"912","name":"Carter Burwell","url":"http://www.last.fm/music/Carter+Burwell","picture":"http://userserve-ak.last.fm/serve/252/38916203.png"} +{"number":"913","name":"Destiny's Child","url":"http://www.last.fm/music/Destiny%27s+Child","picture":"http://userserve-ak.last.fm/serve/252/43232019.png"} +{"number":"914","name":"Medina","url":"http://www.last.fm/music/Medina","picture":"http://userserve-ak.last.fm/serve/252/60964027.png"} +{"number":"915","name":"Alexandra Burke","url":"http://www.last.fm/music/Alexandra+Burke","picture":"http://userserve-ak.last.fm/serve/252/54753767.png"} +{"number":"916","name":"Vanessa S.","url":"http://www.last.fm/music/Vanessa+S.","picture":"http://userserve-ak.last.fm/serve/252/22231925.jpg"} +{"number":"917","name":"Iron Maiden","url":"http://www.last.fm/music/Iron+Maiden","picture":"http://userserve-ak.last.fm/serve/252/22493569.jpg"} +{"number":"918","name":"Megadeth","url":"http://www.last.fm/music/Megadeth","picture":"http://userserve-ak.last.fm/serve/252/8129787.jpg"} +{"number":"919","name":"Savatage","url":"http://www.last.fm/music/Savatage","picture":"http://userserve-ak.last.fm/serve/252/19508887.jpg"} +{"number":"920","name":"Dream Theater","url":"http://www.last.fm/music/Dream+Theater","picture":"http://userserve-ak.last.fm/serve/252/27653715.jpg"} +{"number":"921","name":"Circle II Circle","url":"http://www.last.fm/music/Circle+II+Circle","picture":"http://userserve-ak.last.fm/serve/252/33247703.jpg"} +{"number":"922","name":"Bruce Dickinson","url":"http://www.last.fm/music/Bruce+Dickinson","picture":"http://userserve-ak.last.fm/serve/252/36353539.png"} +{"number":"923","name":"Children of Bodom","url":"http://www.last.fm/music/Children+of+Bodom","picture":"http://userserve-ak.last.fm/serve/252/46620709.jpg"} +{"number":"924","name":"Angra","url":"http://www.last.fm/music/Angra","picture":"http://userserve-ak.last.fm/serve/252/49010785.jpg"} +{"number":"925","name":"Andre Matos","url":"http://www.last.fm/music/Andre+Matos","picture":"http://userserve-ak.last.fm/serve/252/34054139.jpg"} +{"number":"926","name":"Shaaman","url":"http://www.last.fm/music/Shaaman","picture":"http://userserve-ak.last.fm/serve/252/440.jpg"} +{"number":"927","name":"Hangar","url":"http://www.last.fm/music/Hangar","picture":"http://userserve-ak.last.fm/serve/252/40532411.jpg"} +{"number":"928","name":"Helloween","url":"http://www.last.fm/music/Helloween","picture":"http://userserve-ak.last.fm/serve/252/53137767.jpg"} +{"number":"929","name":"Viper","url":"http://www.last.fm/music/Viper","picture":"http://userserve-ak.last.fm/serve/252/583408.jpg"} +{"number":"930","name":"Nightwish","url":"http://www.last.fm/music/Nightwish","picture":"http://userserve-ak.last.fm/serve/252/2124668.jpg"} +{"number":"931","name":"Dr. Sin","url":"http://www.last.fm/music/Dr.+Sin","picture":"http://userserve-ak.last.fm/serve/252/225477.jpg"} +{"number":"932","name":"Matanza","url":"http://www.last.fm/music/Matanza","picture":"http://userserve-ak.last.fm/serve/252/366138.jpg"} +{"number":"933","name":"Wizards","url":"http://www.last.fm/music/Wizards","picture":"http://userserve-ak.last.fm/serve/252/53879027.jpg"} +{"number":"934","name":"Angel Dust","url":"http://www.last.fm/music/Angel+Dust","picture":"http://userserve-ak.last.fm/serve/252/25118605.jpg"} +{"number":"935","name":"Bittencourt Project","url":"http://www.last.fm/music/Bittencourt+Project","picture":"http://userserve-ak.last.fm/serve/252/34040043.jpg"} +{"number":"936","name":"Kamelot","url":"http://www.last.fm/music/Kamelot","picture":"http://userserve-ak.last.fm/serve/252/51238115.jpg"} +{"number":"937","name":"Stratovarius","url":"http://www.last.fm/music/Stratovarius","picture":"http://userserve-ak.last.fm/serve/252/25469173.jpg"} +{"number":"938","name":"Hibria","url":"http://www.last.fm/music/Hibria","picture":"http://userserve-ak.last.fm/serve/252/40665473.png"} +{"number":"939","name":"Mindflow","url":"http://www.last.fm/music/Mindflow","picture":"http://userserve-ak.last.fm/serve/252/36414821.jpg"} +{"number":"940","name":"Rage","url":"http://www.last.fm/music/Rage","picture":"http://userserve-ak.last.fm/serve/252/41607933.jpg"} +{"number":"941","name":"Tribuzy","url":"http://www.last.fm/music/Tribuzy","picture":"http://userserve-ak.last.fm/serve/252/2174373.jpg"} +{"number":"942","name":"Iced Earth","url":"http://www.last.fm/music/Iced+Earth","picture":"http://userserve-ak.last.fm/serve/252/26560821.jpg"} +{"number":"943","name":"Nevermore","url":"http://www.last.fm/music/Nevermore","picture":"http://userserve-ak.last.fm/serve/252/14266543.jpg"} +{"number":"944","name":"Symphony X","url":"http://www.last.fm/music/Symphony+X","picture":"http://userserve-ak.last.fm/serve/252/25730381.jpg"} +{"number":"945","name":"Journey","url":"http://www.last.fm/music/Journey","picture":"http://userserve-ak.last.fm/serve/252/274220.jpg"} +{"number":"946","name":"Avantasia","url":"http://www.last.fm/music/Avantasia","picture":"http://userserve-ak.last.fm/serve/252/16471533.jpg"} +{"number":"947","name":"Testament","url":"http://www.last.fm/music/Testament","picture":"http://userserve-ak.last.fm/serve/252/6955771.jpg"} +{"number":"948","name":"Burning in Hell","url":"http://www.last.fm/music/Burning+in+Hell","picture":"http://userserve-ak.last.fm/serve/252/206427.jpg"} +{"number":"949","name":"Death","url":"http://www.last.fm/music/Death","picture":"http://userserve-ak.last.fm/serve/252/97530.jpg"} +{"number":"950","name":"Kiko Loureiro","url":"http://www.last.fm/music/Kiko+Loureiro","picture":"http://userserve-ak.last.fm/serve/252/324256.jpg"} +{"number":"951","name":"Bon Jovi","url":"http://www.last.fm/music/Bon+Jovi","picture":"http://userserve-ak.last.fm/serve/252/38024753.png"} +{"number":"952","name":"Skid Row","url":"http://www.last.fm/music/Skid+Row","picture":"http://userserve-ak.last.fm/serve/252/32229.jpg"} +{"number":"953","name":"Arch Enemy","url":"http://www.last.fm/music/Arch+Enemy","picture":"http://userserve-ak.last.fm/serve/252/39490073.jpg"} +{"number":"954","name":"Kreator","url":"http://www.last.fm/music/Kreator","picture":"http://userserve-ak.last.fm/serve/252/2149388.jpg"} +{"number":"955","name":"Epica","url":"http://www.last.fm/music/Epica","picture":"http://userserve-ak.last.fm/serve/252/57651417.png"} +{"number":"956","name":"Yngwie Malmsteen","url":"http://www.last.fm/music/Yngwie+Malmsteen","picture":"http://userserve-ak.last.fm/serve/252/32807035.jpg"} +{"number":"957","name":"Blind Guardian","url":"http://www.last.fm/music/Blind+Guardian","picture":"http://userserve-ak.last.fm/serve/252/6564.jpg"} +{"number":"958","name":"Kotipelto","url":"http://www.last.fm/music/Kotipelto","picture":"http://userserve-ak.last.fm/serve/252/13091441.jpg"} +{"number":"959","name":"Queen","url":"http://www.last.fm/music/Queen","picture":"http://userserve-ak.last.fm/serve/252/45625545.png"} +{"number":"960","name":"Outworld","url":"http://www.last.fm/music/Outworld","picture":"http://userserve-ak.last.fm/serve/252/341517.jpg"} +{"number":"961","name":"Tori Amos","url":"http://www.last.fm/music/Tori+Amos","picture":"http://userserve-ak.last.fm/serve/252/55672671.png"} +{"number":"962","name":"The Cardigans","url":"http://www.last.fm/music/The+Cardigans","picture":"http://userserve-ak.last.fm/serve/252/3576712.jpg"} +{"number":"963","name":"Yanni","url":"http://www.last.fm/music/Yanni","picture":"http://userserve-ak.last.fm/serve/252/4528950.jpg"} +{"number":"964","name":"The Mars Volta","url":"http://www.last.fm/music/The+Mars+Volta","picture":"http://userserve-ak.last.fm/serve/252/13548815.jpg"} +{"number":"965","name":"Bush","url":"http://www.last.fm/music/Bush","picture":"http://userserve-ak.last.fm/serve/252/345014.jpg"} +{"number":"966","name":"Nickelback","url":"http://www.last.fm/music/Nickelback","picture":"http://userserve-ak.last.fm/serve/252/26385063.jpg"} +{"number":"967","name":"Sixpence None the Richer","url":"http://www.last.fm/music/Sixpence+None+the+Richer","picture":"http://userserve-ak.last.fm/serve/252/565723.jpg"} +{"number":"968","name":"Armin van Buuren","url":"http://www.last.fm/music/Armin+van+Buuren","picture":"http://userserve-ak.last.fm/serve/252/8307915.jpg"} +{"number":"969","name":"Kasabian","url":"http://www.last.fm/music/Kasabian","picture":"http://userserve-ak.last.fm/serve/252/284053.jpg"} +{"number":"970","name":"Infernal","url":"http://www.last.fm/music/Infernal","picture":"http://userserve-ak.last.fm/serve/252/10421637.jpg"} +{"number":"972","name":"t.A.T.u.","url":"http://www.last.fm/music/t.A.T.u.","picture":"http://userserve-ak.last.fm/serve/252/161875.jpg"} +{"number":"973","name":"Hevia","url":"http://www.last.fm/music/Hevia","picture":"http://userserve-ak.last.fm/serve/252/235954.jpg"} +{"number":"974","name":"Amy Studt","url":"http://www.last.fm/music/Amy+Studt","picture":"http://userserve-ak.last.fm/serve/252/54537561.jpg"} +{"number":"975","name":"Santana","url":"http://www.last.fm/music/Santana","picture":"http://userserve-ak.last.fm/serve/252/2224987.jpg"} +{"number":"976","name":"Agressiva 69","url":"http://www.last.fm/music/Agressiva+69","picture":"http://userserve-ak.last.fm/serve/252/46025403.png"} +{"number":"977","name":"At the Drive-In","url":"http://www.last.fm/music/At+the+Drive-In","picture":"http://userserve-ak.last.fm/serve/252/172563.jpg"} +{"number":"978","name":"Disturbed","url":"http://www.last.fm/music/Disturbed","picture":"http://userserve-ak.last.fm/serve/252/109507.jpg"} +{"number":"979","name":"Smash Mouth","url":"http://www.last.fm/music/Smash+Mouth","picture":"http://userserve-ak.last.fm/serve/252/82063.jpg"} +{"number":"980","name":"Yelle","url":"http://www.last.fm/music/Yelle","picture":"http://userserve-ak.last.fm/serve/252/58186561.png"} +{"number":"982","name":"Foo Fighters","url":"http://www.last.fm/music/Foo+Fighters","picture":"http://userserve-ak.last.fm/serve/252/59495563.jpg"} +{"number":"983","name":"Rodrigo y Gabriela","url":"http://www.last.fm/music/Rodrigo+y+Gabriela","picture":"http://userserve-ak.last.fm/serve/252/4456675.jpg"} +{"number":"984","name":"Tilly and the Wall","url":"http://www.last.fm/music/Tilly+and+the+Wall","picture":"http://userserve-ak.last.fm/serve/252/231071.jpg"} +{"number":"985","name":"Blondie","url":"http://www.last.fm/music/Blondie","picture":"http://userserve-ak.last.fm/serve/252/7553081.jpg"} +{"number":"986","name":"Frank Sinatra","url":"http://www.last.fm/music/Frank+Sinatra","picture":"http://userserve-ak.last.fm/serve/252/39168093.png"} +{"number":"987","name":"Earth, Wind & Fire","url":"http://www.last.fm/music/Earth%252C%2BWind%2B%2526%2BFire","picture":"http://userserve-ak.last.fm/serve/252/52210277.png"} +{"number":"988","name":"Fallulah","url":"http://www.last.fm/music/Fallulah","picture":"http://userserve-ak.last.fm/serve/252/34819757.jpg"} +{"number":"989","name":"The Donnas","url":"http://www.last.fm/music/The+Donnas","picture":"http://userserve-ak.last.fm/serve/252/53519297.png"} +{"number":"990","name":"Iron Butterfly","url":"http://www.last.fm/music/Iron+Butterfly","picture":"http://userserve-ak.last.fm/serve/252/18775749.jpg"} +{"number":"991","name":"Jem","url":"http://www.last.fm/music/Jem","picture":"http://userserve-ak.last.fm/serve/252/58101529.png"} +{"number":"992","name":"Chris Rea","url":"http://www.last.fm/music/Chris+Rea","picture":"http://userserve-ak.last.fm/serve/252/217703.jpg"} +{"number":"993","name":"Simple Minds","url":"http://www.last.fm/music/Simple+Minds","picture":"http://userserve-ak.last.fm/serve/252/28670399.png"} +{"number":"994","name":"Howard Jones","url":"http://www.last.fm/music/Howard+Jones","picture":"http://userserve-ak.last.fm/serve/252/4509147.jpg"} +{"number":"995","name":"China Crisis","url":"http://www.last.fm/music/China+Crisis","picture":"http://userserve-ak.last.fm/serve/252/13224571.jpg"} +{"number":"996","name":"Mike & The Mechanics","url":"http://www.last.fm/music/Mike%2B%2526%2BThe%2BMechanics","picture":"http://userserve-ak.last.fm/serve/252/57142699.png"} +{"number":"997","name":"The Human League","url":"http://www.last.fm/music/The+Human+League","picture":"http://userserve-ak.last.fm/serve/252/51970289.png"} +{"number":"998","name":"Orchestral Manoeuvres in the Dark","url":"http://www.last.fm/music/Orchestral+Manoeuvres+in+the+Dark","picture":"http://userserve-ak.last.fm/serve/252/33943699.png"} +{"number":"999","name":"Thompson Twins","url":"http://www.last.fm/music/Thompson+Twins","picture":"http://userserve-ak.last.fm/serve/252/6943589.jpg"} +{"number":"1000","name":"Information Society","url":"http://www.last.fm/music/Information+Society","picture":"http://userserve-ak.last.fm/serve/252/20361343.jpg"} +{"number":"1001","name":"Pet Shop Boys","url":"http://www.last.fm/music/Pet+Shop+Boys","picture":"http://userserve-ak.last.fm/serve/252/28023467.png"} diff --git a/src/test/resources/org/elasticsearch/hadoop/serialization/parser-test-nested.json b/src/test/resources/org/elasticsearch/hadoop/serialization/parser-test-nested.json new file mode 100644 index 000000000..667eae261 --- /dev/null +++ b/src/test/resources/org/elasticsearch/hadoop/serialization/parser-test-nested.json @@ -0,0 +1,35 @@ +{ + "firstName": "John", + "lastName": "Smith", + "age": 25, + "trap": { + "bogus":true, + "state":"bogus", + "building": { + "hight":"bogus", + "floors": 0, + "flats": 0 + } + }, + "address": { + "streetAddress": "21 2nd Street", + "city": "New York", + "state": "NY", + "postalCode": 10021, + "building": { + "hight":"tall", + "floors": 10, + "flats": 40 + } + }, + "phoneNumbers": [ + { + "type": "home", + "number": "212 555-1234" + }, + { + "type": "fax", + "number": "646 555-4567" + } + ] +} \ No newline at end of file