From 654cea97bf4a269595e75b7b617c452ffcc0e5d4 Mon Sep 17 00:00:00 2001 From: e-strauss Date: Fri, 25 Oct 2024 10:28:34 +0200 Subject: [PATCH] [SYSTEMDS-3782] Bag-of-words Encoder for CP Closes #2130. --- .../estim/sample/SampleEstimatorFactory.java | 2 +- .../sysds/runtime/data/SparseBlockCSR.java | 13 +- .../sysds/runtime/transform/TfUtils.java | 2 +- .../transform/encode/ColumnEncoder.java | 30 +- .../encode/ColumnEncoderBagOfWords.java | 423 ++++ .../transform/encode/ColumnEncoderBin.java | 2 +- .../encode/ColumnEncoderComposite.java | 29 +- .../encode/ColumnEncoderDummycode.java | 23 +- .../encode/ColumnEncoderFeatureHash.java | 1 - .../encode/ColumnEncoderPassThrough.java | 16 +- .../transform/encode/ColumnEncoderRecode.java | 14 +- .../transform/encode/EncoderFactory.java | 27 +- .../transform/encode/MultiColumnEncoder.java | 298 ++- .../sysds/runtime/util/CollectionUtils.java | 9 + .../utils/stats/TransformStatistics.java | 25 +- .../apache/sysds/test/AutomatedTestBase.java | 10 +- .../TransformFrameEncodeBagOfWords.java | 299 +++ ...Digital_Music_Text_Head2k_With_RCD_Col.csv | 2000 +++++++++++++++++ .../TransformFrameEncodeBagOfWords.dml | 78 + 19 files changed, 3142 insertions(+), 159 deletions(-) create mode 100644 src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderBagOfWords.java create mode 100644 src/test/java/org/apache/sysds/test/functions/transform/TransformFrameEncodeBagOfWords.java create mode 100644 src/test/resources/datasets/amazonReview2023/Digital_Music_Text_Head2k_With_RCD_Col.csv create mode 100644 src/test/scripts/functions/transform/TransformFrameEncodeBagOfWords.dml diff --git a/src/main/java/org/apache/sysds/runtime/compress/estim/sample/SampleEstimatorFactory.java b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/SampleEstimatorFactory.java index f7ae6f9ffbe..9cccecd1cf7 100644 --- a/src/main/java/org/apache/sysds/runtime/compress/estim/sample/SampleEstimatorFactory.java +++ b/src/main/java/org/apache/sysds/runtime/compress/estim/sample/SampleEstimatorFactory.java @@ -108,7 +108,7 @@ private static int distinctCountWithHistogram(int numVals, int[] invHist, int[] } } - private static int[] getInvertedFrequencyHistogram(int[] frequencies) { + public static int[] getInvertedFrequencyHistogram(int[] frequencies) { try{ final int numVals = frequencies.length; diff --git a/src/main/java/org/apache/sysds/runtime/data/SparseBlockCSR.java b/src/main/java/org/apache/sysds/runtime/data/SparseBlockCSR.java index cfa1258d37a..a40c567dfb2 100644 --- a/src/main/java/org/apache/sysds/runtime/data/SparseBlockCSR.java +++ b/src/main/java/org/apache/sysds/runtime/data/SparseBlockCSR.java @@ -274,13 +274,16 @@ public void initSparse(int rlen, int nnz, DataInput in) */ public static long estimateSizeInMemory(long nrows, long ncols, double sparsity) { double lnnz = Math.max(INIT_CAPACITY, Math.ceil(sparsity*nrows*ncols)); - - //32B overhead per array, int arr in nrows, int/double arr in nnz + return estimateSizeInMemory(nrows, (long) lnnz); + } + + public static long estimateSizeInMemory(long nrows, long nnz) { + //32B overhead per array, int arr in nrows, int/double arr in nnz double size = 16 + 4 + 4; //object + int field + padding size += MemoryEstimates.intArrayCost(nrows+1); //ptr array (row pointers) - size += MemoryEstimates.intArrayCost((long) lnnz); //indexes array (column indexes) - size += MemoryEstimates.doubleArrayCost((long) lnnz);//values array (non-zero values) - + size += MemoryEstimates.intArrayCost(nnz); //indexes array (column indexes) + size += MemoryEstimates.doubleArrayCost(nnz);//values array (non-zero values) + //robustness for long overflows return (long) Math.min(size, Long.MAX_VALUE); } diff --git a/src/main/java/org/apache/sysds/runtime/transform/TfUtils.java b/src/main/java/org/apache/sysds/runtime/transform/TfUtils.java index b264004b612..67ee1776f1c 100644 --- a/src/main/java/org/apache/sysds/runtime/transform/TfUtils.java +++ b/src/main/java/org/apache/sysds/runtime/transform/TfUtils.java @@ -47,7 +47,7 @@ protected byte toID() { //transform methods public enum TfMethod { - IMPUTE, RECODE, HASH, BIN, DUMMYCODE, UDF, OMIT, WORD_EMBEDDING; + IMPUTE, RECODE, HASH, BIN, DUMMYCODE, UDF, OMIT, WORD_EMBEDDING, BAG_OF_WORDS; @Override public String toString() { return name().toLowerCase(); diff --git a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoder.java b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoder.java index 010736b653e..f10da3d9468 100644 --- a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoder.java +++ b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoder.java @@ -61,10 +61,12 @@ public abstract class ColumnEncoder implements Encoder, Comparable _sparseRowsWZeros = null; + protected int[] sparseRowPointerOffset = null; // offsets created by bag of words encoders (multiple nnz) protected long _estMetaSize = 0; protected int _estNumDistincts = 0; protected int _nBuildPartitions = 0; protected int _nApplyPartitions = 0; + protected long _avgEntrySize = 0; //Override in ColumnEncoderWordEmbedding public void initEmbeddings(MatrixBlock embeddings){ @@ -72,7 +74,7 @@ public void initEmbeddings(MatrixBlock embeddings){ } protected enum TransformType{ - BIN, RECODE, DUMMYCODE, FEATURE_HASH, PASS_THROUGH, UDF, WORD_EMBEDDING, N_A + BIN, RECODE, DUMMYCODE, FEATURE_HASH, PASS_THROUGH, UDF, WORD_EMBEDDING, BAG_OF_WORDS, N_A } protected ColumnEncoder(int colID) { @@ -115,6 +117,9 @@ public MatrixBlock apply(CacheBlock in, MatrixBlock out, int outputCol, int r case WORD_EMBEDDING: TransformStatistics.incWordEmbeddingApplyTime(t); break; + case BAG_OF_WORDS: + TransformStatistics.incBagOfWordsApplyTime(t); + break; case FEATURE_HASH: TransformStatistics.incFeatureHashingApplyTime(t); break; @@ -152,6 +157,7 @@ protected void applySparse(CacheBlock in, MatrixBlock out, int outputCol, int for(int i = rowStart; i < rowEnd; i+=B) { int lim = Math.min(i+B, rowEnd); for (int ii=i; ii in, MatrixBlock out, int outputCol, int // Manually fill the column-indexes and values array SparseBlockCSR csrblock = (SparseBlockCSR)out.getSparseBlock(); int rptr[] = csrblock.rowPointers(); - csrblock.indexes()[rptr[ii]+index] = outputCol; - csrblock.values()[rptr[ii]+index] = codes[ii-rowStart]; + csrblock.indexes()[rptr[ii]+indexWithOffset] = outputCol; + csrblock.values()[rptr[ii]+indexWithOffset] = codes[ii-rowStart]; } } } @@ -336,6 +342,11 @@ public int getEstNumDistincts() { return _estNumDistincts; } + public void computeMapSizeEstimate(CacheBlock in, int[] sampleIndices) { + throw new DMLRuntimeException(this + " does not need map size estimation"); + } + + @Override public int compareTo(ColumnEncoder o) { return Integer.compare(getEncoderType(this), getEncoderType(o)); @@ -355,9 +366,11 @@ public List> getBuildTasks(CacheBlock in) { tasks.add(getBuildTask(in)); } else { + if(this instanceof ColumnEncoderBagOfWords) + ((ColumnEncoderBagOfWords) this).initNnzPartials(in.getNumRows(), blockSizes.length); HashMap ret = new HashMap<>(); for(int startRow = 0, i = 0; i < blockSizes.length; startRow+=blockSizes[i], i++) - tasks.add(getPartialBuildTask(in, startRow, blockSizes[i], ret)); + tasks.add(getPartialBuildTask(in, startRow, blockSizes[i], ret, i)); tasks.add(getPartialMergeBuildTask(ret)); dep = new ArrayList<>(Collections.nCopies(tasks.size() - 1, null)); dep.add(tasks.subList(0, tasks.size() - 1)); @@ -370,7 +383,7 @@ public Callable getBuildTask(CacheBlock in) { } public Callable getPartialBuildTask(CacheBlock in, int startRow, - int blockSize, HashMap ret) { + int blockSize, HashMap ret, int p) { throw new DMLRuntimeException( "Trying to get the PartialBuild task of an Encoder which does not support partial building"); } @@ -381,11 +394,12 @@ public Callable getPartialMergeBuildTask(HashMap ret) { } - public List> getApplyTasks(CacheBlock in, MatrixBlock out, int outputCol) { + public List> getApplyTasks(CacheBlock in, MatrixBlock out, int outputCol, int[] sparseRowPointerOffsets) { List> tasks = new ArrayList<>(); List>> dep = null; + //for now single threaded apply for bag of words int[] blockSizes = getBlockSizes(in.getNumRows(), _nApplyPartitions); - + this.sparseRowPointerOffset = out.isInSparseFormat() ? sparseRowPointerOffsets : null; for(int startRow = 0, i = 0; i < blockSizes.length; startRow+=blockSizes[i], i++){ if(out.isInSparseFormat()) tasks.add(getSparseTask(in, out, outputCol, startRow, blockSizes[i])); @@ -419,7 +433,7 @@ public Set getSparseRowsWZeros(){ return null; } - protected void addSparseRowsWZeros(ArrayList sparseRowsWZeros){ + protected void addSparseRowsWZeros(List sparseRowsWZeros){ synchronized (this){ if(_sparseRowsWZeros == null) _sparseRowsWZeros = new ArrayList<>(); diff --git a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderBagOfWords.java b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderBagOfWords.java new file mode 100644 index 00000000000..c138901ad1c --- /dev/null +++ b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderBagOfWords.java @@ -0,0 +1,423 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.sysds.runtime.transform.encode; + +import org.apache.commons.lang3.NotImplementedException; +import org.apache.sysds.api.DMLScript; +import org.apache.sysds.runtime.compress.estim.sample.SampleEstimatorFactory; +import org.apache.sysds.runtime.controlprogram.caching.CacheBlock; +import org.apache.sysds.runtime.data.SparseBlock; +import org.apache.sysds.runtime.data.SparseBlockCSR; +import org.apache.sysds.runtime.frame.data.FrameBlock; +import org.apache.sysds.runtime.matrix.data.MatrixBlock; +import org.apache.sysds.utils.stats.TransformStatistics; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Callable; + +import static org.apache.sysds.runtime.transform.encode.ColumnEncoderRecode.constructRecodeMapEntry; +import static org.apache.sysds.runtime.util.UtilFunctions.getEndIndex; + +public class ColumnEncoderBagOfWords extends ColumnEncoder { + + public static int NUM_SAMPLES_MAP_ESTIMATION = 16000; + protected int[] nnzPerRow; + private Map tokenDictionary; + protected String seperatorRegex = "\\s+"; // whitespace + protected boolean caseSensitive = false; + protected long nnz = 0; + protected long[] nnzPartials; + protected int defaultNnzCapacity = 64; + protected double avgNnzPerRow = 1.0; + + protected ColumnEncoderBagOfWords(int colID) { + super(colID); + } + + public ColumnEncoderBagOfWords() { + super(-1); + } + + protected void initNnzPartials(int rows, int numBlocks){ + this.nnzPerRow = new int[rows]; + this.nnzPartials = new long[numBlocks]; + } + + public void computeMapSizeEstimate(CacheBlock in, int[] sampleIndices) { + // Find the frequencies of distinct values in the sample after tokenization + HashMap distinctFreq = new HashMap<>(); + long totSize = 0; + final int max_index = Math.min(ColumnEncoderBagOfWords.NUM_SAMPLES_MAP_ESTIMATION, sampleIndices.length/3); + int numTokensSample = 0; + int[] nnzPerRow = new int[max_index]; + for (int i = 0; i < max_index; i++) { + int sind = sampleIndices[i]; + String current = in.getString(sind, this._colID - 1); + Set tokenSetRow = new HashSet<>(); + if(current != null) + for(String token : tokenize(current, caseSensitive, seperatorRegex)) + if(!token.isEmpty()){ + tokenSetRow.add(token); + if (distinctFreq.containsKey(token)) + distinctFreq.put(token, distinctFreq.get(token) + 1); + else { + distinctFreq.put(token, 1); + // Maintain total size of the keys + totSize += (token.length() * 2L + 16); //sizeof(String) = len(chars) + header + } + numTokensSample++; + } + nnzPerRow[i] = tokenSetRow.size(); + } + Arrays.sort(nnzPerRow); + avgNnzPerRow = (double) Arrays.stream(nnzPerRow).sum() / nnzPerRow.length; + // default value for HashSets in build phase -> 75% without resize (Division by 0.9 -> is the resize threshold) + defaultNnzCapacity = (int) Math.max( nnzPerRow[(int) (nnzPerRow.length*0.75)] / 0.9, 64); + // we increase the upperbound of the total count estimate by 20% + double avgSentenceLength = numTokensSample*1.2 / max_index; + + + // Estimate total #distincts using Hass and Stokes estimator + int[] freq = distinctFreq.values().stream().mapToInt(v -> v).toArray(); + _estNumDistincts = SampleEstimatorFactory.distinctCount(freq, (int) (avgSentenceLength*in.getNumRows()), + numTokensSample, SampleEstimatorFactory.EstimationType.HassAndStokes); + + // Based on a small experimental evaluation: + // we increase the upperbound of the total count estimate by 2% + _estNumDistincts = (int) (_estNumDistincts* 1.2); + + // Compute total size estimates for each partial recode map + // We assume each partial map contains all distinct values and have the same size + long avgKeySize = totSize / distinctFreq.size(); + long valSize = 16L; //sizeof(Long) = 8 + header + _avgEntrySize = avgKeySize + valSize; + _estMetaSize = _estNumDistincts * _avgEntrySize; + } + + public static String[] tokenize(String current, boolean caseSensitive, String seperatorRegex) { + // string builder is faster than regex + StringBuilder finalString = new StringBuilder(); + for (char c : current.toCharArray()) { + if (Character.isLetter(c)) + finalString.append(caseSensitive ? c : Character.toLowerCase(c)); + else + finalString.append(' '); + } + return finalString.toString().split(seperatorRegex); + } + + @Override + public int getDomainSize(){ + return tokenDictionary.size(); + } + + @Override + protected double getCode(CacheBlock in, int row) { + throw new NotImplementedException(); + } + + @Override + protected double[] getCodeCol(CacheBlock in, int startInd, int rowEnd, double[] tmp) { + throw new NotImplementedException(); + } + + @Override + protected TransformType getTransformType() { + return TransformType.BAG_OF_WORDS; + } + + + + public Callable getBuildTask(CacheBlock in) { + return new ColumnBagOfWordsBuildTask(this, in); + } + + @Override + public void build(CacheBlock in) { + long t0 = DMLScript.STATISTICS ? System.nanoTime() : 0; + tokenDictionary = new HashMap<>(_estNumDistincts); + int i = 0; + this.nnz = 0; + nnzPerRow = new int[in.getNumRows()]; + HashSet tokenSetPerRow; + for (int r = 0; r < in.getNumRows(); r++) { + // start with a higher default capacity to avoid resizes + tokenSetPerRow = new HashSet<>(defaultNnzCapacity); + String current = in.getString(r, this._colID - 1); + if(current != null) + for(String token : tokenize(current, caseSensitive, seperatorRegex)) + if(!token.isEmpty()){ + tokenSetPerRow.add(token); + if(!this.tokenDictionary.containsKey(token)) + this.tokenDictionary.put(token, i++); + } + this.nnzPerRow[r] = tokenSetPerRow.size(); + this.nnz += tokenSetPerRow.size(); + } + if(DMLScript.STATISTICS) + TransformStatistics.incBagOfWordsBuildTime(System.nanoTime()-t0); + } + + @Override + public Callable getPartialBuildTask(CacheBlock in, int startRow, int blockSize, + HashMap ret, int pos) { + return new BowPartialBuildTask(in, _colID, startRow, blockSize, ret, nnzPerRow, caseSensitive, seperatorRegex, nnzPartials, pos); + } + + @Override + public Callable getPartialMergeBuildTask(HashMap ret) { + tokenDictionary = new HashMap<>(this._estNumDistincts); + return new BowMergePartialBuildTask(this, ret); + } + + // Pair class to hold key-value pairs (colId-tokenCount pairs) + static class Pair { + int key; + int value; + + Pair(int key, int value) { + this.key = key; + this.value = value; + } + } + + protected void applySparse(CacheBlock in, MatrixBlock out, int outputCol, int rowStart, int blk) { + boolean mcsr = MatrixBlock.DEFAULT_SPARSEBLOCK == SparseBlock.Type.MCSR; + mcsr = false; // force CSR for transformencode FIXME + List sparseRowsWZeros = new ArrayList<>(); + for(int r = rowStart; r < getEndIndex(in.getNumRows(), rowStart, blk); r++) { + if(mcsr) { + throw new NotImplementedException(); + } + else { // csr + HashMap counter = countTokenAppearances(in, r, _colID-1, caseSensitive, seperatorRegex); + if(counter.isEmpty()) + sparseRowsWZeros.add(r); + else { + SparseBlockCSR csrblock = (SparseBlockCSR) out.getSparseBlock(); + int[] rptr = csrblock.rowPointers(); + // assert that nnz from build is equal to nnz from apply + assert counter.size() == nnzPerRow[r]; + Pair[] columnValuePairs = new Pair[counter.size()]; + int i = 0; + for (Map.Entry entry : counter.entrySet()) { + String token = entry.getKey(); + columnValuePairs[i] = new Pair(outputCol + tokenDictionary.get(token), entry.getValue()); + i++; + } + // insertion sorts performs better on small arrays + if(columnValuePairs.length >= 128) + Arrays.sort(columnValuePairs, Comparator.comparingInt(pair -> pair.key)); + else + insertionSort(columnValuePairs); + // Manually fill the column-indexes and values array + for (i = 0; i < columnValuePairs.length; i++) { + int index = sparseRowPointerOffset != null ? sparseRowPointerOffset[r] - 1 + i : i; + index += rptr[r] + this._colID -1; + csrblock.indexes()[index] = columnValuePairs[i].key; + csrblock.values()[index] = columnValuePairs[i].value; + } + } + } + } + if(!sparseRowsWZeros.isEmpty()) { + addSparseRowsWZeros(sparseRowsWZeros); + } + } + + private static void insertionSort(Pair [] arr) { + for (int i = 1; i < arr.length; i++) { + Pair current = arr[i]; + int j = i - 1; + while (j >= 0 && arr[j].key > current.key) { + arr[j + 1] = arr[j]; + j--; + } + arr[j + 1] = current; + } + } + + @Override + protected void applyDense(CacheBlock in, MatrixBlock out, int outputCol, int rowStart, int blk){ + for (int r = rowStart; r < Math.max(in.getNumRows(), rowStart + blk); r++) { + HashMap counter = countTokenAppearances(in, r, _colID-1, caseSensitive, seperatorRegex); + for (Map.Entry entry : counter.entrySet()) + out.set(r, outputCol + tokenDictionary.get(entry.getKey()), entry.getValue()); + } + } + + private static HashMap countTokenAppearances( + CacheBlock in, int r, int c, boolean caseSensitive, String separator) + { + String current = in.getString(r, c); + HashMap counter = new HashMap<>(); + if(current != null) + for (String token : tokenize(current, caseSensitive, separator)) + if (!token.isEmpty()) + counter.put(token, counter.getOrDefault(token, 0) + 1); + return counter; + } + + @Override + public void allocateMetaData(FrameBlock meta) { + meta.ensureAllocatedColumns(this.getDomainSize()); + } + + @Override + public FrameBlock getMetaData(FrameBlock out) { + int rowID = 0; + StringBuilder sb = new StringBuilder(); + for(Map.Entry e : this.tokenDictionary.entrySet()) { + out.set(rowID++, _colID - 1, constructRecodeMapEntry(e.getKey(), Long.valueOf(e.getValue()), sb)); + } + return out; + } + + @Override + public void initMetaData(FrameBlock meta) { + throw new NotImplementedException(); + } + + private static class BowPartialBuildTask implements Callable { + + private final CacheBlock _input; + private final int _blockSize; + private final int _startRow; + private final int _colID; + private final boolean _caseSensitive; + private final String _seperator; + private final HashMap _partialMaps; + private final int[] _nnzPerRow; + private final long[] _nnzPartials; + private final int _pos; + + protected BowPartialBuildTask(CacheBlock input, int colID, int startRow, + int blocksize, HashMap partialMaps, int[] nnzPerRow, + boolean caseSensitive, String seperator, long[] nnzPartials, int pos) + { + _input = input; + _blockSize = blocksize; + _colID = colID; + _startRow = startRow; + _partialMaps = partialMaps; + _caseSensitive = caseSensitive; + _seperator = seperator; + _nnzPerRow = nnzPerRow; + _nnzPartials = nnzPartials; + _pos = pos; + } + + @Override + public Object call(){ + long t0 = DMLScript.STATISTICS ? System.nanoTime() : 0; + int endRow = getEndIndex(_input.getNumRows(), _startRow, _blockSize); + HashSet tokenSetPartial = new HashSet<>(); + HashSet tokenSetPerRow; + long nnzPartial = 0; + for (int r = _startRow; r < endRow; r++) { + tokenSetPerRow = new HashSet<>(64); + String current = _input.getString(r, this._colID - 1); + if(current != null) + for(String token : tokenize(current, _caseSensitive, _seperator)) + if(!token.isEmpty()){ + tokenSetPerRow.add(token); + tokenSetPartial.add(token); + } + _nnzPerRow[r] = tokenSetPerRow.size(); + nnzPartial += tokenSetPerRow.size(); + } + _nnzPartials[_pos] = nnzPartial; + synchronized (_partialMaps){ + _partialMaps.put(_startRow, tokenSetPartial); + } + if(DMLScript.STATISTICS){ + TransformStatistics.incBagOfWordsBuildTime(System.nanoTime() - t0); + } + return null; + } + + @Override + public String toString() { + return getClass().getSimpleName() + ""; + } + } + + private static class BowMergePartialBuildTask implements Callable { + private final HashMap _partialMaps; + private final ColumnEncoderBagOfWords _encoder; + + private BowMergePartialBuildTask(ColumnEncoderBagOfWords encoderRecode, HashMap partialMaps) { + _partialMaps = partialMaps; + _encoder = encoderRecode; + } + + @Override + public Object call() { + long t0 = DMLScript.STATISTICS ? System.nanoTime() : 0; + Map tokenDictionary = _encoder.tokenDictionary; + for(Object tokenSet : _partialMaps.values()){ + ( (HashSet) tokenSet).forEach(token -> { + if(!tokenDictionary.containsKey(token)) + tokenDictionary.put((String) token, tokenDictionary.size()); + }); + } + for (long nnzPartial : _encoder.nnzPartials) + _encoder.nnz += nnzPartial; + if(DMLScript.STATISTICS){ + TransformStatistics.incBagOfWordsBuildTime(System.nanoTime() - t0); + } + return null; + } + + @Override + public String toString() { + return getClass().getSimpleName() + ""; + } + } + + private static class ColumnBagOfWordsBuildTask implements Callable { + + private final ColumnEncoderBagOfWords _encoder; + private final CacheBlock _input; + + protected ColumnBagOfWordsBuildTask(ColumnEncoderBagOfWords encoder, CacheBlock input) { + _encoder = encoder; + _input = input; + } + + @Override + public Void call() { + _encoder.build(_input); + return null; + } + + @Override + public String toString() { + return getClass().getSimpleName() + ""; + } + } +} diff --git a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderBin.java b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderBin.java index 8e1055e41d2..74b4737194c 100644 --- a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderBin.java +++ b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderBin.java @@ -321,7 +321,7 @@ public Callable getBuildTask(CacheBlock in) { @Override public Callable getPartialBuildTask(CacheBlock in, int startRow, int blockSize, - HashMap ret) { + HashMap ret, int p) { return new BinPartialBuildTask(in, _colID, startRow, blockSize, _binMethod, ret); } diff --git a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderComposite.java b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderComposite.java index 6fda66113dd..95443729146 100644 --- a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderComposite.java +++ b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderComposite.java @@ -51,6 +51,7 @@ public class ColumnEncoderComposite extends ColumnEncoder { private List _columnEncoders = null; private FrameBlock _meta = null; + private long avgEntrySize = 0L; // map to keep track of which encoder has how many build tasks //private Map _partialBuildTaskMap; @@ -120,17 +121,17 @@ public void build(CacheBlock in, Map equiHeightMaxs) { } @Override - public List> getApplyTasks(CacheBlock in, MatrixBlock out, int outputCol) { + public List> getApplyTasks(CacheBlock in, MatrixBlock out, int outputCol, int[] sparseRowPointerOffsets) { List> tasks = new ArrayList<>(); List sizes = new ArrayList<>(); for(int i = 0; i < _columnEncoders.size(); i++) { List> t; if(i == 0) { // 1. encoder writes data into MatrixBlock Column all others use this column for further encoding - t = _columnEncoders.get(i).getApplyTasks(in, out, outputCol); + t = _columnEncoders.get(i).getApplyTasks(in, out, outputCol, sparseRowPointerOffsets); } else { - t = _columnEncoders.get(i).getApplyTasks(out, out, outputCol); + t = _columnEncoders.get(i).getApplyTasks(out, out, outputCol, sparseRowPointerOffsets); } if(t == null) continue; @@ -373,23 +374,31 @@ public boolean hasEncoder(Class type) { public boolean hasBuild() { for (ColumnEncoder e : _columnEncoders) - if (e.getClass().equals(ColumnEncoderRecode.class) - || e.getClass().equals(ColumnEncoderDummycode.class) - || e.getClass().equals(ColumnEncoderBin.class)) + if (e instanceof ColumnEncoderRecode + || e instanceof ColumnEncoderDummycode + || e instanceof ColumnEncoderBin + || e instanceof ColumnEncoderBagOfWords) return true; return false; } - public void computeRCDMapSizeEstimate(CacheBlock in, int[] sampleIndices) { + public void computeMapSizeEstimate(CacheBlock in, int[] sampleIndices) { int estNumDist = 0; - for (ColumnEncoder e : _columnEncoders) - if (e.getClass().equals(ColumnEncoderRecode.class)) { - ((ColumnEncoderRecode) e).computeRCDMapSizeEstimate(in, sampleIndices); + for (ColumnEncoder e : _columnEncoders){ + if (e.getClass().equals(ColumnEncoderRecode.class) || e.getClass().equals(ColumnEncoderBagOfWords.class)) { + e.computeMapSizeEstimate(in, sampleIndices); estNumDist = e.getEstNumDistincts(); + this.avgEntrySize = e._avgEntrySize; } + } long totEstSize = _columnEncoders.stream().mapToLong(ColumnEncoder::getEstMetaSize).sum(); setEstMetaSize(totEstSize); setEstNumDistincts(estNumDist); + + } + + public long getAvgEntrySize(){ + return this.avgEntrySize; } public void setNumPartitions(int nBuild, int nApply) { diff --git a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderDummycode.java b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderDummycode.java index 375d0be5264..fd6e3410bf1 100644 --- a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderDummycode.java +++ b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderDummycode.java @@ -120,34 +120,35 @@ protected void applySparse(CacheBlock in, MatrixBlock out, int outputCol, int ArrayList sparseRowsWZeros = null; int index = _colID - 1; for(int r = rowStart; r < getEndIndex(in.getNumRows(), rowStart, blk); r++) { + int indexWithOffset = sparseRowPointerOffset != null ? sparseRowPointerOffset[r] - 1 + index : index; if(mcsr) { - double val = out.getSparseBlock().get(r).values()[index]; + double val = out.getSparseBlock().get(r).values()[indexWithOffset]; if(Double.isNaN(val)) { if(sparseRowsWZeros == null) sparseRowsWZeros = new ArrayList<>(); sparseRowsWZeros.add(r); - out.getSparseBlock().get(r).values()[index] = 0; + out.getSparseBlock().get(r).values()[indexWithOffset] = 0; continue; } int nCol = outputCol + (int) val - 1; - out.getSparseBlock().get(r).indexes()[index] = nCol; - out.getSparseBlock().get(r).values()[index] = 1; + out.getSparseBlock().get(r).indexes()[indexWithOffset] = nCol; + out.getSparseBlock().get(r).values()[indexWithOffset] = 1; } else { // csr SparseBlockCSR csrblock = (SparseBlockCSR) out.getSparseBlock(); int rptr[] = csrblock.rowPointers(); - double val = csrblock.values()[rptr[r] + index]; + double val = csrblock.values()[rptr[r] + indexWithOffset]; if(Double.isNaN(val)) { if(sparseRowsWZeros == null) sparseRowsWZeros = new ArrayList<>(); sparseRowsWZeros.add(r); - csrblock.values()[rptr[r] + index] = 0; // test + csrblock.values()[rptr[r] + indexWithOffset] = 0; // test continue; } // Manually fill the column-indexes and values array int nCol = outputCol + (int) val - 1; - csrblock.indexes()[rptr[r] + index] = nCol; - csrblock.values()[rptr[r] + index] = 1; + csrblock.indexes()[rptr[r] + indexWithOffset] = nCol; + csrblock.values()[rptr[r] + indexWithOffset] = 1; } } if(sparseRowsWZeros != null) { @@ -302,12 +303,6 @@ public String toString() { } private static class DummycodeSparseApplyTask extends ColumnApplyTask { - - protected DummycodeSparseApplyTask(ColumnEncoderDummycode encoder, MatrixBlock input, - MatrixBlock out, int outputCol) { - super(encoder, input, out, outputCol); - } - protected DummycodeSparseApplyTask(ColumnEncoderDummycode encoder, MatrixBlock input, MatrixBlock out, int outputCol, int startRow, int blk) { super(encoder, input, out, outputCol, startRow, blk); diff --git a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderFeatureHash.java b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderFeatureHash.java index c57c72f459d..00c65097567 100644 --- a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderFeatureHash.java +++ b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderFeatureHash.java @@ -170,7 +170,6 @@ public String toString() { } public static class FeatureHashSparseApplyTask extends ColumnApplyTask{ - public FeatureHashSparseApplyTask(ColumnEncoderFeatureHash encoder, CacheBlock input, MatrixBlock out, int outputCol, int startRow, int blk) { super(encoder, input, out, outputCol, startRow, blk); diff --git a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderPassThrough.java b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderPassThrough.java index 12077221c05..411e650aa4f 100644 --- a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderPassThrough.java +++ b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderPassThrough.java @@ -100,17 +100,18 @@ protected void applySparse(CacheBlock in, MatrixBlock out, int outputCol, int sparseRowsWZeros = new ArrayList<>(); sparseRowsWZeros.add(ii); } + int indexWithOffset = sparseRowPointerOffset != null ? sparseRowPointerOffset[ii] - 1 + index : index; if (mcsr) { SparseRowVector row = (SparseRowVector) out.getSparseBlock().get(ii); - row.values()[index] = v; - row.indexes()[index] = outputCol; + row.values()[indexWithOffset] = v; + row.indexes()[indexWithOffset] = outputCol; } else { //csr // Manually fill the column-indexes and values array SparseBlockCSR csrblock = (SparseBlockCSR)out.getSparseBlock(); int rptr[] = csrblock.rowPointers(); - csrblock.indexes()[rptr[ii]+index] = outputCol; - csrblock.values()[rptr[ii]+index] = codes[ii-rowStart]; + csrblock.indexes()[rptr[ii]+indexWithOffset] = outputCol; + csrblock.values()[rptr[ii]+indexWithOffset] = codes[ii-rowStart]; } } } @@ -160,13 +161,6 @@ public String toString() { } public static class PassThroughSparseApplyTask extends ColumnApplyTask{ - - - protected PassThroughSparseApplyTask(ColumnEncoderPassThrough encoder, CacheBlock input, - MatrixBlock out, int outputCol) { - super(encoder, input, out, outputCol); - } - protected PassThroughSparseApplyTask(ColumnEncoderPassThrough encoder, CacheBlock input, MatrixBlock out, int outputCol, int startRow, int blk) { super(encoder, input, out, outputCol, startRow, blk); diff --git a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderRecode.java b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderRecode.java index 9569aa69d91..059c1f94589 100644 --- a/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderRecode.java +++ b/src/main/java/org/apache/sysds/runtime/transform/encode/ColumnEncoderRecode.java @@ -76,7 +76,7 @@ public static String constructRecodeMapEntry(String token, Long code) { return constructRecodeMapEntry(token, code, sb); } - private static String constructRecodeMapEntry(Object token, Long code, StringBuilder sb) { + public static String constructRecodeMapEntry(Object token, Long code, StringBuilder sb) { sb.setLength(0); // reset reused string builder return sb.append(token).append(Lop.DATATYPE_PREFIX).append(code.longValue()).toString(); } @@ -129,7 +129,7 @@ private long lookupRCDMap(Object key) { return _rcdMap.getOrDefault(key, -1L); } - public void computeRCDMapSizeEstimate(CacheBlock in, int[] sampleIndices) { + public void computeMapSizeEstimate(CacheBlock in, int[] sampleIndices) { if (getEstMetaSize() != 0) return; @@ -160,7 +160,8 @@ public void computeRCDMapSizeEstimate(CacheBlock in, int[] sampleIndices) { // We assume each partial map contains all distinct values and have the same size long avgKeySize = totSize / distinctFreq.size(); long valSize = 16L; //sizeof(Long) = 8 + header - long estMapSize = estDistCount * (avgKeySize + valSize); + this._avgEntrySize = avgKeySize + valSize; + long estMapSize = estDistCount * _avgEntrySize; setEstMetaSize(estMapSize); } @@ -187,7 +188,7 @@ public Callable getBuildTask(CacheBlock in) { @Override public Callable getPartialBuildTask(CacheBlock in, int startRow, - int blockSize, HashMap ret) { + int blockSize, HashMap ret, int p) { return new RecodePartialBuildTask(in, _colID, startRow, blockSize, ret); } @@ -378,11 +379,6 @@ public String toString() { } private static class RecodeSparseApplyTask extends ColumnApplyTask{ - - public RecodeSparseApplyTask(ColumnEncoderRecode encoder, CacheBlock input, MatrixBlock out, int outputCol) { - super(encoder, input, out, outputCol); - } - protected RecodeSparseApplyTask(ColumnEncoderRecode encoder, CacheBlock input, MatrixBlock out, int outputCol, int startRow, int blk) { super(encoder, input, out, outputCol, startRow, blk); diff --git a/src/main/java/org/apache/sysds/runtime/transform/encode/EncoderFactory.java b/src/main/java/org/apache/sysds/runtime/transform/encode/EncoderFactory.java index be0680379f1..82a866bb13d 100644 --- a/src/main/java/org/apache/sysds/runtime/transform/encode/EncoderFactory.java +++ b/src/main/java/org/apache/sysds/runtime/transform/encode/EncoderFactory.java @@ -19,10 +19,6 @@ package org.apache.sysds.runtime.transform.encode; -import static org.apache.sysds.runtime.util.CollectionUtils.except; -import static org.apache.sysds.runtime.util.CollectionUtils.intersect; -import static org.apache.sysds.runtime.util.CollectionUtils.unionDistinct; - import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -46,6 +42,11 @@ import org.apache.wink.json4j.JSONArray; import org.apache.wink.json4j.JSONObject; +import static org.apache.sysds.runtime.util.CollectionUtils.except; +import static org.apache.sysds.runtime.util.CollectionUtils.intersect; +import static org.apache.sysds.runtime.util.CollectionUtils.unionDistinct; +import static org.apache.sysds.runtime.util.CollectionUtils.naryUnionDistinct; + public interface EncoderFactory { final static Log LOG = LogFactory.getLog(EncoderFactory.class.getName()); @@ -117,22 +118,17 @@ public static MultiColumnEncoder createEncoder(String spec, String[] colnames, i List binIDs = TfMetaUtils.parseBinningColIDs(jSpec, colnames, minCol, maxCol); List weIDs = Arrays.asList(ArrayUtils .toObject(TfMetaUtils.parseJsonIDList(jSpec, colnames, TfMethod.WORD_EMBEDDING.toString(), minCol, maxCol))); - - //check if user passed an embeddings matrix - if(!weIDs.isEmpty() && embeddings == null) - throw new DMLRuntimeException("Missing argument Embeddings Matrix for transform [" + TfMethod.WORD_EMBEDDING + "]"); + List bowIDs = Arrays.asList(ArrayUtils + .toObject(TfMetaUtils.parseJsonIDList(jSpec, colnames, TfMethod.BAG_OF_WORDS.toString(), minCol, maxCol))); // NOTE: any dummycode column requires recode as preparation, unless the dummycode // column follows binning or feature hashing rcIDs = unionDistinct(rcIDs, except(except(dcIDs, binIDs), haIDs)); - // NOTE: Word Embeddings requires recode as preparation - //rcIDs = unionDistinct(rcIDs, weIDs); // Error out if the first level encoders have overlaps - if (intersect(rcIDs, binIDs, haIDs, weIDs)) - throw new DMLRuntimeException("More than one encoders (recode, binning, hashing, word_embedding) on one column is not allowed"); + if (intersect(rcIDs, binIDs, haIDs, weIDs, bowIDs)) + throw new DMLRuntimeException("More than one encoders (recode, binning, hashing, word_embedding, bag_of_words) on one column is not allowed"); - List ptIDs = except(except(except(UtilFunctions.getSeqList(1, clen, 1), unionDistinct(rcIDs, haIDs)), - binIDs), weIDs); + List ptIDs = except(UtilFunctions.getSeqList(1, clen, 1), naryUnionDistinct(rcIDs, haIDs, binIDs, weIDs, bowIDs)); List oIDs = Arrays.asList(ArrayUtils .toObject(TfMetaUtils.parseJsonIDList(jSpec, colnames, TfMethod.OMIT.toString(), minCol, maxCol))); List mvIDs = Arrays.asList(ArrayUtils.toObject( @@ -153,6 +149,9 @@ public static MultiColumnEncoder createEncoder(String spec, String[] colnames, i if(!weIDs.isEmpty()) for(Integer id : weIDs) addEncoderToMap(new ColumnEncoderWordEmbedding(id), colEncoders); + if(!bowIDs.isEmpty()) + for(Integer id : bowIDs) + addEncoderToMap(new ColumnEncoderBagOfWords(id), colEncoders); if(!binIDs.isEmpty()) for(Object o : (JSONArray) jSpec.get(TfMethod.BIN.toString())) { JSONObject colspec = (JSONObject) o; diff --git a/src/main/java/org/apache/sysds/runtime/transform/encode/MultiColumnEncoder.java b/src/main/java/org/apache/sysds/runtime/transform/encode/MultiColumnEncoder.java index c514e010a72..0417e67ba1f 100644 --- a/src/main/java/org/apache/sysds/runtime/transform/encode/MultiColumnEncoder.java +++ b/src/main/java/org/apache/sysds/runtime/transform/encode/MultiColumnEncoder.java @@ -23,6 +23,7 @@ import java.io.ObjectInput; import java.io.ObjectOutput; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -63,6 +64,8 @@ import org.apache.sysds.runtime.util.IndexRange; import org.apache.sysds.utils.stats.TransformStatistics; +import static org.apache.sysds.utils.MemoryEstimates.intArrayCost; + public class MultiColumnEncoder implements Encoder { protected static final Log LOG = LogFactory.getLog(MultiColumnEncoder.class.getName()); @@ -166,7 +169,8 @@ private List> getEncodeTasks(CacheBlock in, MatrixBlock out List> tasks = new ArrayList<>(); List> applyTAgg = null; Map depMap = new HashMap<>(); - boolean hasDC = getColumnEncoders(ColumnEncoderDummycode.class).size() > 0; + boolean hasDC = !getColumnEncoders(ColumnEncoderDummycode.class).isEmpty(); + boolean hasBOW = !getColumnEncoders(ColumnEncoderBagOfWords.class).isEmpty(); boolean applyOffsetDep = false; boolean independentUpdateDC = false; _meta = new FrameBlock(in.getNumColumns(), ValueType.STRING); @@ -178,9 +182,11 @@ private List> getEncodeTasks(CacheBlock in, MatrixBlock out // Create the build tasks List> buildTasks = e.getBuildTasks(in); tasks.addAll(buildTasks); - if(buildTasks.size() > 0) { + boolean compositeHasDC = e.hasEncoder(ColumnEncoderDummycode.class); + boolean compositeHasBOW = e.hasEncoder(ColumnEncoderBagOfWords.class); + if(!buildTasks.isEmpty()) { // Check if any Build independent UpdateDC task (Bin+DC, FH+DC) - if (e.hasEncoder(ColumnEncoderDummycode.class) + if (compositeHasDC && buildTasks.size() > 1 //filter out FH && !buildTasks.get(buildTasks.size()-2).hasDependency(buildTasks.get(buildTasks.size()-1))) independentUpdateDC = true; @@ -203,7 +209,7 @@ private List> getEncodeTasks(CacheBlock in, MatrixBlock out new Integer[] {tasks.size() - 1, tasks.size()}); //Build/UpdateDC } // AllocMetaTask never depends on the UpdateDC task - if (e.hasEncoder(ColumnEncoderDummycode.class) && buildTasks.size() > 1) + if (compositeHasDC && buildTasks.size() > 1) depMap.put(new Integer[] {1, 2}, //AllocMetaTask (2nd task) new Integer[] {tasks.size() - 2, tasks.size()-1}); //BuildTask else @@ -220,11 +226,13 @@ private List> getEncodeTasks(CacheBlock in, MatrixBlock out new Integer[] {0, 1}); //Allocation task (1st task) ApplyTasksWrapperTask applyTaskWrapper = new ApplyTasksWrapperTask(e, in, out, pool); - if(e.hasEncoder(ColumnEncoderDummycode.class)) { - // Allocation depends on build if DC is in the list. + if(compositeHasDC || compositeHasBOW) { + // Allocation depends on build if DC or BOW is in the list. // Note, DC is the only encoder that changes dimensionality - depMap.put(new Integer[] {0, 1}, //Allocation task (1st task) - new Integer[] {tasks.size() - 1, tasks.size()}); //BuildTask + depMap.put(new Integer[]{0, 1}, //Allocation task (1st task) + new Integer[]{tasks.size() - 1, tasks.size()}); //BuildTask + } + if(compositeHasDC || compositeHasBOW){ // UpdateOutputColTask, that sets the starting offsets of the DC columns, // depends on the Build completion tasks depMap.put(new Integer[] {-2, -1}, //UpdateOutputColTask (last task) @@ -233,7 +241,7 @@ private List> getEncodeTasks(CacheBlock in, MatrixBlock out applyOffsetDep = true; } - if(hasDC && applyOffsetDep) { + if((hasDC || hasBOW) && applyOffsetDep) { // Apply tasks depend on UpdateOutputColTask depMap.put(new Integer[] {tasks.size(), tasks.size() + 1}, //ApplyTask new Integer[] {-2, -1}); //UpdateOutputColTask (last task) @@ -249,7 +257,7 @@ private List> getEncodeTasks(CacheBlock in, MatrixBlock out // Create the getMetadata task tasks.add(DependencyThreadPool.createDependencyTask(new ColumnMetaDataTask(e, _meta))); } - if(hasDC) + if(hasDC || hasBOW) // Create the last task, UpdateOutputColTask tasks.add(DependencyThreadPool.createDependencyTask(new UpdateOutputColTask(this, applyTAgg))); @@ -355,6 +363,7 @@ public MatrixBlock apply(CacheBlock in, MatrixBlock out, int outputCol, int k boolean hasDC = false; boolean hasWE = false; + //TODO adapt transform apply for BOW int distinctWE = 0; int sizeWE = 0; for(ColumnEncoderComposite columnEncoder : _columnEncoders) { @@ -363,10 +372,10 @@ public MatrixBlock apply(CacheBlock in, MatrixBlock out, int outputCol, int k if(enc instanceof ColumnEncoderWordEmbedding){ hasWE = true; distinctWE = ((ColumnEncoderWordEmbedding) enc).getNrDistinctEmbeddings(); - sizeWE = ((ColumnEncoderWordEmbedding) enc).getDomainSize(); + sizeWE = enc.getDomainSize(); } } - outputMatrixPreProcessing(out, in, hasDC, hasWE, distinctWE, sizeWE); + outputMatrixPreProcessing(out, in, hasDC, hasWE, distinctWE, sizeWE, 0, null, -1); if(k > 1) { if(!_partitionDone) //happens if this method is directly called deriveNumRowPartitions(in, k); @@ -394,7 +403,7 @@ private List> getApplyTasks(CacheBlock in, MatrixBlock out, List> tasks = new ArrayList<>(); int offset = outputCol; for(ColumnEncoderComposite e : _columnEncoders) { - tasks.addAll(e.getApplyTasks(in, out, e._colID - 1 + offset)); + tasks.addAll(e.getApplyTasks(in, out, e._colID - 1 + offset, null)); offset = getOffset(offset, e); } return tasks; @@ -414,7 +423,8 @@ private void applyMT(CacheBlock in, MatrixBlock out, int outputCol, int k) { if(APPLY_ENCODER_SEPARATE_STAGES) { int offset = outputCol; for (ColumnEncoderComposite e : _columnEncoders) { - pool.submitAllAndWait(e.getApplyTasks(in, out, e._colID - 1 + offset)); + // for now bag of words is only used in encode + pool.submitAllAndWait(e.getApplyTasks(in, out, e._colID - 1 + offset, null)); offset = getOffset(offset, e); } } else @@ -454,6 +464,7 @@ private void deriveNumRowPartitions(CacheBlock in, int k) { int nThread = OptimizerUtils.getTransformNumThreads(); //VCores int minNumRows = 16000; //min rows per partition List recodeEncoders = new ArrayList<>(); + List bowEncoders = new ArrayList<>(); // Count #Builds and #Applies (= #Col) int nBuild = 0; for (ColumnEncoderComposite e : _columnEncoders) @@ -461,6 +472,8 @@ private void deriveNumRowPartitions(CacheBlock in, int k) { nBuild++; if (e.hasEncoder(ColumnEncoderRecode.class)) recodeEncoders.add(e); + if (e.hasEncoder(ColumnEncoderBagOfWords.class)) + bowEncoders.add(e); } int nApply = in.getNumColumns(); // #BuildBlocks = (2 * #PhysicalCores)/#build @@ -470,27 +483,44 @@ private void deriveNumRowPartitions(CacheBlock in, int k) { if (numBlocks[1] == 0 && nApply > 0 && nApply < nThread*2) numBlocks[1] = Math.round(((float)nThread*2)/nApply); - // Reduce #blocks if #rows per partition is too small - while (numBlocks[0] > 1 && nRow/numBlocks[0] < minNumRows) - numBlocks[0]--; - while (numBlocks[1] > 1 && nRow/numBlocks[1] < minNumRows) - numBlocks[1]--; + int bowNumBuildBlks = numBlocks[0]; + int bowNumApplyBlks = numBlocks[1]; - // Reduce #build blocks for the recoders if all don't fit in memory + // Reduce #blocks if #rows per partition is too small + // while (numBlocks[0] > 1 && nRow/numBlocks[0] < minNumRows) + // numBlocks[0]--; + // while (numBlocks[1] > 1 && nRow/numBlocks[1] < minNumRows) + // numBlocks[1]--; + // the two while loop should be equal to following code: + int optimalPartitions = Math.max(1, nRow / minNumRows); + numBlocks[0] = Math.min(numBlocks[0], optimalPartitions); + numBlocks[1] = Math.min(numBlocks[1], optimalPartitions); int rcdNumBuildBlks = numBlocks[0]; - if (numBlocks[0] > 1 && recodeEncoders.size() > 0) { - // Estimate recode map sizes - estimateRCMapSize(in, recodeEncoders); - // Memory budget for maps = 70% of heap - sizeof(input) - long memBudget = (long) (OptimizerUtils.getLocalMemBudget() - in.getInMemorySize()); - // Worst case scenario: all partial maps contain all distinct values (if < #rows) - long totMemOverhead = getTotalMemOverhead(in, rcdNumBuildBlks, recodeEncoders); - // Reduce recode build blocks count till they fit in the memory budget - while (rcdNumBuildBlks > 1 && totMemOverhead > memBudget) { - rcdNumBuildBlks--; - totMemOverhead = getTotalMemOverhead(in, rcdNumBuildBlks, recodeEncoders); - // TODO: Reduce only the ones with large maps - } + + // Use a smaller minNumRows for BOW encoders because of a larger computational overhead per row + optimalPartitions = Math.max(1, nRow / (minNumRows / 16)); + bowNumBuildBlks = Math.min(bowNumBuildBlks, optimalPartitions); + bowNumApplyBlks = Math.min(bowNumApplyBlks, optimalPartitions); + + + // RC: Reduce #build blocks for all encoders if all don't fit in memory + if (numBlocks[0] > 1 && !recodeEncoders.isEmpty() && bowEncoders.isEmpty()) { + rcdNumBuildBlks = getNumBuildBlksMemorySafe(in, recodeEncoders, rcdNumBuildBlks, false); + } + // BOW: Reduce #build blocks for all encoders if all don't fit in memory + else if (bowNumBuildBlks > 1 && recodeEncoders.isEmpty() && !bowEncoders.isEmpty()) { + bowNumBuildBlks = getNumBuildBlksMemorySafe(in, bowEncoders, bowNumBuildBlks, true); + } + // RC + BOW: check if all encoders fit into memory + else if (bowNumBuildBlks > 1 || rcdNumBuildBlks > 1) { + // Estimate map sizes, fused with other encoders (bag_of_words) + List> encoders = new ArrayList<>(); + encoders.add(recodeEncoders); + encoders.add(bowEncoders); + int[] bldBlks = new int[]{rcdNumBuildBlks, bowNumBuildBlks}; + getNumBuildBlksMixedEncMemorySafe(in, encoders, bldBlks); + rcdNumBuildBlks = bldBlks[0]; + bowNumBuildBlks = bldBlks[1]; } // TODO: If still don't fit, serialize the column encoders @@ -506,23 +536,89 @@ private void deriveNumRowPartitions(CacheBlock in, int k) { int rcdNumBlocks = rcdNumBuildBlks; recodeEncoders.forEach(e -> e.setNumPartitions(rcdNumBlocks, numBlocks[1])); } + if (bowNumBuildBlks > 0) { + final int bowNumBlocks = bowNumBuildBlks; + final int bowApplyBlks = bowNumApplyBlks; + bowEncoders.forEach(e -> e.setNumPartitions(bowNumBlocks, bowApplyBlks)); + } //System.out.println("Block count = ["+numBlocks[0]+", "+numBlocks[1]+"], Recode block count = "+rcdNumBuildBlks); } - private void estimateRCMapSize(CacheBlock in, List rcList) { + private int getNumBuildBlksMemorySafe(CacheBlock in, List encoders, int numBldBlks, boolean hasBOW) { + estimateMapSize(in, encoders); + // Memory budget for maps = 70% of heap - sizeof(input) + long memBudget = (long) (OptimizerUtils.getLocalMemBudget() - in.getInMemorySize()); + if(hasBOW){ + // integer arrays: nnzPerRow for each bow encoder + memBudget -= encoders.size()*(long) intArrayCost(in.getNumRows()); + } + // Worst case scenario: all partial maps contain all distinct values (if < #rows) + long totMemOverhead = getTotalMemOverhead(in, numBldBlks, encoders); + + // Reduce recode build blocks count till they fit in the memory budget + while (numBldBlks > 1 && totMemOverhead > memBudget) { + numBldBlks--; + totMemOverhead = getTotalMemOverhead(in, numBldBlks, encoders); + // TODO: Reduce only the ones with large maps + } + return numBldBlks; + } + + private void getNumBuildBlksMixedEncMemorySafe(CacheBlock in, List> encs, int[] blks) { + // Memory budget for maps = 70% of heap - sizeof(input) + long memBudget = (long) (OptimizerUtils.getLocalMemBudget() - in.getInMemorySize()); + // integer arrays: nnzPerRow for each bow encoder + memBudget -= encs.get(1).size()*((long) intArrayCost(in.getNumRows())); + + int numOfEncTypes = encs.size(); + long[] totMemOverhead = new long[numOfEncTypes]; + for (int i = 0; i < numOfEncTypes; i++) { + estimateMapSize(in, encs.get(i)); + // Worst case scenario: all partial maps contain all distinct values (if < #rows) + totMemOverhead[i] = getTotalMemOverhead(in, blks[i], encs.get(i)); + } + + int next = blks[1] > 1 ? 1 : 0; + // round-robin reducing + int skipped = 0; + while (skipped != numOfEncTypes && Arrays.stream(totMemOverhead).sum() > memBudget) { + if(blks[next] > 1){ + blks[next]--; + totMemOverhead[next] = getTotalMemOverhead(in, blks[next], encs.get(next)); + next = (next + 1) % numOfEncTypes; + skipped = 0; + } else + skipped++; + } + // TODO: Reduce the large encoder types, similar to getNumBuildBlksMemorySafe + } + +// not used rn, commented because of missing code coverage +// private long estimateSparseOutputSize(List bowEncs, int nApply, int nRows){ +// // #rows x (#col - #bowEncs + bow-avg-nnz) +// double avgNnzPerRow = 0.0; +// for (ColumnEncoderComposite enc : bowEncs){ +// ColumnEncoderBagOfWords bow = enc.getEncoder(ColumnEncoderBagOfWords.class); +// avgNnzPerRow += bow.avgNnzPerRow; +// } +// long nnzBow = (long) (avgNnzPerRow*nRows); +// long nnzOther = (long) nRows *(nApply - bowEncs.size()); +// long nnz = nnzBow + nnzOther; +// return estimateSizeInMemory(nRows, nnz); +// } + + private void estimateMapSize(CacheBlock in, List encList) { long t0 = DMLScript.STATISTICS ? System.nanoTime() : 0; // Collect sample row indices int k = OptimizerUtils.getTransformNumThreads(); - int sampleSize = (int) (0.1 * in.getNumRows()); - int seed = (int) System.nanoTime(); - int[] sampleInds = ComEstSample.getSortedSample(in.getNumRows(), sampleSize, seed, 1); + int[] sampleInds = getSampleIndices(in, (int) (0.1 * in.getNumRows()), (int) System.nanoTime(), 1); // Concurrent (column-wise) recode map size estimation ExecutorService pool = CommonThreadPool.get(k); try { pool.submit(() -> { - rcList.stream().parallel().forEach(e -> { - e.computeRCDMapSizeEstimate(in, sampleInds); + encList.stream().parallel().forEach(e -> { + e.computeMapSizeEstimate(in, sampleInds); }); }).get(); } @@ -534,32 +630,40 @@ private void estimateRCMapSize(CacheBlock in, List rc } if(DMLScript.STATISTICS) { - LOG.debug("Elapsed time for RC map size estimation: " + ((double) System.nanoTime() - t0) / 1000000 + " ms"); + LOG.debug("Elapsed time for encoder map size estimation: " + ((double) System.nanoTime() - t0) / 1000000 + " ms"); TransformStatistics.incMapSizeEstimationTime(System.nanoTime() - t0); } } + private int[] getSampleIndices(CacheBlock in, int sampleSize, int seed, int k){ + return ComEstSample.getSortedSample(in.getNumRows(), sampleSize, seed, k); + } + // Estimate total memory overhead of the partial recode maps of all recoders - private long getTotalMemOverhead(CacheBlock in, int nBuildpart, List rcEncoders) { + private long getTotalMemOverhead(CacheBlock in, int nBuildpart, List encoders) { long totMemOverhead = 0; if (nBuildpart == 1) { // Sum the estimated map sizes - totMemOverhead = rcEncoders.stream().mapToLong(ColumnEncoderComposite::getEstMetaSize).sum(); + totMemOverhead = encoders.stream().mapToLong(ColumnEncoderComposite::getEstMetaSize).sum(); return totMemOverhead; } // Estimate map size of each partition and sum - for (ColumnEncoderComposite rce : rcEncoders) { - long avgEntrySize = rce.getEstMetaSize()/ rce.getEstNumDistincts(); + for (ColumnEncoderComposite enc : encoders) { int partSize = in.getNumRows()/nBuildpart; - int partNumDist = Math.min(partSize, rce.getEstNumDistincts()); //#distincts not more than #rows - long allMapsSize = partNumDist * avgEntrySize * nBuildpart; //worst-case scenario + int partNumDist = Math.min(partSize, enc.getEstNumDistincts()); //#distincts not more than #rows + if(enc.getAvgEntrySize() == 0) + throw new DMLRuntimeException("Error while estimating entry size of encoder map"); + long allMapsSize = partNumDist * enc.getAvgEntrySize() * nBuildpart; //worst-case scenario totMemOverhead += allMapsSize; } return totMemOverhead; } - private static void outputMatrixPreProcessing(MatrixBlock output, CacheBlock input, boolean hasDC, boolean hasWE, int distinctWE, int sizeWE) { + private static void outputMatrixPreProcessing(MatrixBlock output, CacheBlock input, boolean hasDC, boolean hasWE, + int distinctWE, int sizeWE, int numBOW, int[] nnzPerRowBOW, int nnz) { long t0 = DMLScript.STATISTICS ? System.nanoTime() : 0; + if(nnz < 0) + nnz = output.getNumRows() * input.getNumColumns(); if(output.isInSparseFormat()) { if (MatrixBlock.DEFAULT_SPARSEBLOCK != SparseBlock.Type.CSR && MatrixBlock.DEFAULT_SPARSEBLOCK != SparseBlock.Type.MCSR) @@ -591,15 +695,20 @@ private static void outputMatrixPreProcessing(MatrixBlock output, CacheBlock } } else { //csr - int size = output.getNumRows() * input.getNumColumns(); - SparseBlock csrblock = new SparseBlockCSR(output.getNumRows(), size, size); + SparseBlockCSR csrblock = new SparseBlockCSR(output.getNumRows(), nnz, nnz); // Manually fill the row pointers based on nnzs/row (= #cols in the input) // Not using the set() methods to 1) avoid binary search and shifting, // 2) reduce thread contentions on the arrays - int[] rptr = ((SparseBlockCSR)csrblock).rowPointers(); - for (int i=0; i input, Ma } @Override - public Object call() throws Exception { - boolean hasUDF = _encoder.getColumnEncoders().stream().anyMatch(e -> e.hasEncoder(ColumnEncoderUDF.class)); - boolean hasWE = false; + public Object call() { + boolean hasUDF = false, hasDC = false, hasWE = false; int distinctWE = 0; int sizeWE = 0; - for (ColumnEncoder enc : _encoder.getEncoders()) - if(enc instanceof ColumnEncoderWordEmbedding){ + long nnzBOW = 0; + int numBOWEncoder = 0; + int[] nnzPerRowBOW = null; + for (ColumnEncoderComposite enc : _encoder.getEncoders()){ + if(enc.hasEncoder(ColumnEncoderUDF.class)) + hasUDF = true; + else if (enc.hasEncoder(ColumnEncoderDummycode.class)) + hasDC = true; + else if(enc.hasEncoder(ColumnEncoderBagOfWords.class)){ + ColumnEncoderBagOfWords bowEnc = enc.getEncoder(ColumnEncoderBagOfWords.class); + numBOWEncoder++; + nnzBOW += bowEnc.nnz; + if(nnzPerRowBOW != null) + for (int i = 0; i < bowEnc.nnzPerRow.length; i++) { + nnzPerRowBOW[i] += bowEnc.nnzPerRow[i]; + } + else { + nnzPerRowBOW = bowEnc.nnzPerRow.clone(); + } + } + else if(enc.hasEncoder(ColumnEncoderWordEmbedding.class)){ hasWE = true; - distinctWE = ((ColumnEncoderWordEmbedding) enc).getNrDistinctEmbeddings(); - sizeWE = ((ColumnEncoderWordEmbedding) enc).getDomainSize(); + distinctWE = enc.getEncoder(ColumnEncoderWordEmbedding.class).getNrDistinctEmbeddings(); + sizeWE = enc.getDomainSize(); } + } + int numCols = _encoder.getNumOutCols(); - boolean hasDC = _encoder.getColumnEncoders(ColumnEncoderDummycode.class).size() > 0; - long estNNz = (long) _input.getNumRows() * (hasUDF ? numCols : _input.getNumColumns()); + long estNNz = (long) _input.getNumRows() * (hasUDF ? numCols : _input.getNumColumns() - numBOWEncoder) + nnzBOW; boolean sparse = MatrixBlock.evalSparseFormatInMemory(_input.getNumRows(), numCols, estNNz) && !hasUDF; _output.reset(_input.getNumRows(), numCols, sparse, estNNz); - outputMatrixPreProcessing(_output, _input, hasDC, hasWE, distinctWE,sizeWE); + outputMatrixPreProcessing(_output, _input, hasDC, hasWE, distinctWE, sizeWE, numBOWEncoder, nnzPerRowBOW, (int) estNNz); return null; } @@ -1195,7 +1321,8 @@ private static class ApplyTasksWrapperTask extends DependencyWrapperTask private final MatrixBlock _out; private final CacheBlock _in; /** Offset because of dummmy coding such that the column id is correct. */ - private int _offset = -1; + private int _offset = -1; + private int[] _sparseRowPointerOffsets = null; private ApplyTasksWrapperTask(ColumnEncoder encoder, CacheBlock in, MatrixBlock out, DependencyThreadPool pool) { @@ -1207,7 +1334,7 @@ private ApplyTasksWrapperTask(ColumnEncoder encoder, CacheBlock in, @Override public List> getWrappedTasks() { - return _encoder.getApplyTasks(_in, _out, _encoder._colID - 1 + _offset); + return _encoder.getApplyTasks(_in, _out, _encoder._colID - 1 + _offset, _sparseRowPointerOffsets); } @Override @@ -1224,6 +1351,10 @@ public void setOffset(int offset) { _offset = offset; } + public void setSparseRowPointerOffsets(int[] offsets) { + _sparseRowPointerOffsets = offsets; + } + @Override public String toString() { return getClass().getSimpleName() + ""; @@ -1252,19 +1383,28 @@ public String toString() { public Object call() throws Exception { int currentCol = -1; int currentOffset = 0; + int[] sparseRowPointerOffsets = null; for(DependencyTask dtask : _applyTasksWrappers) { + ((ApplyTasksWrapperTask) dtask).setOffset(currentOffset); + if(sparseRowPointerOffsets != null) + ((ApplyTasksWrapperTask) dtask).setSparseRowPointerOffsets(sparseRowPointerOffsets.clone()); int nonOffsetCol = ((ApplyTasksWrapperTask) dtask)._encoder._colID - 1; if(nonOffsetCol > currentCol) { currentCol = nonOffsetCol; - currentOffset = _encoder._columnEncoders.subList(0, nonOffsetCol).stream().mapToInt(e -> { - ColumnEncoderDummycode dc = e.getEncoder(ColumnEncoderDummycode.class); - if(dc == null) - return 0; - return dc._domainSize - 1; - }).sum(); + ColumnEncoderComposite enc = _encoder._columnEncoders.get(nonOffsetCol); + if(enc.hasEncoder(ColumnEncoderDummycode.class)) + currentOffset += enc.getEncoder(ColumnEncoderDummycode.class)._domainSize - 1; + else if (enc.hasEncoder(ColumnEncoderBagOfWords.class)) { + ColumnEncoderBagOfWords bow = enc.getEncoder(ColumnEncoderBagOfWords.class); + currentOffset += bow.getDomainSize() - 1; + if(sparseRowPointerOffsets == null) + sparseRowPointerOffsets = bow.nnzPerRow.clone(); + else + for (int r = 0; r < sparseRowPointerOffsets.length; r++) { + sparseRowPointerOffsets[r] += bow.nnzPerRow[r] - 1; + } + } } - ((ApplyTasksWrapperTask) dtask).setOffset(currentOffset); - } return null; } diff --git a/src/main/java/org/apache/sysds/runtime/util/CollectionUtils.java b/src/main/java/org/apache/sysds/runtime/util/CollectionUtils.java index b891c0b49c6..837cc088b54 100644 --- a/src/main/java/org/apache/sysds/runtime/util/CollectionUtils.java +++ b/src/main/java/org/apache/sysds/runtime/util/CollectionUtils.java @@ -139,6 +139,15 @@ public static List unionDistinct(List a, List b) { } return ret; } + + @SafeVarargs + public static List naryUnionDistinct(List... lists) { + List out = lists[0]; + for (int i = 1; i < lists.length; i++) { + out = unionDistinct(out, lists[i]); + } + return out; + } public static List unionAll(List a, List b) { return CollectionUtils.asList(a, b); diff --git a/src/main/java/org/apache/sysds/utils/stats/TransformStatistics.java b/src/main/java/org/apache/sysds/utils/stats/TransformStatistics.java index 9ace7294627..005fd118971 100644 --- a/src/main/java/org/apache/sysds/utils/stats/TransformStatistics.java +++ b/src/main/java/org/apache/sysds/utils/stats/TransformStatistics.java @@ -27,6 +27,7 @@ public class TransformStatistics { //private static final LongAdder buildTime = new LongAdder(); private static final LongAdder recodeBuildTime = new LongAdder(); private static final LongAdder binningBuildTime = new LongAdder(); + private static final LongAdder bowBuildTime = new LongAdder(); private static final LongAdder imputeBuildTime = new LongAdder(); //private static final LongAdder applyTime = new LongAdder(); @@ -34,6 +35,7 @@ public class TransformStatistics { private static final LongAdder dummyCodeApplyTime = new LongAdder(); private static final LongAdder wordEmbeddingApplyTime = new LongAdder(); + private static final LongAdder bagOfWordsApplyTime = new LongAdder(); private static final LongAdder passThroughApplyTime = new LongAdder(); private static final LongAdder featureHashingApplyTime = new LongAdder(); private static final LongAdder binningApplyTime = new LongAdder(); @@ -61,6 +63,11 @@ public static void incWordEmbeddingApplyTime(long t){ wordEmbeddingApplyTime.add(t); } + public static void incBagOfWordsApplyTime(long t){ + bagOfWordsApplyTime.add(t); + } + + public static void incBinningApplyTime(long t) { binningApplyTime.add(t); } @@ -93,6 +100,10 @@ public static void incBinningBuildTime(long t) { binningBuildTime.add(t); } + public static void incBagOfWordsBuildTime(long t) { + bowBuildTime.add(t); + } + public static void incImputeBuildTime(long t) { imputeBuildTime.add(t); } @@ -111,14 +122,15 @@ public static void incMapSizeEstimationTime(long t) { public static long getEncodeBuildTime() { return binningBuildTime.longValue() + imputeBuildTime.longValue() + - recodeBuildTime.longValue(); + recodeBuildTime.longValue() + bowBuildTime.longValue(); } public static long getEncodeApplyTime() { return dummyCodeApplyTime.longValue() + binningApplyTime.longValue() + featureHashingApplyTime.longValue() + passThroughApplyTime.longValue() + recodeApplyTime.longValue() + UDFApplyTime.longValue() + - omitApplyTime.longValue() + imputeApplyTime.longValue() + wordEmbeddingApplyTime.longValue(); + omitApplyTime.longValue() + imputeApplyTime.longValue() + wordEmbeddingApplyTime.longValue() + + bagOfWordsApplyTime.longValue(); } public static void reset() { @@ -127,11 +139,14 @@ public static void reset() { recodeBuildTime.reset(); binningBuildTime.reset(); imputeBuildTime.reset(); + bowBuildTime.reset(); // applyTime.reset(); recodeApplyTime.reset(); dummyCodeApplyTime.reset(); passThroughApplyTime.reset(); featureHashingApplyTime.reset(); + bagOfWordsApplyTime.reset(); + wordEmbeddingApplyTime.reset(); binningApplyTime.reset(); UDFApplyTime.reset(); omitApplyTime.reset(); @@ -157,6 +172,9 @@ public static String displayStatistics() { if(imputeBuildTime.longValue() > 0) sb.append("\tImpute build time:\t").append(String.format("%.3f", imputeBuildTime.longValue()*1e-9)).append(" sec.\n"); + if(bowBuildTime.longValue() > 0) + sb.append("\tBagOfWords build time:\t").append(String.format("%.3f", + bowBuildTime.longValue()*1e-9)).append(" sec.\n"); sb.append("TransformEncode apply time:\t").append(String.format("%.3f", getEncodeApplyTime()*1e-9)).append(" sec.\n"); @@ -172,6 +190,9 @@ public static String displayStatistics() { if(wordEmbeddingApplyTime.longValue() > 0) sb.append("\tWordEmbedding apply time:\t").append(String.format("%.3f", wordEmbeddingApplyTime.longValue()*1e-9)).append(" sec.\n"); + if(bagOfWordsApplyTime.longValue() > 0) + sb.append("\tBagOfWords apply time:\t").append(String.format("%.3f", + bagOfWordsApplyTime.longValue()*1e-9)).append(" sec.\n"); if(featureHashingApplyTime.longValue() > 0) sb.append("\tHashing apply time:\t").append(String.format("%.3f", featureHashingApplyTime.longValue()*1e-9)).append(" sec.\n"); diff --git a/src/test/java/org/apache/sysds/test/AutomatedTestBase.java b/src/test/java/org/apache/sysds/test/AutomatedTestBase.java index 0ea20b8be26..4c0f25d74a1 100644 --- a/src/test/java/org/apache/sysds/test/AutomatedTestBase.java +++ b/src/test/java/org/apache/sysds/test/AutomatedTestBase.java @@ -901,13 +901,17 @@ protected static String readDMLLineageDedupFromHDFS(String fileName) { } protected static FrameBlock readDMLFrameFromHDFS(String fileName, FileFormat fmt) throws IOException { - return readDMLFrameFromHDFS(fileName, fmt, getMetaData(fileName)); + return readDMLFrameFromHDFS(fileName, fmt, getMetaData(fileName), true); } - protected static FrameBlock readDMLFrameFromHDFS(String fileName, FileFormat fmt, MetaDataAll meta) + protected static FrameBlock readDMLFrameFromHDFS(String fileName, FileFormat fmt, boolean base) throws IOException { + return readDMLFrameFromHDFS(fileName, fmt, getMetaData(fileName), base); + } + + protected static FrameBlock readDMLFrameFromHDFS(String fileName, FileFormat fmt, MetaDataAll meta, boolean base) throws IOException { // read frame data from hdfs - String strFrameFileName = baseDirectory + OUTPUT_DIR + fileName; + String strFrameFileName = base ? baseDirectory + OUTPUT_DIR + fileName : fileName; FrameReader reader = FrameReaderFactory.createFrameReader(fmt); if(meta.getSchema() == null) return reader.readFrameFromHDFS(strFrameFileName, meta.getDim1(),meta.getDim2()); diff --git a/src/test/java/org/apache/sysds/test/functions/transform/TransformFrameEncodeBagOfWords.java b/src/test/java/org/apache/sysds/test/functions/transform/TransformFrameEncodeBagOfWords.java new file mode 100644 index 00000000000..e3d1c07be26 --- /dev/null +++ b/src/test/java/org/apache/sysds/test/functions/transform/TransformFrameEncodeBagOfWords.java @@ -0,0 +1,299 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF 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.apache.sysds.test.functions.transform; + +import org.apache.commons.lang3.NotImplementedException; +import org.apache.sysds.common.Types; +import org.apache.sysds.common.Types.ExecMode; +import org.apache.sysds.runtime.DMLRuntimeException; +import org.apache.sysds.runtime.frame.data.FrameBlock; +import org.apache.sysds.runtime.matrix.data.MatrixValue; +import org.apache.sysds.runtime.transform.encode.ColumnEncoderBagOfWords; +import org.apache.sysds.test.AutomatedTestBase; +import org.apache.sysds.test.TestConfiguration; +import org.apache.sysds.test.TestUtils; +import org.junit.Test; + +import java.io.BufferedWriter; +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.nio.file.Files; + +import static org.apache.sysds.runtime.transform.encode.ColumnEncoderBagOfWords.tokenize; +import static org.junit.Assert.assertThrows; + +public class TransformFrameEncodeBagOfWords extends AutomatedTestBase +{ + private final static String TEST_NAME1 = "TransformFrameEncodeBagOfWords"; + private final static String TEST_DIR = "functions/transform/"; + private final static String TEST_CLASS_DIR = TEST_DIR + TransformFrameEncodeBagOfWords.class.getSimpleName() + "/"; + // for benchmarking: Digital_Music_Text.csv + private String DATASET = "amazonReview2023/Digital_Music_Text_Head2k_With_RCD_Col.csv"; + + @Override + public void setUp() { + TestUtils.clearAssertionInformation(); + addTestConfiguration(TEST_NAME1, new TestConfiguration(TEST_CLASS_DIR, TEST_NAME1)); + } + + // These tests result in dense output + @Test + public void testTransformBagOfWords() { + runTransformTest(TEST_NAME1, ExecMode.SINGLE_NODE, false, false); + } + + @Test + public void testTransformBagOfWordsError() { + runTransformTest(TEST_NAME1, ExecMode.SINGLE_NODE, false, false, false, true, false, false); + } + + @Test + public void testTransformBagOfWordsPlusRecode() { + runTransformTest(TEST_NAME1, ExecMode.SINGLE_NODE, true, false); + } + + @Test + public void testTransformBagOfWords2() { + runTransformTest(TEST_NAME1, ExecMode.SINGLE_NODE, false, true); + } + + @Test + public void testTransformBagOfWordsPlusRecode2() { + runTransformTest(TEST_NAME1, ExecMode.SINGLE_NODE, true, true); + } + + // AmazonReviewDataset transformation results in a sparse output + @Test + public void testTransformBagOfWordsAmazonReviews() { + runTransformTest(TEST_NAME1, ExecMode.SINGLE_NODE, false, false, true); + } + + @Test + public void testTransformBagOfWordsAmazonReviews2() { + runTransformTest(TEST_NAME1, ExecMode.SINGLE_NODE, false, true, true); + } + + @Test + public void testTransformBagOfWordsAmazonReviewsAndRandRecode() { + runTransformTest(TEST_NAME1, ExecMode.SINGLE_NODE, true, false, true); + } + + @Test + public void testTransformBagOfWordsAmazonReviewsAndDummyCode() { + // TODO: compare result + runTransformTest(TEST_NAME1, ExecMode.SINGLE_NODE, true, false, true, false, true, false); + } + + @Test + public void testTransformBagOfWordsAmazonReviewsAndPassThrough() { + // TODO: compare result + runTransformTest(TEST_NAME1, ExecMode.SINGLE_NODE, false, false, true, false, false, true); + } + + @Test + public void testTransformBagOfWordsAmazonReviewsAndRandRecode2() { + runTransformTest(TEST_NAME1, ExecMode.SINGLE_NODE, true, true, true); + } + + @Test + public void testNotImplementedFunction(){ + ColumnEncoderBagOfWords bow = new ColumnEncoderBagOfWords(); + assertThrows(NotImplementedException.class, () -> bow.initMetaData(null)); + } + + //@Test + public void testTransformBagOfWordsSpark() { + runTransformTest(TEST_NAME1, ExecMode.SPARK, false, false); + } + + private void runTransformTest(String testname, ExecMode rt, boolean recode, boolean dup){ + runTransformTest(testname, rt, recode, dup, false); + } + + private void runTransformTest(String testname, ExecMode rt, boolean recode, boolean dup, boolean fromFile){ + runTransformTest(testname, rt, recode, dup, fromFile, false, false, false); + } + + private void runTransformTest(String testname, ExecMode rt, boolean recode, boolean dup, boolean fromFile, boolean error, boolean dc, boolean pt) + { + //set runtime platform + ExecMode rtold = setExecMode(rt); + try + { + getAndLoadTestConfiguration(testname); + fullDMLScriptName = getScript(); + + // Create the dataset by repeating and shuffling the distinct tokens + String[][] columns = fromFile ? readTwoColumnStringCSV(DATASET_DIR + DATASET) : new String[][]{{"This is the " + + "first document","This document is the second document", + "And this is the third one","Is this the first document"}, {"A", "B", "A", "C"}}; + String[] sentenceColumn = columns[0]; + String[] recodeColumn = recode ? columns[1] : null; + if(!fromFile) + writeStringsToCsvFile(sentenceColumn, recodeColumn, baseDirectory + INPUT_DIR + "data", dup); + + int mode = 0; + if(error) + mode = 1; + if(dc) + mode = 2; + if(pt) + mode = 3; + programArgs = new String[]{"-stats","-args", fromFile ? DATASET_DIR + DATASET : input("data"), + output("result"), output("dict"), String.valueOf(recode), String.valueOf(dup), + String.valueOf(fromFile), String.valueOf(mode)}; + if(error) + runTest(true, EXCEPTION_EXPECTED, DMLRuntimeException.class, -1); + else{ + runTest(true, EXCEPTION_NOT_EXPECTED, null, -1); + FrameBlock dict_frame = readDMLFrameFromHDFS( "dict", Types.FileFormat.CSV); + int cols = recode? dict_frame.getNumRows() + 1 : dict_frame.getNumRows(); + if(dup) + cols *= 2; + if(mode == 0){ + HashMap res_actual = readDMLMatrixFromOutputDir("result"); + double[][] result = TestUtils.convertHashMapToDoubleArray(res_actual, Math.min(sentenceColumn.length, 100), + cols); + checkResults(sentenceColumn, result, recodeColumn, dict_frame, dup ? 2 : 1); + } + + } + + + } + catch(Exception ex) { + throw new RuntimeException(ex); + } + finally { + resetExecMode(rtold); + } + } + + private String[][] readTwoColumnStringCSV(String s) { + try { + FrameBlock in = readDMLFrameFromHDFS(s, Types.FileFormat.CSV, false); + String[][] out = new String[2][in.getNumRows()]; + for (int i = 0; i < in.getNumRows(); i++) { + out[0][i] = in.getString(i, 0); + out[1][i] = in.getString(i, 1); + } + return out; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @SuppressWarnings("unchecked") + public static void checkResults(String[] sentences, double[][] result, + String[] recodeColumn, FrameBlock dict, int duplicates) + { + HashMap[] indices = new HashMap[duplicates]; + HashMap[] rcdMaps = new HashMap[duplicates]; + int frameCol = 0; + // even when the set of tokens is the same for duplicates, the order in which the tokens dicts are merged + // is not always the same for all columns in multithreaded mode + for (int i = 0; i < duplicates; i++) { + indices[i] = new HashMap<>(); + rcdMaps[i] = new HashMap<>(); + for (int j = 0; j < dict.getNumRows(); j++) { + String[] tuple = dict.getString(j, frameCol).split("\u00b7"); + indices[i].put(tuple[0], Integer.parseInt(tuple[1])); + } + frameCol++; + if(recodeColumn != null){ + for (int j = 0; j < dict.getNumRows(); j++) { + String current = dict.getString(j, frameCol); + if(current == null) + break; + String[] tuple = current.split("\u00b7"); + rcdMaps[i].put(tuple[0], Integer.parseInt(tuple[1])); + } + frameCol++; + } + } + + // only check the first 100 rows + for (int row = 0; row < Math.min(sentences.length, 100); row++) { + // build token count dictionary once + String sentence = sentences[row]; + HashMap count = new HashMap<>(); + String[] words = tokenize(sentence, false, "\\s+"); + for (String word : words) { + if (!word.isEmpty()) { + word = word.toLowerCase(); + Integer old = count.getOrDefault(word, 0); + count.put(word, old + 1); + } + } + // iterate through the results of the column encoders + int offset = 0; + for (int j = 0; j < duplicates; j++) { + // init the zeroIndices with all columns + List zeroIndices = new ArrayList<>(); + for (int i = 0; i < indices[j].size(); i++) { + zeroIndices.add(i); + } + // subtract the nnz columns + for (String word : words) { + if (!word.isEmpty()) { + zeroIndices.remove(indices[j].get(word)); + } + } + + // compare results: bag of words + for(Map.Entry entry : count.entrySet()){ + String word = entry.getKey(); + int count_expected = entry.getValue(); + int index = indices[j].get(word); + assert result[row][index + offset] == count_expected; + } + for(int zeroIndex : zeroIndices) + assert result[row][offset + zeroIndex] == 0; + offset += indices[j].size(); + // compare results: recode + if(recodeColumn != null){ + assert result[row][offset] == rcdMaps[j].get(recodeColumn[row]); + offset++; + } + } + } + } + + public static void writeStringsToCsvFile(String[] sentences, String[] recodeTokens, String fileName, boolean duplicate) throws IOException { + Path path = Paths.get(fileName); + Files.createDirectories(path.getParent()); + try (BufferedWriter bw = Files.newBufferedWriter(path)) { + for (int i = 0; i < sentences.length; i++) { + String out = sentences[i] + (recodeTokens != null ? "," + recodeTokens[i] : ""); + if(duplicate) + out = out + "," + out; + bw.write(out); + bw.newLine(); + } + } catch (IOException e) { + e.printStackTrace(); + } + } +} diff --git a/src/test/resources/datasets/amazonReview2023/Digital_Music_Text_Head2k_With_RCD_Col.csv b/src/test/resources/datasets/amazonReview2023/Digital_Music_Text_Head2k_With_RCD_Col.csv new file mode 100644 index 00000000000..39765474dd5 --- /dev/null +++ b/src/test/resources/datasets/amazonReview2023/Digital_Music_Text_Head2k_With_RCD_Col.csv @@ -0,0 +1,2000 @@ +"If i had a dollar for how many times I have played this cd and how many times I have asked Alexa to play it, I would be rich. Love this singer along with the Black Pumas. Finding a lot of new music that I like a lot on amazon. Try new things.",i +awesome sound - cant wait to see them in person - always miss them when they are in town !,L +This is a great cd. Good music and plays well. Seller responded back very quicky and received it within 3 days,s +"These are not real German singers, they have accents. It is nothing what they advertised it. Music stinks.",k +"I first heard this playing in a Nagoya shop and fell in love with the remix of Ke$ha's ""Tik Tok"" (Ke$ha and Lady Gaga are EVERYWHERE in Japan), which then morphed into several other recent pop hits. When the salesgirl handed me the CD, I was pleased to see that it actually included thirty pop songs by the likes of Lady Gaga, Usher, La Roux, Katy Perry, Black-Eyed Peas, Ke$ha, etc. I researched more after I got home, and discovered that this Japanese label (Manhattan Records) exists solely to put out rerecorded and remixed dance CDs based on recent pop hits, the key word being rerecorded. I guessed as much as I was listening to the CD in-store; several of the soundalikes do a great job, others are just so-so. It's great for workouts, though, since the tracks fade one into another with no downtime (the bad side is that if you've got in in shuffle, the beginning of the next song, which is often in a different key, is kind of an awkward fade-out when not played in sequence).",R +"I just saw ""Star Wars: L'attaque des clones"" last night (in Ste-Foy, Québec) at a sold-out show (it was playing in English at the theatre next door but I opted for the French version instead). First of all, the theatre itself was an experience: an immense screen that looked as large as an IMAX screen, intense surround THX sound, and the digital picture (even at that size) was amazingly crisp, the colours vibrant, and the sound effects were ear-splitting and exhilarating (especially the moments of silence that preceded the sonic depth charge explosions!)I have seen the other four Star Wars movies but avoided reading publicity and reviews for ""Attack of the Clones"" in order to enjoy every moment without worrying over a particular critic's faults with the film. This film is much darker than ""The Phantom Menace"" could have ever aspired to be. Lucas has moved from stilted, juvenile scriptwriting on ""TPM"" to (still) stilted, mythological, more adult scriptwriting on ""Attack of the Clones."" Here we see Anakin as a headstrong, angry, frequently obnoxious young adult who loves backtalking, disobeying orders, and getting his way--a most dangerous thing when you are destined to become the most powerful Jedi of all time.The dialogue is very politically minded (I had to wonder at the ""separatist"" politics of the film, being in Québec) and the romance scenes feel wooden and awkward, but Natalie Portman and Hayden Christensen definitely have onscreen chemistry. (I have since read that Hayden purposely would flash a thumbs-up at the end of his takes so Lucas would have to reshoot the kissing scenes...I did get a laugh out of that.)Jar Jar Binks has (thankfully) been reduced to a few mere minutes of screentime, in which he manages to be (awkwardly) regal and somewhat less annoying that in Episode I. My favourite part of the ""Star Wars"" universe has always been the astonishing variety of aliens and alien languages...when I was a kid I wanted to be the one that came up with all the cool sounding languages spoken in the film (I still do as a matter of fact).As usual, Natalie Portman's costumes are to die for, incorporating Asian and European fashion influences while still managing to be gracefully fresh and exotic. Only one I could have done without was the black leather getup ---elbow-high leather gloves and a black leather bustier, yet she's telling Anakin she's not interested in a romantic relationship...riiiight (Curiously enough, Anakin wears black leather as well...and if you listen carefully you can hear a whole lot of leather creaking onscreen). Portman certainly runs around baring her abs-of-steel midriff for much of the film (along with being very braless in a very tight white jumpsuit, which I found tasteless and unnecessary...I mean, the costumers spent the last film and part of this one coming up with these gorgeous gowns and now they're raiding Lara Croft's closet?)The CGI imagery was breathtaking, especially the chase through Coruscant, the clone army and all of the alien landscapes. John Williams' score incorporates themes from all of the previous ""Star Wars"" soundtracks, including the addition of a new love theme, ""Across the Stars."" The fight scenes were white-knuckle intense and had me on the edge of my seat. The battle scene with all of the Jedis of all different species was *awesome,* as was Mace Windu's (Samuel L. Jackson) purple lightsaber...a first in ""Star Wars"" history. And Yoda: we finally see him in action and understand why he is the greatest living Jedi. Another first: Yoda is completely computer-animated, and is extremely lifelike.The French translation of the film was almost an exact translation from the English...I actually found that some of the dialogue flowed more smoothly in French. Understanding some of the alien accents (especially the metallic robot ones) was a bit of a challenge for me, but it adds to the fun.Lucas has effectively planted the seeds for the next and final chapter in the ""Star Wars"" saga, redeeming himself from the sometimes mediocre acting, wooden dialogue and annoying creatures from ""The Phantom Menace"".Episode II is full of treachery, death, assassination attempts, massacres, and battles between good and evil where right and wrong are sometimes obscured. Unlike Jake Lloyd, the child actor who played Anakin in ""The Phantom Menace,"" in Hayden we see Anakin torn between his inner feelings and his obligations as a Jedi...his romance with Padmé, the (implied) massacre of the Tusken camp, and his defiance, all pointing to his shift to the Dark Side of the Force.The film is rated PG, but it does contain some very intense moments: onscreen kissing and sexual tension of sorts, a decapitation, mutilation, creepy space caterpillars, a gladiator-style execution arena, a massacre and other intense battle scenes and that might be disturbing to younger children. But overall it is a very worthy addition to the ""Star Wars"" universe. Can't wait until Episode III!",F +"This is without a doubt my favorite CD, and I may be close to wearing it out before the end of the month LOL Personally, I never looked at the song list, just popped it in the player and let 'er rip. I sat back and was just amazed at the choices Ronnie made to put on this CD. Highly recommend it to any fan of his. And if you're not, try it, you will be !!!!!",C +Sounds great,u +This disk is fantastic I wanted the record but recieved a couple bad ones so I purchased this i was not disappointed,K +"This is a fantastic collection of music superb musicians all Mark Soklin keyboards, Paul Soclow bass, Romero Lubambo guitar, Duduka Fonesca drums, Claudio Roditi trumpet, on Sonhos a previous reviewer said musicians were not listed on the CD these people inspire me to play they are all amazing I am a guitarist and listen to lots of music but this record is spectacular",h +There are two separate CDs one is called sand in the vasaline one is called popular favorites that is disc two although disc two actually lists all the tracks from sand in the vasaline it is a separate disc,T +It exceeds my expectations I wish all retailers where as great as mayan,o +"astoundingly beautiful, haunting, moving Chorale !!",T +Fabulous Cd well worth purchasing...,x +Very Good CD All the original songs not redone love this CD! Well Reccomended!!! A lot of these songs I grew up with!,j +Great cd and what's nice its all of the original artists. I f you lived during this era you will love this!,N +"With movies taking over Elvis' post-army career, their soundtracks, naturally, became the main focus of his album release schedule. The annual barrage of soundtrack LPs and EPs would yield enough space for just one new studio LP and one or two singles per year in his busy release schedule. ""Elvis is Back!"" (1960), ""Something for Everybody"" (1961), ""Pot Luck"" (1962), plus the ""His Hand in Mine"" gospel album (released at Christmas, 1960 and a steady catalog seller), were all well received best-sellers. So, when Elvis entered RCA's Studio B in Nashville in May of 1963, it was expected by fans to be for the purpose of recording a new non-movie LP to follow up the previous year's ""Pot Luck with Elvis"" album. The big summer '63 single (Devil in Disguise) whet the fanbase appetite for a new studio LP, and then ... a Greatest Hits set (Elvis' Golden Records, Volume 3) was issued, followed by yet another soundtrack album (""Fun in Acapulco""). The recordings that made up what came to be known as ""The Lost Album"" would be sliced and diced and served up as various singles A-sides and B-sides, EP filler (""Tickle Me""), soundtrack album 'bonus songs,' and patched together album fodder (""Elvis for Everyone!"") over the following five years. And it's a plain shame, because it didn't have to be that way. ""Golden Records Volume 3"" could've / should've been held back for summer '64 release since RCA didn't see fit to issue a ""Viva Las Vegas!"" soundtrack album, despite the runaway success of the movie, leaving a wide-open space in the summer '64 schedule -- the perfect slot for ""Golden Records Volume 3."" After ""Kissin' Cousins"" (March '64) another new Elvis LP wouldn't turn up until the ""Roustabout"" soundtrack was released in late November. A more than worthy follow-up to the Top 5 success of ""Pot Luck,"" the so-called Lost Album would have met with great success in 1963, a time when Elvis' recording career momentum could surely have used a boost. Sales of the recent ""It Happened at the World's Fair"" soundtrack had been tepid, by Elvis' standards. More importantly, 'Lost' was / is a fine stand-alone album. As was the ongoing pattern, 15 songs had been recorded: Two singles A-sides (Devil in Disguise and the planned, but ultimately unissued single Memphis, Tennessee), an immediate B-side for 'Devil' (Please Don't Drag That String Around), and a twelve-track LP deftly mixing pop, rock, love ballads, and hints of Elvis' blues and country roots. Some 60 years down the line, it still adds up to a wonderfully eclectic collection from the guy who once said, ""I sing all kinds,"" and he was neither boasting nor kidding. Four shining stars and definitely recommended.",M +"Import version, basically the same as the domestic release but featuring a better cover (imho). Released originally to accompany / capitalize on the release of his autobiography. And in that sense, it's all good, I suppose, although this material has been collected and re-collected numerous times. But if you're looking to have just one Eric Clapton retrospective in your collection, this just may be the one for you. Disc One covers the various group and early solo years. Disc Two centers in on the Warner Brothers releases. And not surprisingly, the first disc is the really interesting one for classic rockers. The second cd has a lot of big singles and radio hits; for the most part, however, they're not as strong as the savage young Eric. Five stars for the casual fan, four stars for the rest of us. Definitely recommended.",B +"Daylight Again began as a duo project for Stephen Stills and Graham Nash, as David Crosby was in the throes of his legendary and debilitating drug addiction. Atlantic Records passed on the duo concept and suggested that Stills and Nash should somehow work Croz into the project. It was the group's last best hope. Sales for the CSN trio's third LP together would surely outpace a SN duo release. They did, and it did, and all was well, for a while anyway. The album produced a pair of genuine hit singles (Nash's Wasted on the Way, Stills' Southern Cross) that pushed it up into the top ten charts and on to platinum sales certification. The Cali soft-rock sound that stoked the album was right in the pocket for the times, and the Crosby contributions (Delta, Might as Well Have a Good Time) felt right in the mix, never mind that Art Garfunkel and Timothy B. Schmidt were all over the vocal harmonies throughout the project. It was the last time an original CSN collection would be so well received. Four stars from me and highly recommended.",c +"What would the Band be like without Robbie Robertson? People wanted to know back in the early '80s. Well, if Live in Tokyo 1983 is any indication, they'd have been a kick-butt Americana outfit that still had a whole lotta life left in them. Playing with the Cate Brothers Band, the Band mixed classic rock covers alongside familiar J.R. Robertson written favorites to create a wildly entertaining setlist that aimed to -- and succeeded at -- pleasing their decades long followers. Levon Helm, Richard Manuel, Garth Hudson, Rick Danko. They never rested on their laurels; along with the Cate Brothers, they give it their all. If you can find this two-disc set at a reasonable price, and you were ever a big Band fan, don't hesitate. Buy it, you won't be disappointed.",B +"The follow-up to his ""The Street Was Always There"" collection, and basically, it's more of the same. A journeyman's collection of folk and folk-rock covers (mostly) delivered in Andersen's smooth, thoughtful style. If you were a fan of 'Street,' then ""Waves"" will be right in your wheelhouse. Three-and-one-half stars and a mildly enthusiastic recommendation from this longtime fan. I much prefer him in singer / songwriter mode rather than as an interpreter of other writers' works.",K +"Eddie and David Brigati's lost masterpiece. Why hasn't this classic Rascals side-bar story been reissued on CD? It's easily the equal of ""Felix Cavaliere"" / ""Destiny,"" and far outshines ""Treasure."" That something as good as this remains unattainable in today's music marketplace is positively mindboggling. I bought the LP back in 1976 (and I still own and play it!) after hearing the title track played by the late, great Scott ""Scotso"" Muni on the late, lamented WNEW-FM 102.7, where rock lived in those days. While it was the title track that drew me in at first, the entire album grabbed hold of my musical soul and has never let go -- some 46 years on. It's that good, yes! C'mon, Rhino, this one's right in your Rascals lovin' wheelhouse. Make ""Lost in The Wilderness"" available for people to fall in love with again -- or for the very first time. Thanks for all the great music, Eddie & Dave, I highly recommend this one to everyone who loves music made with genuine care and feeling. Prepare to be moved!",s +"Another Chicago retrospective, this time an import. Trouble is the same as ever. The Columbia recordings far outshine the Warners years, leaving their '80s output seeming incredibly limp and middle-of-the-road. Once they were an adventurous and risk-taking creative juggernaut embraced by both the AM-radio singles market and the FM-classic rock format. By the '80s, however, Chicago had devolved into an Adult Contemporary radio power ballad machine, becoming bland, less significant, and ever more meaningless with each new release. Compilations such as this one drive that sad fact home all too often. Three stars for the Columbia output featured. Minus two stars for the mindless power ballad inclusions.",K +"As a rule I tend to really enjoy covers discs. Not this go 'round. Does neither Tom Petty nor Lucinda Williams justice. A good idea on paper, maybe, but poorly executed. Sorry, but this completely misses Petty's much loved, much missed rock 'n' roll heart. Recommended for diehard Lu fans. Most Petty fans, like me, will find it lacking.",m +"This three-disc import mini-box makes the near-perfect companion to the two-disc ""The Fever The Remastered Epic Recordings"" compilation (2017). Following hot on the heels of the band's masterwork (1978's ""Hearts of Stone""), the first Mercury album (""The Jukes"") may have seemed kind of hit-and-miss upon release, but it opens on a high note with a pair of near-classics (All I Want is Everything, I'm So Anxious) and closes with a stirring standout (Vertigo); the songwriting (mostly by Billy Rush) is never less than interesting. The follow-up (""Love Is a Sacrifice"") upped the ante even further with the emergence of Rush and Southside (John Lyon) as production partners. In addition to the classic title track, the album contains what would become live Jukes concert staples (Love When It's Strong, Restless Heart, On the Beach), as well as the band's most beautiful, if heart-wrenching ballad (It Hurts). The third and final Mercury release (""Reach Up And Touch The Sky"") was originally a two-LP live set. Here, it's all on one disc and it still kills: Hits, misses, and covers, all pushed along by Southside's vocals, the great core band, and those hot Jukes horns, 'til everyone goes home sweaty, smiling and happy. The original outer LP artwork is replicated for each of the three discs, with the rest contained within the mini-box's booklet, along with credits, Malcolm Dome's interesting and informative notes, and single picture sleeve replicas. The Mercury Years is a must-have for long-time Jukes fans ... like yours truly.",H +"A fun listen! Although out of date and overpriced on Amazon, really enjoyed Travolta's singing and many of the lyrics were interesting and useful. Road to Freedom and Way to Happiness were quite useful to me. Music a bit out dated, would love to hear it rerecorded. Recommended.",T +"Arrived quickly, good price",j +"This is a remarkable book filled with great moments showing how God uses His strength and power for our welfare and to further His divine purpose.I enjoyed the honesty, integrity, fortitude, he showed in his response to life circumstances. Dr. Stanley’s young life, upbringing, dreams, goals, and aspirations are also nicely highlighted. I appreciate how He interacts with others, those standing with him, and those who choose to go against him.Life was not sweet or kind every day, so we see Dr. Stanley develop strength, grit, and determination.Life kept knocking him down, as a child and tried again later in his life, so we see him learn to stand his ground, lean on God, listen for inspired directions, and stand firm in His faith.Dr. Stanley comes off as a interesting man, with a true and strong heart for God. This is a soothing and moving book to read. This book has lots of great examples of how to walk an enlightened path of faith, and it is A book that I highly recommend.",O +Great CD,v +This is for my husband's 50th high school reunion. We'll see how they like it.,v +My niece loved it for her 21st birthday!!!!!!,i +Nice,s +I miss Dana Key. I’ve had most of his D&K music and “the Journey” for years and never even knew he had another solo project. I ran across this album on YouTube and went looking for the CD - found it used and cheap on Amazon. Glad I did - this is good listening.,k +I love this CD. It’s been a long time since I have found a CD I love. Ed Sheeran has different kinds of music on the same CD. His style varies and makes it so interesting. I am a middle aged woman and I am obsessed with Ed Sheeran’s music.,L +"Great deal, sounds terrific. Fast shipping",G +"Mostly old standards with some new ones, very pleased with this album.",a +Not perfect but a really good album worth while having.,z +I love this... It was signed by the entire group. I love the music and book.,O +I collect the Elvis Legacy CD's and this has got to be one of his best. I love how Sony/ Legacy does these CD's with the booklets and artwork. The price is very reasonable also!,T +"Good price, quick ship. Thanks!",R +A great production. I loved it. It is great for kids to listen too and make the scenes come alive in their head. Great alternative to TV.,I +Package was fine Item case came broken and it was suppose to be a gift.,z +There’s nothing to dislike. Great music to unwind and sing along with the lyrics and/or tunes of the hymns of the Christian faith.,k +"If you know this song, you MUST have a copy. Excellent cd; great band. Will bring back wonderful memories.",M +Peter Hollens is a truly fantastic vocalist. The singers he has accompanying him on several tracks are also great. The album contains a nice selection of his work.,x +"howdy y'all,i use this for background music when i am reading. it's quite nifty as foreground music, too! [*grin*] if you like harpsichord music, you will likely enjoy this.take care,lee",] +PERFECT,O +Glad to hear his other music beside the one or two they used to play on the radio back then.,t +I only new him by just one songs. The man put out a lot of great ones and glad I brought the cd.,R +I am used to Redd Foxx when had a red vinyl This is mostly his stand up comedy at night clubs,L +Great music,f +The finest voice I know. As usual her band is tight and the music is perfection! She has the chops!,a +love the eagles,[ +"When music was really good, and you could sing along with no swearing words. The good Ole days",m +Relaxing music,V +"This is a two-disc set that contains bonus tracks. Refer to my comment to this review for tracks.DISC ONE: The musicians on the first four tracks were not aware that the tape was rolling and this was never intended to be anything more than a memento of the party. However,it was released as an album on September 28, 1973.The first four tracks of the Miles Davis set were recorded live at the Persian Room in the Plaza Hotel in NYC on September 9, 1958. The sextet features Davis on trumpet, Cannonball Adderley on alto sax, John Coltrane on tenor sax, and a rhythm section led by Bill Evans on piano, Paul Chambers on bass and Jimmy Cobb on drums. This is about six months before the same sextet recorded Kind of Blue.Tracks 5 through 7 were recorded at the Spotlight Lounge in Washington D.C. on November 1, 1958, and the remaining tracks on this disc were recorded Birdland in NYC on January 3, 1959. On all of these bonus tracks Red Garland replaces Evans on piano.DISC TWO: Tracks 1 through 7 were on the original album and also recorded at the Persian Room in the Plaza Hotel in NYC on September 9, 1958. It features Ellington's orchestra that was comprised of Clark Terry, Shorty Baker and Cat Anderson on trumpet, Ray Nance on trumpet and violin, Quentin Jackson, Britt Woodman and John Sanders on trombone, Jimmy Hamilton on clarinet and tenor sax, Russell Procope on alto sax and clarinet, Johnny Hodges on alto sax, Paul Gonsalves on tenor sax, and Harry Carney on baritone sax, bass clarinet and clarinet. The rhythm setion is led by Ellington on piano, Jimmy Woode on bass and Sam Woodyard on drums.Track 5 features Jimmy Rushing on vocals, and track 6 features Billie Holiday on vocals backed by non-Ellington personnel Buck Clayton on trumpet, Mal Waldron on piano and unidentified bassist and drummer.Tracks 8 through 13 feature the main Ellington Orchestra (minus Rushing) and was recorded live at Salle Pleyel, Paris, France on November 20, 1958.",W +"Like the first collaboration between Evans and Manne on the 1962 album titled Empathy, this album has a completely different feel from Evans' other work. Most of the album is uptempo, and while harmony is still Evans' focus, melody and rhythm are given equal attention. I am sure that much of this is due to Shelly Manne's approach to drums, which is always creative), and Eddie Gomez' bass.The sound samples on this page convey the music far better than I can in words, so let those be your guide. I will say that if you are a drummer this is definitely an album you want to dissect. Same for bassists. Gomez is right up there with Scott LaFaro in my opinion. Not a clone, by any means, but still an excellent fit to Evans' playing (which is reinforced by future albums they would do together). Evans is legendary for giving his rhythm section space to stretch and this album is no exception. The breaks and solos by both Manne and Gomez are masterful.This album was recorded for Verve in a single October 4, 1966 session at Rudy van Gelder's Englewood Cliffs, NJ studio. According to some sources (including the liner notes on the LP) there were multiple takes, so the session was probably a trying day. However, each track sounds like spontaneous, effortless art to my ears, so whoever did the post production must have added a lot to the album too.",F +"At a mere twenty-seven minutes this album is brief. However, every note counts. At the time of this review the album seems to be credited to drummer Philly Joe Jones. It's the brainchild of Elmo Hope. In fact, each of the six songs are Hope's compositions:A1 Hot Sauce 3:35A2 When The Groove Is Low 4:58A3 De-Dah 4:26B1 Abdula 3:42B2 Freffie 3:36B3 Stars Over Marakesh 6:43If you are unfamiliar with Hope's work you are in for a treat. For one thing, despite being classified as a bebop pianist, he is nothing like Bud Powell, Al Haig, Monk, Dodo Marmarosa, etc. Actually, he is closer to Marmarosa than the others mentioned. Moreover, he spent a lot of time in both the NYC and Los Angeles scenes, so he absorbed both styles. Finally, there is a heavy touch of blues, as well as classical in his playing.He is backed on this album by Paul Chambers on bass and Philly Joe Jones on drums. Both are perfect for the arrangements and compositions on the album. I am very familiar with Chambers (especially his work with Red Garland), and am greatly influenced by Jones.This was recorded for the Celebrity label in NYC in 1961. I was unable to nail down an exact date or studio. I suspect that this was recorded in a single session since many albums (and especially ones as short as this one) were one day affairs. I would appreciate any additional information, which you can place in a comment.",W +"This is another of my vinyl treasures because it features Waldron solo. Another big plus is the sound quality is far better than I expected considering my LP is in Good to VG condition. Every note is crystal clear. This is especially impressive considering that this was recorded live.There are only four tracks, but the album still clocks in at a respectable 37:08. Granted, the LP could have held more, but I am not complaining because this is the only recording I own of Waldron solo.Since there is no track list, I am including it here: A1 Mingus Lives 7:48A2 Snake Out 10:42B1 Tensile Structure 7:54B2 Here, There and Everywhere 10:44The entire album is comprised of Waldron compositions. This is the second tribute album he recorded for former bandleaders. The first was Blues for Lady Day (also solo) in 1972 and this one. This was recorded for the Enja label at the Chapati Club in Spa, Belgium on February 28, 1979.",E +"This is not an essential album, but does have some excellent arrangements thanks to Roger Neumann's tenor and soprano sax, and flute. His addition to the ensemble adds an extra dimension to Anita's vocals over and above her typical jazz piano trio backing.Ao vivo means live in Portuguese; this album was recorded live for the Estudio Eldorado label at the 150 Night Club located in the Maksoud Plaza Hotel in São Paulo, Brazil on August 21 and 22, 1984. Anita's voice on this album is still in good form, and her vocals are delivered with her trademark rhythmic intensity. The sound quality is above average.In addition to Neumann she is backed by Harold Danko on piano, Rick Laird on bass and longtime collaborator John Poole on drums. They are tight and back her perfectly. Here are the tracks:A1 Wave 3:36A2 It Had To Be You 3:44A3 Sleeping Bee 3:38A4 Time After Time 4:30A5 Stomping At The Savoy 4:26A6 P-Town 0:48B1 The Man With The Horn 3:28B2 Body And Soul 6:04B3 Lush Life 3:44B4 The Song Is You 3:49B5 Wave 2:01This isn't what I would consider an essential album, but for we fans the recording quality and performances make it a worthwhile buy.",d +"Up front, a good copy of this album in vinyl has excellent sound quality, but I want to disclose that to my ears the arrangements are a little too heavy and a little too busy. I am not going to subtract a star because of it because the musicianship is superb and my personal taste should not enter into the equation.There are some redeeming moments in this album. Specifically, everything that Previn plays is beyond reproach and I think that is something to which any listener would agree. I also loved Bud Shank's flute work on the tracks on which he played that instrument. In fact, the entire wind section (Shank on alto and flute, Jimmy Giuffre on baritone, and Bob Cooper on tenor and soprano) are uniformly excellent throughout. Milt Bernhart's trombone and Shorty Rogers' trumpet were well played, but I think the trombone in particular was a too low for the arrangements, and Rogers was a bit on the busy side.Shelly Manne is the drummer on all tracks, but the bassists are Joe Mondragon ( tracks A1-A3, A5, B1-B3 and B5), with Curtis Counce on the remaining tracks. Jack Marshall is on guitar on tracks A1 and A2, and B1 and B2, with Al Hendrickson on the remaining tracks.The tracks on this album that clocks in at 35:14 are:A1 It's DelovelyA2 PorterhouseA3 Heat WaveA4 40° BelowA5 You Stepped Out Of A DreamA6 ClaudiaB1 You Do Something To MeB2 Call For ColeB3 Everything I've GotB4 Some AnticsB5 It Only Happens When I Dance With YouB6 General ClusterPrevin is the composer for tracks A2, A4 and A6, with Rogers getting composer credits for tracks B2, B4 and B6. The remaining tracks are standards by the usual suspects (Porter, Berlin, Hart, Rogers, Kahn et al.) This album was recorded for RCA Victor at over the course of three sessions at their Los Angeles studios in 1954: March 30 (tracks 3, 5, 9 and 11), June 14 (tracks 4, 6, 10 and 12), and September 14, (tracks 1, 2, 7 and 8.)",O +"This is one of my thrift shop finds (the vinyl was in pristine condition!) This album proves beyond a doubt that Previn's jazz chops are considerable. All songs are penned by Loewe & Lerner, but it's the way Previn arranged them for a jazz piano trio that is a mark of his genius (as well as his and his rhythm sections performance.) There are places where you can hear Ahmad Jamal's influence. Indeed, at one point on A3 (A Toujours) the trio sounded like the Jamal/Crosby/Fournier line-up from their Pershing album. In other tracks it's pure Previn with both Manne's and Mitchell's signature styles.Since there is no track list as I write this, there is what comprises the 38:54 minutes on the album:A1 The Parisians 4:32A2 I Remember I Will 4:30A3 A Toujours 5:38A4 It's A Bore 4:18B1 Aunt Alicia's March 5:01B2 Thank Heaven For Little Girls 4:31B3 Gigi 5:39B4 She Is Not Thinking Of Me 4:09As a drummer the stand out for me is Shelly Manne's brushwork, but I also love Red Mitchell's bass, and, of course, Previn's approach to piano (and arranging). The trio is tight and the way they are all three locked in, yet all three participating, make this as instructive as it is enjoyable.This was recorded for the Contemporary label in their Los Angeles studios on April 7 and 8, 1958. This is West Coast jazz at its best, and a showcase for Previn's versatility.",Q +"I received these as a free review sample. Before you read my observations take the time to check out Tom's very thorough and helpful video review dated April 5, 2016.These are far better suited to home use than camping in my opinion for a number of reasons. Foremost (and why I subtracted a star) is they are not easy to start, even with a lighter, and secondary, they are heavy. For campers a box of these is extremely heavy (for camping) and add almost nine pounds to carrying weight.For starting fire places, wood stoves and grills and even charcoal grills these are excellent. More importantly, these work beautifully in hibachi style grills and they do last the claimed amount of time.Despite my caveat about these being less than optimal for camping, one or two in a backpack are a good hedge against an emergency. Just be prepared to first build a kindling fire and build on that before lighting one of these pucks because you are not going to directly get one started with a firestick.",W +"This album is yet another of my thrift shop finds. The mystery is on the cover it's credited as Piano Solos With Instrumental Accompaniment. I took that as a challenge because there were no hints in the liner notes as to who was backing him. I pretty much suspected Red Mitchell on bass and Shelly Manne on drums, but beyond that had no clue. I managed to track down the dates and personnel on each track, which I will provide below.First, you can listen to the tracks on a larger compilation made up of albums and tracks from the same era: [[ASIN:B008FTAH2O Previn's Touch]]. Run through the sound samples and you will be treated to a display of Previn's jazz chops. Although he had the touch and feel for classical, and the innate sensibility for sound tracks and scores, he was a first rate jazz musician, but his other work (and the fact that he was in the West Coast scene) overshadowed that. This album is evidence that he was the real thing in jazz too.Here is the track list, which is missing at the time I am writing this:A1 Flying Down To Rio 2:30A2 You Are Too Beautiful 3:19A3 When You Wish Upon A Star 2:05A4 London In July 3:25A5 Moonlight In Vermont 3:18A6 Island In The West Indies 3:29B1 How Are Things In Glocca Morra 3:43B2 It Could Happen To You 3:03B3 Honolulu 2:51B4 My Foolish Heart 2:11B5 But Beautiful 3:11B6 Sidewalks Of Cuba 2:24It took some sleuthing to discover who was backing him. All of the tracks were recorded for the Decca label in Los Angeles in April 1955 and March 1956. Here are the sessions and musicians in chronological order:April 1955 has Previn backed by Al Hendrickson on guitar, Red Mitchell on bass and Irv Cottler on drums:Track B1: April 11.Tracks A1,A4 and A6: April 13.Tracks A5, B3 and B6: April 18.March 1956 has Previn backed by Al Hendrickson on guitar, Carson Smith on bass and Shelly Manne on drums:Tracks B2 and B5: March 28.Tracks A2, A3 and B4: March 29.This album was released by Decca as DL 74350 in 1963 and clocks in at 35:35. If you are a pianist or a West Coast jazz fan this is a great album.",D +"This is yet another of my thrift shop finds. What caught my eye is this album contains the iconic track that was captured on video by Bert Stern, [[ASIN:B00003OSU4 Jazz on a Summer's Day]], which is track A4 on this LP. Since the tracks are not listed at the time I am writing this, here they are:A1 Lover Come Back To Me 2:16A2 Back Water Blues 4:33A3 Crazy Love 3:27A4 All Of Me 5:18B1 Backstage Blues 9:10B2 Julie And Jake 8:40The entire album was recorded live at the Newport Jazz Festival on July 6, 1958. All of Side A features Dinah Washington, but Side B contains only instrumentals. The first three tracks of Side A are: Dinah Washington backed by Blue Mitchell on trumpet, Melba Liston on trombone (and arranger), Harold Ousley on tenor sax and Sahib Shibab on baritone sax. The rhythm section is Wynton Kelly on piano, Paul West on bass and Max Roach on drums.Track A4 is a change up with Washington backed by Don Elliott on mellophone, Urbie Green on trombone, Terry Gibbs (and briefly Washington) on vibraphone, with the same rhythm section.Side B is Don Elliott on mellophone, Urbie Green on trombone, Terry Gibbs on vibraphone and the same rhythm section as Side A.",J +"Up front: the sound quality is acceptable, but not up to audiophile standards. I have no complaints, but if you are a discriminating listener with a lot of money in audio gear you may want to look elsewhere.Nine of the ten discs span the period between 1952 and 1959, with one disc from 1981, the year before Art passed. It's difficult to pick a favorite disc, and that is subjective anyway. What I'll do is list the number of tracks on each, when and where the tracks were recorded, and on as many as I can, the musicians on the tracks.Also note that each disc sleeve has a title as well. Those titles do not map to specific albums, although it's easy to extrapolate some albums from which the tracks came by noting the dates, personnel and tracks. I'll list each track in a comment to this review.DISC ONE: 11 tracks.Sleeve title is Diablo's Dance. Recorded in 1957 in Los Angeles. Appears to be Art on tenor, with Red Garland on piano, Paul Chambers on bass and Philly Joe Jones on drums from the sessions that produced Art Pepper Meets The Rhythm Section.DISC TWO. 11 tracks.Sleeve title is Long Ago and Far Away. Recorded in 1956 and 1957 in Los Angeles. Personnel include the following ensembles backing Art:Carl Perkins (piano), Ben Tucker (bass) and Chuck Flores (drums)Russ Freeman (piano), Ben Tucker (bass) and Chuck Flores (drums)Russ Freeman (piano), Leroy Vinnegar (bass) and Shelly Manne (drums)Gerry Wiggins (piano), Red Norvo (vibraphone), Ben Tucker (bass) and Joe Morello (drums)DISC THREE. 12 tracks.Sleeve title is Angel's Wings. Recorded in Los Angeles in 1956 and 1957. Personnel include the following ensembles backing Art:Russ Freeman (piano), Jack Sheldon (trumpet), Leroy Vinnegar (bass) and Shelly Manne (drums)Russ Freeman (piano), Ben Tucker (bass) and Chuck Flores (drums)Gerry Wiggins (piano), Red Norvo (vibraphone), Ben Tucker (bass) and Joe Morello (drums)Carl Perkins (piano), Ben Tucker (bass) and Chuck Flores (drums)DISC FOUR. 13 tracks.Sleeve title is Blues In. Recorded in Los Angeles between 1952 and 1957. Unable to track down the backing musicians.DISC FIVE. 10 tracks.Sleeve title is Pepper Blues. Recorded in Los Angeles between 1952 and 1957. Backing musicians are:Red Garland (piano), Paul Chambers (bass) and Philly Joe Jones (drums)Hampton Hawes (piano), Joe Mondragon (bass) and Larry Bunker (drums and vibraphone)DISC SIX. 4 tracks.Sleeve title is Road Waltz. Recorded in Los Angeles in 1981 (a year before Art's passing). George Cables (piano), David Williams (bass) and Carl Burnett (drums). Art is on clarinet on When You're Smiling.DISC SEVEN. 10 tracks.Sleeve title is Art Pepper Big Band. Recorded in Los Angeles in 1959. Unable to track down the backing musicians.DISC EIGHT. 9 tracks.Sleeve title is Art Pepper & Marty Paich Quartet. This disc is identical to The Marty Paich Quartet Featuring Art Pepper, which was recorded in Los Angeles in 1956 and features Marty Paich on piano, Buddy Clark on bass and Frank Capp on drums backing Art on alto.DISC NINE. 12 tracks.Sleeve title is Spices. Recorded in Los Angeles in 1954 and features the following backing musicians:Jack Montrose (ténor), Claude Williamson (piano),; Monty Budwig (bass) and Paul Ballerina (drums)Hampton Hawes (piano), Joe Mondragon (bass) and Larry Bunker (drums)Jack Montrose (ténor), Claude Williamson (piano), Monty Budwig (bass) and Larry Bunker (drums)DISC TEN. 12 tracks.Sleeve title is With Mel Torme & Marty Paich Quartet. Recorded in Los Angeles in 1959. Some featured musicians include Jack Shelton on trumpet, Victor Feldman on vibraphone, Marty Piach on piano and Mel Torme on vocals.To me this set is a valuable collection of one of the most lyrical alto saxophonists in jazz. Art is among my top three favorite alto players - the other two are Johnny Hodges and Paul Desmond. Please see the comment below for the complete track list, which will give clues about the albums and recording sessions from which they came.",x +"This gem is one of my thrift shop finds. It's based on a 1959 Broadway musical that enjoyed 452 performances at the 46th Street Theatre. The music on this album mimics the play in that it's relatively obscure, but enjoyable. This album is from an era when soundtracks from plays and musicals were very popular.This one has a slight Dixieland feel to it, and because it is performed by a sextet instead of a full orchestra is interestingly arranged. When researching this album an interesting (but unverified) tidbit surfaced: the ensemble did not rehearse - they walked into the studio and 'had fun' creating the tracks that were selected for this album from among the eighteen that were actually performed in acts I and II in the play itself. Here are the tracks on this album and their lengths. The entire album clocks in at 33:36:A1 My Girl Is Just Enough For Me 4:05A2 Just For Once 2:22A3 I Feel Mighty Marvelous 3:54A4 The Right Finger Of My Left Hand 2:27A5 Look Who's In Love 3:30B1 Uncle Sam Rag 3:52B2 Two Faces In The Dark 3:25B3 I'm Back In Circulation 2:45B4 Erbie Fitch's Dilemma 2:01B5 Pick Pocket Tango 4:15That's still almost two thirds of the actual songs from the play. All are catchy and nicely interpreted and performed. My favorite is B5, but I love songs with a tango feel. I cannot begin to describe the music since words are inadequate, but if you like more traditional jazz that is more easy listening and commercial, this is an interesting album and a fun listen.This was originally recorded for the Design Records label in 1959 and features Rex Stewart on trumpet backed by John Bunch on piano, Joe Venuto on marimba, Leonard Gaskin on bass and Mousey Alexander on drums.",N +"This four album set comes on two discs and features Flanagan's first album as a leader, and the rest as either co-leader or sideman. The sound quality is very good. I have come to trust this label for putting together competently remastered sets of albums that are mostly out of print.The tracks are not in chronological order, but they are grouped by the source albums from which they are taken. Also, the albums are as originally issued. There may be re-releases that have bonus tracks, but this set does not.Disc One starts off with an album titled Jazz...It's Magic! that was recorded for the Regent label at Rudy van Gelder's Hackensack, NJ studio on September 5, 1957. It's credited as the Curtis Fuller-Tommy Flanagan Quintet and features Fuller on trombone and Sonny Redd on alto sax, backed by Flanagan on piano, George Tucker on bass and Louis Hayes on drums. The tracks are:1. Two Ton2. Medley (It's Magic/My One And Only/They Didn't Believe Me)3. Soul Station4. Club Car5. Upper BerthRemaining tracks are from the album titled The King and I that was recorded by the Wilbur Harden Quartet in NYC for the Savoy label on September 23 and 30, 1958. Hardin is on trumpet and flugelhorn backed by Flanagan on piano and celeste, George Duvivier on bass and Granville Hogan on drums. Tracks are:6. Getting To Know You7. My Lord And Master8. Shall We Dance9. We Kiss In A Shadow10. I Have Dreamed11. I Whistle A Happy Tune12. Hello Young Lovers13. Something Wonderful14: `Trio Overseas'14. Relaxin' At CamarilloDisc Two starts off Trio Overseas, which is Flanagan's first album as a leader. It was recorded for the Metronome label in Stockholm, Sweden on August 15, 1957. Flanagan is backed by Wilbur Little on bass and Elvin Jones on drums. Tracks are:1. Chelsea Bridge2. Eclypso3. Beats Up4. Skal Brothers5. Little Rock6. Verdandi7. Dalarna8. Willow Weep For MeTracks 9 through 13 are the album titled The Cats that was recorded for the New Jazz and Prestige labels at Rudy van Gelder's Hackensack, NJ studio on April 18, 1957. This album is credited to the Prestige All Stars and features Flanagan as co-leader. The line-up is Idrees Sulieman on trumpet with John Coltrane on tenor sax, backed by Flanagan on piano, Kenny Burrell on guitar, Doug Watkins on bass and Louis Hayes on drums. Tracks are:9. Minor Mishap10. How Long Has This Been Going On?11. Eclypso12. Solacium13. Tommy's Time",E +"This is the first of four collaborations between Peterson and Jackson and it presages the ones to come. At a few seconds short of forty one minutes this album is almost a tease.One of the nice things about this album is it has some of the restraint Jackson used with the Modern Jazz Quartet. However, Peterson's piano is a major contrast to John Lewis' of the MJQ and that contrast makes this album interesting. For one thing, Peterson and bassist Ray Brown were renown for intensity even on ballads, and drummer Ed Thigpen fit right into that as well.Despite my alluding to intensity, the music comes across as relaxed and laid back. They made it sound so easy! When you closely listen the intensity is there under a layer of virtuosity and musicality. This is Jackson and Peterson's trio at their finest in my opinion.Since there is no track list (as I write this) I am providing one for reference:1. On Green Dolphin Street 7:322. Heartstrings 5:433. Work Song 7:354. John Brown's Body 7:49 5. A Wonderful Guy 4:57 6. Reunion BluesThe album was recorded for Verve in NYC over the course of two sessions (September 15 and 18, 1961.) Sadly there were five unissued tracks from the sessions, two of which are logged as 'untitled Peterson originals' that I would have loved to have been included in a reissue. Regardless, the music that is on this album is definitely worthwhile.",r +"This recording was made for the Belgium Philippe Defalle label in November 1979 in Brussles.I subtracted two stars because Baker's vocals are very weak here. His performances during this period were always hit or miss and this was must have caught him on a bad day. His trumpet work, though, does not suffer in the least. He is in full command of that instrument.There is no track list at the time of this review, so I am providing one and the times for each track:01. This is always 7:4902. Sweet Martine 8:2003. Beatrice 6:4904. Deep in a dream 5:5305. Once I loved 4:51The album itself is somewhat pensive, with a laid back feeling among the entire ensemble. The instrumental sections are tight as though rehearsed a few times. I am not sure that was actually the case, but that is one of the key strengths of this album and what makes it so enjoyable. There is a touch of Baker's good days in those parts of the album.Another interesting thing is this does not come across as a hastily thrown together session. The ensemble is fairly large for Baker during this era in his career. He is accompanied by Steve Houben on alto sax, and backed by a solid rhythm section comprised of Dennis Luxion on piano (credited as Denis Luxion) Bill Frisell on guitar, Kermit Driscoll on bass and Bruno Castellucci on drums. In particuar Houben, Luxion and Frisell stand out, while Driscoll and Castellucci are solid on pulse and time. No complaints with any member of the ensemble or the sound quality of this CD.",R +"This is actually a studio album that was put together by Granz as a recorded (and supervised) jam. The line-up is first rate, and has a large alto and tenor saxophone section, fronted by Charlie Shavers on trumpet and backed by a solid rhythm section.Basically, this is along the lines with one of Granz' Jazz At The Philharmonic sessions. Here are the tracks:A Jam Blues 14:39B Ballad Medley 17:21C What Is This Thing Called Love 15:48D Funky Blues 13:26A and B are LP #1 and C and D are the second LP. If you find this set in VG condition the sound quality is superb. As can be seen from the track times each song is an extended rendition of a standard, with the obligatory ballad medley. Funky Blues is a Johnny Hodges composition, and the Ballad Medley is comprised of the following: All The Things You Are, Dearly Beloved, The Nearness Of You, I'll Get By, Everything Happens To Me, The Man I Love, What's New?, Someone To Watch Over Me and Isn't It Romantic?The line-up is a front line featuring Charlie Shavers on trumpet, accompanied by an alto sax section comprised of Johnny Hodges, Charlie Parker and Benny Carter, and a tenor sax section comprised of Flip Phillips and Ben Webster. The rhythm section was led by the Oscar Peterson Trio (Peterson on piano, Barney Kessel on guitar and Ray Brown on bass), and J. C. Heard on drums.This album was originally released on both Mercury and Clef, then by Verve. It was recorded at Radio Recorders in Los Angeles on June 17, 1952 (some discographies cite July 1952.If you are familiar with JATP sessions from this era you already have an idea how this will sound. To my ears it's a fantastic reminder that folks were making some great music sixty-three years ago.",w +"This is a great two album set with two additional tracks from a third album. The interesting thing is tracks 13 through 18 are Burrell's first album as a leader, and the first twelve tracks feature him on vocals as well as guitar.Since there is no track list as I write this I am providing it for reference:1. I'll Buy You A Star2. Weaver Of Dreams3. The More I See You4. I'm Just A Lucky So And So5. A Fine Romance6. Until The Real Thing Comes Along7. The Blues Is Awful Mean8. That Old Feeling9. If I Had You10. Hootchie-Koo11. Afternoon In Paris12. Like Someone In Love13. This Time The Dream's On Me14. Fugue'n Blues15. Takeela16. Weaver Of Dreams17. Delilah18. Blues For Skeeter19. Get Happy20. But Not For MeThe music is what you would expect from Burrell who was a veteran player when his inaugural album was recorded in 1956. He is backed by some of the best NYC musicians of the era, and that, too, indicates the high standard of music on this album. Add in the fact that the sound quality is superb for a CD and this album is a winner all around.Tracks 1 through 12 were recorded for Columbia in NYC in four different sessions between October 18, 1960 and April 11, 1961. These tracks feature Burrell on both guitar and vocals. He has a similar singing voice to Nat King Cole who I am guessing inspired him. It's a pity Burrell didn't sing more on other albums because his voice is excellent.Tracks 1, 2 and 9 were recorded on March 1, 1961 with Burrell accompanied by Bobby Jasper on tenor sax and backed by Tommy Flanagan on piano, Wendell Marshall on bass and Bobby Donaldson on drums.Tracks 3, 4, 7 and 12 were recorded on November 1, 1960 with the same line-up with a few changes on specific tracks. On track 7 Joe Benjamin is on bass and Bill English is on drums.Tracks 5, 6 and 8 were recorded on October 18, 1960 with Burrell accompanied by Bobby Jasper on tenor sax and flute, and backed by Wendell Marshall on bass and Bobby Donaldson on drums.Tracks 10 and 11 were recorded on April 11, 1961 with Burrell accompanied by Bobby Jasper on tenor sax and backed by a rhythm section comprised of Tommy Flanagan on piano, Joe Benjamin on bass and Bill English on drums.Tracks 13 through 18 are the album titled [[ASIN:B004ADWXTY Introducing Kenny Burrell]] that was recorded on May 29 and 30, 1956 for Blue Note at Rudy Van Gelder's Hackensack, NJ studio. Kenny's guitar is backed by Tommy Flanagan on piano, Paul Chambers on bass, Kenny Clarke on drums and Candido Camero on congas.Tracks 19 and 20 are from the album titled Kenny Burrell, Vol. 2, which was a follow-on to Introducing Kenny Burrell. These two tracks were recorded during the same May 29 and 30 1956 sessions with the same line up on track 19. He is solo on track 20.",r +"This album has a WWI theme because the song selection was chosen for their popularity during the 19-teens. Many of the songs are timeless, like track 1, After You've Gone. Others have fallen into obscurity, but when this album was recorded in 1957 many were still in the public's mind since WWI ended less than forty years prior. Note that the last track, Vamp's Blues, was written by Charlie Mariano who was not born until after the war (in 1923). Still, the overall theme of this album, with a distinct West Coast flavor, is what drew me to it.Since there is no track list on this page (at the time of this review) I am including it here:1. After You've Gone 4:492. When Johnny Comes Marching Home 5:133. Deep River 5:404. Till We Meet Again 4:155. K-K-K-Katy 5:276. 'til The Clouds Roll By 2:397. Over There 4:228. Ja Da 4:179. Hello, Central, Give Me No Man's Land 3:5710. Vamp's Blues 7:05The entire album clocks in at 48:14 and you can listen to sound samples on a reissue of this album at [[ASIN:B008C7POJ0 The Vamp's Blues]]. Let those guide you because they convey a lot more about this album than I can possibly put into words.This was recorded in Los Angeles for the World Pacific label at the Radio Recorders studio on December 10 and 11, 1957. Charlie Mariano is on alto sax (and recorder on Deep River), Jerry Dodgion is on alto sax and flute, Victor Feldman is on vibraphone, Jimmy Rowles on piano, Monty Budwig on bass and Shelly Manne on drums.",U +"I discovered this album in near pristine condition at a local thrift shop and it was love at first listen. I have been both an avid Paul Desmond and Modern Jazz Quartet for for decades. While members of the MJQ often were on other albums (both bassist Percy Heath and drummer Connie Kay were on Desmond-led albums), and Milt Jackson was prolific as a sideman with other groups they were a closed unit. Being invited to site in was a signal honor for Desmond, and that they were musically on the same page is evident when listening to this album.Sadly there are no sound samples on this page, although of you scour the web you will find some of the tracks from this album. Desmond's slow, dynamically soft playing is a perfect fit. I am especially impressed with John Lewis on piano and Jackson on vibraphone who seemed to be connected to Desmond on a one of those rare levels that we musicians experience. The rhythm section is absolutely perfect here. This is a major contrast to the Dave Brubeck Quartet rhythm section, which was Desmond's home for nearly twenty years.Also missing from this page is the track listing. I am including it and track times so you know what this album contains:SIDE AA1 Greensleeves 3:10A2 You Go To My Head 7:56A3 Blue Dove/La Paloma Azul 4:55A4 Jesus Christ Superstar 4:54SIDE BB1 Here's That Rainy Day 4:50B2 East Of The Sun 8:25B3 Bags New Groove 5:10Think of this as the MJQ with a special guest (which, of course it is), because John Lewis holds the reins tightly on this live album. However, in this case he did not have to rein in Desmond who was so perfectly suited to the MJQ that one has to wonder why there is only a single collaboration between him and the entire quartet. This is definitely an album to own!",\ +"This album is parts of two separate radio broadcasts - one from the Basin Street East in NYC and the other from the Blue Note Club in Chicago. The sound quality is surprisingly good considering all of the tracks are live and in a live setting. The announcements either add to the ambiance or wreck the album, depending on your point of view. For me I actually like them. You may have other thoughts, but at least you know that a lot of the broadcast has not been edited out.Since there is no track list at the time of this review I am providing one:1. Theme (The Duke) & Introduction2. Stardust3. Gone With The Wind4. Stompin' for Mili5. Out of Nowhere (Incomplete)6. A Minor Thing7. In Your Own Sweet Way8. The Trolley Song9. Introduction & The Duke10. Love Walked In11. Here Lies Love12. All the Things You Are13. Theme (The Duke) And Introduction14. I'm In A Dancing Mood15. The Song Is YouTracks 1 through 12 - the bulk of this album - are from the February 1956 engagement(s) at the Basin Street East. I am not sure of the date or dates and each discography I have checked does not provide exact details. The line-up is the one before the classic Wright/Morello rhythm section joined the quartet. This one has Dave Brubeck on piano, Paul Desmond on alto sax, Norman Bates on bass and Joe Dodge on drums.The last three tracks are from the March 1957 engagement at the Blue Note Club. Joe Morello had replaced Joe Dodge by the time these tracks were recorded and may, in fact, be the first recorded tracks with the Quartet featuring Joe Morello.Some of the tracks on this album are on other live compilations, but not all of them. Frankly this is not what I would consider an essential album, but it is definitely worth it if you are a fan.",V +"This album shares a lot with [[ASIN:B000CCB1PE Keyboard]] because it features Peterson in a duo with a bassist (two, in fact, with both Major Holley and Ray Brown swapping depending on the track). Also, some of the tracks for both albums were recorded during the same session.Since there is no track list at the time of this review I am providing what the CD contains:1. Debut 2:522. They Didn't Believe Me 2:553. Lover, Come Back to Me 2:514. Where or When 3:275. Three O'Clock in the Morning 3:396. All the Things You Are 3:367. Tenderly 3:468. Oscar's Blues 3:149. Little White Lies 3:0910. In the Middle of a Kiss 3:1311. Nameless 2:5512. Two Sleepy People 3:18Total play time is a little over 40 minutes, which was long for 1950 when this album was recorded. Although there are no sound samples, each track sounds like Peterson solo with the underpinnings of bass. When I first listened I though he was solo and playing some remarkable stuff with his left hand. Given his prowess that is not a stretch, but on the first eight tracks the bass is Ray Brown and on the remaining four it's Major Holley. The Peterson/Brown tracks were recorded in NYC for the Clef label during March 1950. I was never able to track down specific dates, but it does place the session a year or so away from Peterson's first trio featuring Brown on bass and Herb Ellis on guitar. The last four tracks were recorded on May 8, 1950 in NYC during the same session that produced Keyboard mentioned above.The mastering on this CD is excellent and audiophile quality. If you are willing to go budget this album is also available in a budget-priced, ten disc set titled [[ASIN:B00KBZ74O4 Oscar Peterson & Friends]]. The sound quality of that one is good, but nowhere near the quality of this CD.",p +"I love this album mainly for the collaboration between Milt Jackson and Miles, but I have to give due credit to McLean's contributions too. Not only is his playing masterful, but Dr. Jackle and Minor March are his compositions. Ray Bryant contributed a composition as well: Changes. The remaining track is a Thad Jones composition that somehow got slipped into the track list.At the time of this review there is no track list on this product page, The album clocks in a scant 31 minutes with the following four tracks:01 Dr. Jackle02 Bitty Ditty03 Minor March04 ChangesRay Bryant is another reason why I love this album. I own a large number of his albums and his playing here does not disappoint. Credit is also due Percy Heath on bass and Art Taylor on drums, both of whom make definite contributions by staying under the music and supporting it. Taylor is one of my favorite drummers from that era simply because he could play any groove and do so unobtrusively. He manages the same here and is locked in with Heath to give the melody instruments a foundation.The album was recorded in a single session at Rudy van Gelder's Hackensack, NJ studio on August 5, 1955 and released the next year by Prestige as PRLP 7034. Some accounts of this session claim Miles' was disgusted with McLean for being high, but that is not evident in the playing.",V +"This album was recorded on January 24, 1980 for the Ornament label in Aachen, Germany (according to notes I researched it was in the home of Friedrich and Gabi Klemme.) The sound quality is a little on the muddy side and the snippets of 'crowd' noise that creep into the recording indicate that this was either a party or a private recital with possibly a dozen or more folks in attendance.The music is what counts and in my growing collection of Donegan recordings that is one thing that never disappoints. Here she is backed by George Linges on bass and Tony Mann on drums. While the title implies a slant towards Ellington, the only tribute to the Duke is the title track. Otherwise, it's a collection of mainly recognizable standards including two medleys that give Donegan room to show her amazing improvisational skills. She can do the seemingly impossible going from one song to the next, often quoting even more songs within the melody.Since there is no track list at the time of this review I am providing it:1. Jumpin' at the Woodside2. Medley: In the Mood/Mack the Knife3. Medley: Sophisticated Lady/Dorothy's Themes4. Smile5. September in the Rain6. Medley: Send in the Clowns/Jesse/You'll Never Walk Alone/If I Loved You/My wayThe album clocks in at 49:41, so there is a decent amount of music. I would not call this 'essential', but if you are a fan it's certainly worth grabbing. If you are a stickler for sound quality, though, you may want to pass this one up.",U +"Most of the tracks on this compilation were recorded in 1947 with Marmarosa as leader. The exception being is six tracks from a 1946 Lucky Thompson album called out below. Marmarosa is in excellent form on every track and if you are a pianist interested in this era of music (heavy bebop influence with his unique approach to the instrument) this set is a treasure trove.Sadly there are no sound samples on this page at the time of this review. Nor is there a track listing. I am going to provide the tracks and their sources, but they will be a little out of order because I am also going to include personnel, locations and other pertinent information to help you identify the context of each track. The album clocks in at a few seconds longer than an hour and eighteen minutes, so there is a lot of Mamarosa's recorded work here.The first ten track are from a Phoenix Jazz LP titled Dodo Marmarosa: Piano Man that was recorded in Los Angeles on July 16, 1947. Mamarosa is backed by Barney Kessel on guitar and Gene Englund on bass. Tracks are:1. Dodo's Bounce2. Opus #53. You Thrill Me4. Compadoo5. I'm In Love6. Bopmatism7. Lover Come Back To Me8. Raindrops9. Smoke Gets In Your Eyes10. EscapeTracks 11 through 14 are from an Armed Forces Radio Service broadcast (11 and 12) and private recordings (13 and 14). 11 and 12 are from February 1946, and 13 and 14 were recorded on an unspecified date in 1947. All are solo and all were recorded in Los Angeles.11. Deep Purple12. Tea For Two13. Tone Paintings I14. Tone Paintings IITracks 15 through 19 and alternate takes (tracks 26 through 30) are from a December 3, 1947 recording session for both the Spotlite and Dial labels in Los Angeles. Marmarosa is backed by Harry Babasin on cello and Jackie Mills on drums.15. Bopmatism16. Dodo's Dance17. Trade Winds18. Dary Departs19. Cosmo Street26. Bopmatism27. Dodo's Dance28. Trade Winds29. Dary Departs30. Cosmo StreetTracks 20 through 25 are from a Lucky Thompson album recorded for the Down Beat label in Los Angeles on September 13, 1946. Thompson is the leader and tenor saxophonist backed by Marmarosa on piano, Red Callender on bass and Jackie Mills on drums.20. Dodo's Bounce21. Dodo's Lament22. Slam's Mishap23. Shuffle That Riff24. Smooth Sailing25. Commercial EyesIf you are studying Marmarosa the value of this set is obvious - especially since many of the source albums are now out-of-print.",l +"You will love Teddy Charles on vibraphone on this album. This was Charles' eighth album as a leader (he was 29 when this was recorded in a single session for Jubilee in NYC on May 29, 1957. It was released later that year under catalog number JGM 1047. It clocks in at 39:47, so it's not super short like so many albums from the era.This is laid back with virtually no vibrato and the same relaxed playing that characterizes Red Norvo. You could not ask for better accompaniment with Oscar Pettiford on bass and Hal Overton on piano. I personally love the fact there there is no drummer because each track showcases Charles, Pettiford and Overton and drums would have been an intrusion. If you are new to Charles, he was a Julliard trained percussionist who was equally skilled on piano, vibraphone and drums.There is no track listing on this page at the time of this review so I am providing it - all Ellington tunes as the title implies:A1 Main Stem 4:45A2 Do Nothing 'Til You Hear From Me 5:06A3 Sophisticated Lady 9:36B1 Don't Get Around Much Anymore 5:57B2 Sherman Shuffle 7:07B3 The Mooch 6:20Overall, the music flows languorously with each member of the trio participating and playing off one another. Make no mistake that Charles is the centerpiece, but he gives ample space to both Pettiford and Overton. Also important (to me) is there no real liberties taken with the melodies of each track. The Duke, I am sure, approved when this album was released.I found my copy in a thrift shop and even on a cheap turntable and fair-to-good condition vinyl this album has reasonable sound quality. A copy in VG to pristine condition played on a decent turntable will be a real treat.",p +"This is one of Garland's late 50s albums he cut for Prestige's Moodsville subsidiary. This album should not be confused with a similar titled one featuring Garland's trio and Eddie Lockjaw Davis titled, [[ASIN:B000000YK4 Moodsville, Volume 1]]. At the time of this review there is no track list. The album contains the following tracks, which clock in at 43:26:A1 I Love You Yes I DoA2 Blues For AnnA3 I'll Never Stop Loving YouB1 And The Angels SingB2 Tain't Nobody's BusinessB3 Bassment BluesEvery track is basically a ballad or slow tempo blues. Great for late evening background (or romantic) music. For me it is not an album I frequently play, although the tracks are in a few playlists on my MP3 player for when I want to wind down and chill.Unfortunately there are no sound samples (at the time of this review) to be found anywhere on Amazon, but you can find most of the tracks if you search the web. If you are familiar with Garland during the late 50s you probably have a good idea how this album will sound. This is especially true since he has Paul Chambers on bass and Art Taylor on drums backing him. The three of them were one of the tightest piano jazz trios from the era and their music here as on other albums has a definite swing and cohesiveness.This was recorded at Rudy van Gelder's Hackensack, NJ studio on November 21, 1958 and released in 1960. If you want to add some slower trio music to your collection this is a prime candidate.",w +"This is a live performance circa 1964 (I have been unable to nail down the exact date). The even is the After Hours Concert at the Esquire Theater in Los Angeles and is much wider than Just McCann. This concert (and to a degree album) has some historic significance because McCann introduces Monty Alexander whose debut album was recorded at the same concert - both for the Pacific Jazz label. Alexander 'borrowed' McCann's rhythm section that was comprised of Victor Gaskin on bass and Paul Humphrey on drums.Aside from the playing this album has wonderful ambiance from the crowd. I will say that if you dislike crowd noise and interaction you are NOT going to like this album. I thought it added to the listening pleasure, but have to admit that it's a bit much in spots. Otherwise the album has excellent sound quality.At the time of this review there are no sound samples or a track list on this page. There are entire tracks floating around on the web that are easy to search out, and the track list is:A1 El Soulo 3:16A2 Lavande 6:40A3 Spanish Onions 8:50B1 Get Them Grits 4:30B2 I Am In Love 5:30B3 Arabella 5:43B4 Maxie's Farewell 5:15With the exception of Cole Porter's I Am In Love, the album contains all of McCann's own compositions.",v +"Note: there are a few albums by this title, one of which is a compilation. I am reviewing the vinyl LP titled [[ASIN:B006LMZBD2 Alexander The Great]] that Alexander recorded for Pacific Jazz during the After Hours Concert at the Esquire Theater in Los Angeles. This is Alexander's debut album that established him as a 'contender' in 1964.At the time of this review there is no track listing or sound samples on this page. I can provide the tracks:A1 John Brown's Body 5:30A2 Jitterbug Waltz 3:59A3 Comin' Home Baby 5:45A4 If I Were A Bell 6:18B1 The Grabber 5:30B2 Autumn Leaves 5:59B3 I've Never Been In Love Before 5:30B4 Blues For Jilly 4:40Since this was live it explains how two of Alexander's compositions (The Grabber and Blues for Jilly) made it on the album. Normally new artists stick with standards, which do comprise the remaining tracks. From the start you can hear Les McCann's influence on Alexander (indeed, McCann also performed at the concert and introduced Alexander).The rhythm section is comprised of Victor Gaskin on bass and Paul Humphrey on drums. They also backed McCann at the same concert and I am under the impression that Alexander was there as a guest. I cannot be sure, but I am surmising that given McCann's unmistakable influence on Alexander's playing that continues to this day that there were some behind the scenes machinations that got Alexander on this album. If you are inclined to track down McCann's album from the same concert it's titled Les McCann - Spanish Onions Pacific Jazz PJ-10097.Sound quality on this LP is excellent, even on my cheap USB turntable, and if you are an Alexander fan this is ground zero in his career.",J +"I've owned a copy of this album for decades and am reviewing the music and not the media nor the price.What I love is the arrangements, which are totally different from any I have heard. For example, Green Dolphin Street (as originally composed) has an ABAB format. The rendition on this album does too, but the A section almost seamlessly segues into the B section. Nearly every other arrangement I've heard (and played) has an abrupt transition between the two. The other tracks are similar in that Sample takes a lot of liberties with the melody line as well as the phrasing and arrangements.Another thing is on this album Sample is back on an acoustic piano. I loved his playing on his earlier albums like Lighthouse 62 and Freedom Sound, and to be honest had drifted away when he and the Crusaders went the fusion route in the late 1960s/early 1970s. So this album is a nice change in that respect.Of course the rhythm section makes a lot of difference and having Ray Brown on bass practically guarantees that there would be a swinging pulse. As a drummer who has long admired Shelly Manne I have a personal reason for loving this album.Sadly there are no sound samples on this page, but if you are diligent and do a web search you will be rewarded with some. From those you can get a better picture of what I mean by arrangements being unique.This album was recorded in Los Angeles on November 28, 1975 for the Japanese label East Wind. Although the recording was done at the Warner Brothers Studios the production team was Japanese. The sound quality is superb. One final note: this product page at the time of this review does not contain a track listing. Here it is:A1 Yearnin' 5:04A2 On Green Dolphin Street 5:17A3 Satin Doll 5:38B1 Manha Do Carnaval 5:59B2 Round About Midnight 4:34B3 Funky Blues 5:10Total play time is 29:53.",D +"This album is really a double album set that is [[ASIN:B003OFC2SG SINGS WITH THE PAT MORAN TRIO]] with the trio comprised of Patti Moran [McCoy] on piano, Scott LaFaro on bass and Gene Gammage on drums backing Beverly Kelly on vocals, and [[ASIN:B004FBEQN2 This is Pat Moran]] featuring just the trio. Both albums are listed in various discographies as being recorded in December 1957, although the trio-only one is claimed by Moran-McCoy to have been recorded a year later. Not disputed is both were recorded in New York.Here are some discrepancies that make these an incomplete collection of the trio recordings: Track 20 does not show up in discographies, including the one dedicated to these two albums that span pages 262 through 264 in Lafaro's biography written by his sister: [[ASIN:1574412736 Jade Visions: The Life and Music of Scott LaFaro]]. Also missing from this album are Spring is Here and This Love of Mine, both of which are from the Kelly album, and Stella by Starlight, When Your Lover Has Gone and Farewells from the Trio album.Despite the fact that this does not contain all the tracks from the trio sessions, it does contain a record of LaFaro's genius even at 21. That alone merits five stars in my opinion in spite of some missing tracks.All discographies that I have checked, including LaFaro's biography cited above confirm that LaFaro was, in fact, the bassist and Gammage the drummer on all tracks.Sadly there are no sound samples here, nor on the Kelly album page, but there are on the Trio page, which will give you a good idea of the music on this album. If you are a bassist, jazz historian or musicologist, or a LaFaro fan this album is a treasure despite the discrepancies.",f +"This album was recorded mere weeks after Chet Baker and Bird were 'home recorded' at the same venue. Like that album - [[ASIN:B005MVLS94 Bird & Chet: Live at the Trade Winds, 1952]], the sound quality on this one is not spectacular, but the music is. In fact some of the same musicians from the earlier session are on this one.Sadly there are no sound samples on this page at the time of this review. In a nutshell, this is mainly a horn-centric session with a young Chet Baker playing a lot more aggressively than he would on subsequent albums. There are two tracks where the ensemble downshifts and gives Al Haig a lot of room and places him in the front line: I'll Remember April and Sweet Georgia Brown.Here are the tracks:1. There'll Never Be Another You2. Bernie's Tune3. My Old Flame4. Avalon5. I'll Remember April6. How High The Moon7. Sweet Georgia BrownIf you love energetic music this is a great album. It was recorded live at the Trade Winds Club in Inglewood (a Los Angeles suburb) on August 4, 1952 and features the following line-up:Chet Baker on trumpet with Ted Ottison on trumpet on tracks 5 and 7, Sonny Criss on alto on tracks 1, 3 and 7, Jack Montrose on tenor sax on tracks 1, 4 and 7, Les Thompson on harmonica on track 4, and a rhythm section comprised of Al Haig on piano, Dave Bryant on bass and Larry Bunker on drums.",b +"Nearly every track in this compilation is excellent, although there is a handful that are marginal to my ears. I do not view that as a show-stopper, but if you are an audiophile you may want to spend the money and purchased each album in vinyl.Since there is no track list on this page at the time of this review I will include the tracks here. Also, I will list the original album titles, but in some cases will link to albums that have identical tracks but a lightly different title. More than a few reissues came out with slightly different titles. Also, I am not going to provide detailed information on each album because it would greatly clutter this page. Here is what is on the four discs that are in this set:==========DISC ONE==========The first 11 tracks is the album titled [[ASIN:B00D0Z7HE4 The Herbie Mann - Sam Most Quintet]] and the remaining tracks is the album titled [[ASIN:B00006J3ZW Flute Flight]]:01. Fascinating Rhythm02. Why Do I Love You?03. It's Only Sunshine04. Love Letters05. Let's Get Away from It All06. Flying Home07. I'll Remember April08. Empathy09. It Might as Well Be Spring10. Just One of Those Things11. Seven Comes Eleven12. Tutti Flutie13. Bo Do14. Flute Bass Blues15. Flute Bob16. Solacium==========DISC TWO==========The first 8 tracks is the album titled [[ASIN:B000QZRMCI Flute Fraternity with Herbie Mann and Buddy Collette]] and the remaining tracks is the album originally titled Sultry Serenade, but is now issued as [[ASIN:B0055WHK92 Let Me Tell You]].01. Herbie's Buddy02. Perdido03. Bangles and Beads Baubles04. Give a Little Whistle05. Here's Pete06. Theme from 'Theme From'07. Nancy with the Laughing Face08. Morning After09. Let Me Tell You10. When the Sun Comes Out11. Professor12. Lazy Bones13. Sultry Serenade14. Little Man You've Had a Busy Day15. One Morning In May16. Swing Til the Girls Come Home==========DISC THREE==========The first four tracks are from the album titled Flute Souffle, but are now found on the same Let Me Tell you album cited for the previous disc. The remaining tracks on this disc are the album titled [[ASIN:B002G6ESH2 Yardbird Suite]].01. Tel Aviv02. Somewhere Else03. Let's March04. Chasin' the Bird05. Yardbird Suite06. Here's That Man07. One for Tubby08. Squire's Parlor09. Who Knew10. Opicana==========DISC FOUR==========The first six tracks are the album titled [[ASIN:B000000Z2L Just Wailin]] and the remaining tracks are the album titled [[ASIN:B000W178WA Flautista! Herbie Mann Plays Afro-Cuban Jazz]]. Note, that album contains bonus tracks that are not on this one.01. Minor Groove02. Gospel Truth03. Blue Echoes04. Trinidad05. Jumpin' with Symphony Sid06. Blue Dip07. Todos Loco's08. Cuban Potato Chip09. Come On Mule10. The Amazon River11. CaravanNote that this is identical to an MP3 album that contains all 53 tracks in the same order, so if you want immediate gratification check it out: [[ASIN:B00EE7SQ7O The Jazz Collection Volume One]]. Even if you opt for physical discs visit that page and listen to the sound samples for an idea of both the sound quality and the music.Overall this is a great collection of Herbie's work at the height of his pure jazz period. He would continue to grow and experiment musically - sometimes in ways that lost him fans - but if you like the earlier period this set is golden.",L +"Note that this album is out of print, but is part of a value-priced set titled [[ASIN:B00CSFXQZ6 8 Classic Albums]] as the first ten tracks on Disc Four.This 1962 album contains tracks that clearly show Lewis' strategy to move from a jazz piano trio format to more popular music (perhaps in response to Ray Charles' success in crossing over from R&B to pop.) The first five tracks on the album have a string section backing the trio.While the first five tracks comprising Side A is a foray into pop, Side B is still heavily rooted in jazz. The album has a consistent theme - Spring - and Lewis contributed three compositions to reinforce that theme. He wrote A1, A3 and B1, and B5 was written by bassist Eldee Young. Since there is no track list at the time of this review here are the tracks on the album:SIDE AA1 Sound Of Spring 2:30A2 Spring Can Really Hang You Up The Most 4:25A3 Blue Spring 3:00A4 Spring Is Here 4:22A5 Spring Will Be A Little Late This Year 2:43SIDE BB1 Spring Fever 2:35B2 It Might As Well Be Spring 3:52B3 Soft Winds 3:10B4 There'll Be Another Spring 2:30B5 Truly, Truly Spring 3:27The trio is comprised of Ramsey Lewis on piano, Eldee Young on bass and Red Holt on drums. The album was recorded for Argo at the United Recordings Studio in Los Angeles on February 14 and 15, 1962 and released later that year. Interestingly Ruth Brown (credited as the single artist who made Atlantic Records a name) and Leonard Chess of Chess records have producer credits.",l +"First: this is identical to an MP3 album that contains all 53 tracks in the same order, so if you want immediate gratification check it out: [[ASIN:B00EE7SQ7O The Jazz Collection Volume One]]. Even if you opt for physical discs visit that page and listen to the sound samples for an idea of both the sound quality and the music.Nearly every track in this compilation is excellent, although there is a handful that are marginal to my ears. I do not view that as a show-stopper, but if you are an audiophile you may want to spend the money and purchased each album in vinyl.Since there is no track list on this page at the time of this review I will include the tracks here. Also, I will list the original album titles, but in some cases will link to albums that have identical tracks but a lightly different title. More than a few reissues came out with slightly different titles. Also, I am not going to provide detailed information on each album because it would greatly clutter this page. See my review for the MP3 version of this set for that info.Here is what is on the four discs that are in this set:==========DISC ONE==========The first 11 tracks is the album titled [[ASIN:B00D0Z7HE4 The Herbie Mann - Sam Most Quintet]] and the remaining tracks is the album titled [[ASIN:B00006J3ZW Flute Flight]]:01. Fascinating Rhythm02. Why Do I Love You?03. It's Only Sunshine04. Love Letters05. Let's Get Away from It All06. Flying Home07. I'll Remember April08. Empathy09. It Might as Well Be Spring10. Just One of Those Things11. Seven Comes Eleven12. Tutti Flutie13. Bo Do14. Flute Bass Blues15. Flute Bob16. Solacium==========DISC TWO==========The first 8 tracks is the album titled [[ASIN:B000QZRMCI Flute Fraternity with Herbie Mann and Buddy Collette]] and the remaining tracks is the album originally titled Sultry Serenade, but is now issued as [[ASIN:B0055WHK92 Let Me Tell You]].01. Herbie's Buddy02. Perdido03. Bangles and Beads Baubles04. Give a Little Whistle05. Here's Pete06. Theme from 'Theme From'07. Nancy with the Laughing Face08. Morning After09. Let Me Tell You10. When the Sun Comes Out11. Professor12. Lazy Bones13. Sultry Serenade14. Little Man You've Had a Busy Day15. One Morning In May16. Swing Til the Girls Come Home==========DISC THREE==========The first four tracks are from the album titled Flute Souffle, but are now found on the same Let Me Tell you album cited for the previous disc. The remaining tracks on this disc are the album titled [[ASIN:B002G6ESH2 Yardbird Suite]].01. Tel Aviv02. Somewhere Else03. Let's March04. Chasin' the Bird05. Yardbird Suite06. Here's That Man07. One for Tubby08. Squire's Parlor09. Who Knew10. Opicana==========DISC FOUR==========The first six tracks are the album titled [[ASIN:B000000Z2L Just Wailin]] and the remaining tracks are the album titled [[ASIN:B000W178WA Flautista! Herbie Mann Plays Afro-Cuban Jazz]]. Note, that album contains bonus tracks that are not on this one.01. Minor Groove02. Gospel Truth03. Blue Echoes04. Trinidad05. Jumpin' with Symphony Sid06. Blue Dip07. Todos Loco's08. Cuban Potato Chip09. Come On Mule10. The Amazon River11. CaravanOverall this is a great collection of Herbie's work at the height of his pure jazz period. He would continue to grow and experiment musically - sometimes in ways that lost him fans - but if you like the earlier period this set is golden.",l +"This is one of two collaborations between one of that era's most promising bassists whose life was tragically cut short at 25 and Pat Moran who now goes by Patti Moran McCoy.The 12 tracks comprising this album were recorded in New York in late 1957 with the following personnel: Pat Moran on piano, Scott LaFaro on bass and Gene Gammage on drums backing vocalist Beverly Kelly. The bassist and drummer are sometimes incorrectly credited as bassist John Doling and drummer Johnny Whited, but according to Scott LaFaro's sister in her biography of him ([[ASIN:1574412736 Jade Visions: The Life and Music of Scott LaFaro]] it was indeed LaFaro and Gammage.Since there is no track list at the time of this review here is what is on the album:01. Lover Come Back To Me02. The Man I Love03. I Get A Kick Out Of You04. I Wish I Knew05. You Don't Know What Love Is06. I'm Glad There Is You07. Sometimes I'm Happy08. You And The Night And The Music09. But Not For Me10. This Love Of Mine11. Embraceable You12. Spring Is HereTen of these tracks are included in the double album titled [[ASIN:B000N2H8I2 Complete Trio Sessions]], which while missing some original tracks from this and the other album which comprises it may be a better value if you are not a completest.Sadly there are no sound samples on this page, but LaFaro's playing is clear and unmistakable.",F +"This 3-disc set is out-of-print, but still offered used. As I write this review the other formats offered - MP3 Music 39 Songs 2010 and Audio CD 2011 - are for different products that are not the same as this album.What makes this album great is it's a sampler that contains six tracks each for six truly great ladies in jazz. I'll list the contents of each disc below.Of course the choices are not going to satisfy everyone. For example, it would have been great had one of my favorites been included - Anita O'Day or Maxine Sullivan - and Dinah Washington is better known as an R&B artist. On the other hand, one cannot argue with either the choices made since each vocalist is one of the greats, or the tracks which do showcase their talents. And as for Ms. Washington: although she was often billed as the Queen of the Blues, she was also one great jazz singer. Her tracks on this album prove that.Here is what the set contains:DISC ONE:=============Sarah Vaughan=============01. Lullaby of Birdland02. All of Me03. Lover Man04. Over the Rainbow05. Autumn in New York06. My Heart Stood Still=============Dinah Washington=============07. Caravan08. Ain't Misbehavin'09. Is You or Is You Ain't My Baby10. Smoke Gets in Your Eyes11. I Get a Kick Out of You12. Stormy WeatherDISC TWO:=============Billie Holiday=============01. A Foggy Day02. I've Got Love to Keep Me Warm03. God Bless the Child04. Too Marvelous for Words05. I Thought About You06. Lady Sings the Blues=============Ella Fitzgerald=============07. It Don't Mean a Thing (if it ain't got that swing)08. The Lady is a Tramp09. Cheek to Cheek10. Oh, Lady Be Good11. Satin Doll12. Let's Call the Whole Thing OffDISC THREE:=============Nina Simone=============01. Mood Indigo02. Black is the Color of My True Love's Hair03. I Loves You Porgy04. Don't Let Me be Misunderstood05. Wild is the Wind06. I Put a Spell on You=============Pearl Bailey=============07. Easy Street08. Here You Come With Love09. I've Taken a Fancy to You10. My Man11. You Waited Too Long12. Sweet Georgia BrownAs you can see, the tracks are familiar. But that is what makes this a great sampler: familiar songs performed by some of the best ever jazz vocalists. No surprises or weirdness, and the tracks are organized by vocalist instead of scattered across the three discs with no coherency or reason.",g +"This album contains some rare tracks spanning the period between 1946 and 1950. As I write this review it's out-of-print, but fortunately still available as Disc Six in a 10-disc set titled [[ASIN:B005D4XYUQ Kind of Clarke]].Sadly there are no sound samples on this page. It would be presumptuous of me to attempt to describe the songs comprising these tracks because such descriptions would only reflect my personal tastes and biases. I can, however, provide detailed information regarding when, where and my whom each track was recorded. That information will give you some idea as to the music itself based on the signature sounds of the musicians in the sessions.The first four tracks were recorded in NY on September 5, 1946 and feature Kenny Dorham and Fats Navarro on trumpet, Sonny Stitt on alto sax, Ray Abrams on tenor sax, Eddie DeVerteuil on baritone sax, Bud Powell on piano, John Collins on guitar, Al Hall on bass and Kenny Clarke on drums.Tracks 5-7 are Kenny Clarke Sextet, recorded in Paris on March 2, 1948 and feature Benny Bailey on trumpet, John Brown on alto sax, Cecil Payne on baritone sax, Ralph Schecroun on piano, Al McKibbon on bass and Kenny Clarke on drums.Tracks 8-11 were recorded in Paris on May 4, 1948 as the the Kenny Clarke Orchestra. Personnel are: Dick Collins on trumpet, Hubert Fol on alto sax, Jean-Claude Fohrenbach on tenor sax, Andre Hodeir under the pseudonym of ""Claude Laurence"" on violin, Jacques Denjean on piano, Charles Montaggioni on guitar, Alf Masselier on bass and Kenny Clarke on drums.Tracks 12 through 15 were recorded in Paris on May 5, 1948 as the Michel De Villers et Son Orchestre comprised of Claude Dunson on trumpet, Michel De Villers on alto sax, Jean-Claude Fohrenbach on tenor sax, Jacques Denjean on piano, Charles Montaggioni on guitar, Alf Masselier on bass and Kenny Clarke on drums.Tracks 16-19 were recorded in Paris on October 29, 1949 as the Hubert Fol et ses Be-Bop Minstrels. This group was comprised of Nat Peck on trombone, Hubert Fol on alto sax, Bernard Peiffer on piano, Jean Bouchety on bass and Kenny Clarke on drums.Final two tracks were recorded in Paris on June 7, 1950 as Kenny Clarke and his Orchestra. Personnel on this session were: Nat Peck on trombone, Hubert Fol on alto sax, James Moody on tenor sax, Gerald Wiggins on piano, Pierre Michelot on bass and Kenny Clarke on drums.If you are a Klook fan this album documents an important period of his playing approximately a decade after he went professional. I do recommend that you consider the 10-disc set mentioned above if you want this album plus a wider set of examples of Klook's work.",^ +"It's hard to dislike this album, although there are better deals that document Anita's Verve Years. For one, I would go with [[ASIN:B005I4UMCO 8 Classic Albums]], which is slightly more expensive with a lot more tracks. However, this album contains some true gems.Since as of the date of this review there is no track listing for this album I am providing one (and their source albums).The first seven tracks, plus track 23 are from her 1958 album titled [[ASIN:B000WLVBBE Anita O'Day Sings The Winners]]. They are: Take the A Train, Early Autumn, Four Brothers, My Funny Valentine, Sing, Sing, Sing, What's Your Story Morning Glory, Peanut Vendor and Star Eyes.Track 8, Whisper Not, is from one of my favorite albums titled [[ASIN:B000V697T8 Anita O'Day And The Three Sounds]]. Sadly this is the only track that is from that amazing album.Stomping at the Savoy and Don't Be That Way (tracks 9 & 11) are from [[ASIN:B000V6728Q Pick Yourself Up]], while Hershey Bar (track 10) is from [[ASIN:B001NZJ3HW Cool Heat]] and Peel Me a Grape (track 12) is from [[ASIN:B000VRH2IU Time For Two]] (another album I absolutely love and wish was better represented in this collection.)Tracks 13-21 are from [[ASIN:B000WLQKHO Swings Cole Porter]], consisting of Just One of Those Things, Love for Sale, Easy to Love, I Get a Kick Out of You, What Is This Thing Called Love, Night and Day, Get Out of Town, I've Got You Under My Skin, and De-Lovely. Somehow the producer managed to include every one of my favorites!Interestingly, the next-to-last track - Honeysuckle Rose - is from her first Verve album, [[ASIN:B00000470I This Is Anita]].Overall, this is a solid collection of Anita's Verve years, and are drawn from albumsnext-to-last that are more centered on orchestral arrangements and backing (two exceptions are tracks 8 and 12.) If you are new to Anita's music this is a good starting place for her post-Swing era work, although I recommend that you check [[ASIN:B005I4UMCO 8 Classic Albums]] before making a final purchase decision.",S +"Not quite up to previous efforts, but maybe I just need to listen to it a few more times.",m +"I'm old and decrepit, so I no longer know anyone who speaks Spanish. I bought the CD to hear the old language again and to hear ""Soledad"" in Spanish. I've only heard the song in French, which is nice, but I don't understand French. Emilio Jose recorded all the songs in Spanish, which is nice. I detect a Castilian pronunciation, but Spanish is Spanish, despite the diverse accents. I like this CD, and as an added bonus, I finally understand ""Soledad.""",H +"I bought this as a gift, and the person I gave it to LOVED it.",W +Love it,c +I purchased this CD thinking it was a certain album that I used to own (on cassette).. but it turns out that it was basically not even the particular recordings that I was searching for.There were no samples from the CD that I could listen to.. so I had no idea that this was not what I was after. (All the tracks listed were the those found on my old cassette). But the tracks on the CD were NOT even the songs they were supposed to be (apart from one of them.. and even that one was listed under the wrong name)I did end up liking the tracks on this CD (even though they were not the ones listed at all) so I gave it a couple of stars -- because I liked it and it was very inexpensive..EDIT: At least one of the songs that was found on it was the track that I was looking for.,t +Bootleg item.....so cheap....CD is a single...only one song.,M +"i like this mixtape,its got different beats and a new vibe...its nice too here sumthin different.clams casino makes different beats.peso is my favorite joint",` +"This is a phenomenal debut album from Daniel Powter. The stubbly-jawed, green-eyed, beanie-wearing 34 year old Canadian singer/songwriter gives a very impressive account of himself, and has already sold more than 100,000 albums, due mainly to a killer first single ""Bad Day"", something with which we can all identify.""Where is the moment when we need it the mostYou kick up the leaves and the magic is lostThey tell me your blue sky's faded to greyThey tell me your passion's gone awayAnd I don't need no carrying onStand in the line just ahead of the lawYou're faking a smile with the coffee you goYou tell me your life's been way off lineYou're falling to pieces every timeAnd I don't need no carrying onCause you had a bad dayYou're taking one downYou sing a sad song just to turn it aroundYou say you don't knowYou tell me don't lieYou work at a smile and you go for a rideYou had a bad dayThe camera don't lieYou're coming back down and you really don't mindYou had a bad dayYou had a bad day""Powter wrote the songs on this album over a one year period, his musical taste inspired by childhood violin sessions, accompanying his mother on piano. The second single will be ""Free Loop"", but the excellent ""Song 6"" would also have been a great pick.This album is definitely a keeper, other great tracks being ""Jimmy Gets High"" and ""Lie to Me"" (falsetto bits and all), ""Styrofoam"" and ""Hollywood"", but then every track is a potential single. The only problem is that there are only ten songs, and that's a crying shame.Amanda Richards, October 6, 2005",e +Great,B +Buy this CD!,M +"In the opinion of most concert pianists Josef Hofmann was the greatest pianist of the recording era. Of course there's some dispute on such topics. He was certainly the greatest interpreter of Chopin. Being born and raised in Poland didn't hurt that one little bit. This completes my collection of Hofmann and it's a great conclusion to the sequence of the sequence of ""The Complete Josef Hofmann"".",b +"Great music - great orchestra, great conductor. I'm very pleased.",I +"Good, but not Toscanini great!",O +One of her beat CD's if not the best,r +"Arrived on time with all the other ""release date"" shipments! Packaging was excellent - I've had so many bad experiences with packaging and damaged merch lately but there was no such problem with this item! There were even a couple freebies! Thank you!",f +"Awe, great mix of tunes of well known artists. I haven't heard of all these songs before, so it has been a treat to listen to something ""new"". If you dig jazz and related sounds, then I see no reason not to give this a try.",y +"Listening to this CD is a relaxing, enjoyable learning opportunity. So many of us struggle on a daily basis with food choices. What should we eat? What will happen if we eat the wrong thing? This CD helps the listener relax and find inner peace with eating, food and the entire process of nourishing your body, mind and spirit.The voice of the narrator is calming and peaceful. He walks you through some easy meditation practices and breathing exercises. By the time I was finished listening to this, I felt calmer and was visibly more relaxed. If you are struggling with a weight issue or some type of eating disorder, this may give you a chance to stop and evaluate the choices you are making. I found that after listening to this CD I was able to clearly identify the problems with my own eating habits and look at them from a new perspective.The one thing I would have changed about this, was the jarring chime. I didn't feel that it fit in with the relaxing environment the rest of the CD produces for the listener. I almost felt that while it was attention grabbing, I was ringside at a boxing match. The rest of the CD is excellent though, so it is hard to find too much fault.I would recommend this CD to anyone who is looking for a relaxing experience that will help you understand yourself, your relationship with food and is looking to learn basic practices to calm your mind. A great choice.",z +This CD absolutely blew me away. Who needs tranquilizers with this around?? A must buy!,N +"I bought this CD to give as a gift, I love this CD. jb",D +I have been buying this fragrance since the 70's,T +this is a replacement copy for me. The playing is absolutely first class.,K +The performances of Ferdinand Klinda are first class.He plays these works with authority! jb,h +this is an historical recording that has been re-issued. i was very happy to buy it! jb,` +"legendary organist piet kee plays the works of cesar franck with authority. piet kee was the municipal organist at st. bavos in harlem, the netherlands for many years.",` +"Great CD of songs from the 60's, my time. Audio is clear, crisp, and broad in range. Great.",P +Every song on this wonderful album is a joy to listen to - this couple is so talented and so full of energy.I could play it all day.,e +One of the greatest soundtracks ever.,C +I like the CDs. The music is very enjoyable. I prefer buying CDs as I buy the product one time. I don’t continuously rebuy it as you do with streaming services. My car is ancient with a CD player and a 3.5 mm plug for my old iPod.,h +My husband and I were in Bayside Miami a few weeks back. We were joined by my Cousin who I had not seen in over 30 years. As we walked in the Bayside area he mentioned of the music that usually plays there and how people enjoy dancing. So we went there and were able to appreciate the music of Erica Denz and her band. It was great! The dancer and back up singer of the group started to go through the group of expectators selling what was supposed to be this CD posted on this Amazon website. To our surprise the CD had some rappers music with questionable lyrics even though the label shows Ms. Denz's songs. We've attempted to call and send messages to Ms. Denz multiple time to let her know of this matter but have been unsuccessful. She does not return calls nor replies to her Facebook messages. This is very disappointing and disconcerting to us. We paid $20 for two CDs. Folks I would think it twice before buying this CD. We were ripped off!!! We reported this incident to the Miami Attorney General.No. I would not buy her CDs because she has evaded/ignored our telephone messages and facebook emails...So sad to take advantage of tourists like that because they know they are leaving and most likely won't listen to the CD's until after they are gone from Miami.,X +"I'm a serious side sleeper with neck issues and have tried any number of pillows over the years. This actually works! I wasn't sure going in that it would be any better or worse than anything else but I'm very happy I took the chance. It's soft and the shaping allows it to support my head without canting my neck either up or down. I did cover it with one of my king size pillowcases (I'm not sure it would fit a regular size) but have been using it for several days now. I'm sleeping better and waking up less stiff. If you are a side sleeper, I'd say this is definitely worth a try.",M +"Throat Culture were I believe an Ohio-based A-Capella Quintet whose career blew thru' in early to mid nineties.In short, they were brilliant!This is their first album, and unlike the latter ""Up with Angst"" it's primarily composed of cover material. but oh wow, what covers!It kicks off with a brilliant and lively rendition of Desmond Deckers' ""Israelites"" that morphs into Sam Cooke's 'Working in Coal Mine' and back out again, followed by a guitar-free version of Hendrix' 'Crosstown Traffic' that somehow sounds better than the original. Later in the album are such classics as 'Fly Like an Eagle', '50 ways to leave your lover' and 'St James Infirmary' - all delivered note perfect, and using only sounds made by voice and percussing the human frame.Outstanding in this CD are originals 'Easter Island Head' - such a catchy and funny song I still attempt my own version in the shower, and the tongue in cheek 'Sitting On the Groom's Side' - a humorous take on weddings.As i recall, this was pretty much their live set....although notably missing is the showstopping 'Sweet Dreams' which somehow didn't make this recording and was saved for the ""Angst"" album, oh well, nothing can be perfect.If you enjoy acapella OR if you remember this talented band from wayback - time to put this CD on the deck and listen.PS - somehow there was a connection to the band 'Oswald & the Herringbones' - someone somewhere MUST have a bunch of copies of this album on CD or tape.TIME for a re-issue!!!!1",Z +"Brilliance!Clash at the peak of their game with a heartfelt 'I fought the law' - reportedly recorded just after the 'guns on the roof' arrest, ""Capital radio"" (not 'capitol radio' which was a re-recording.....and done for (I think Kenny Everret Show?), and the two songs hanging over from 101'ers and London SS....'Gates of the West' and ""Groovy Times"" - which don't show up anyplace else very often.ANYWAY - if you're a Clash aficionado, i don't need to rabbit on about the wonder of this EP, if you're not a Clash fan.....you're unlikely to care anyway.But for those of us who were there at the Electric Circus in Manchester, or wherever you caught Clash gigging.....THIS raw sounding (but not live) set is a wonderful trip down punk memory lane.Watchout for all the gobbin' down in the front row...........:-)",r +Bought for my Dad who loves it. We did not even realize it has 3 CDs and brings back memories of the Welk years for my Dad.,U +"New condition in shrink wrap. Packaged very well in bubble wrap.Tracking number didn't work and was delayed, but I still got the package.",P +I was very disappointed with this CD as there are No Vocals.These are just the studio backing tracks.Although they are interesting it’s not what I thought I was getting for my wife fo Mother’s Day.,z +The album is fantastic. Hight quality. Sechskies is awesome and never fails.,r +"the balance of the music blended very well.....there are no great songs and a throw one here and there ...but each selection was a hit, this is a great recommendation for a first time Motown fan or a veteran who only wants a blending...",C +This album is on repeat.,l +"Some of the recordings are poor recordings, but I wanted it for the songs more than the quality of the recordings.",f +Not as expected. This is NOT the Dada who did Puzzle.,b +This CD was a bit of a disappointment. Some tracks are muddy but that may be the fault of analogue equipment in less than ideal live conditions. Otherwise OK.,F +"Shipped securely, delivered on time, Great CD, wonderful music. I have enjoyed listening to this cd daily",c +This a beautiful soundtrack even if you have never seen the movie,k +Was a gift for a fan of his:well pleased.,I +I really in joy Aretha Franklin CD great song.,e +Great.,H +Can't go wrong with Elvis gospel. Love it.,E +I love this 3 cd collection of smooth jazz. Great to listen to while cooking.,d +Beautiful quality leggings my favorite pair I ordered yet,T +"This album is a must. It has rousing Brothers Grimm, Hans Christian Anderson, and other tales told in the classic ballad-story tradition. Disc includes: Sleeping Beauty, Jack & the Beanstalk, Little Red Riding Hood, Cinderalla, The Pied Piper, Puss in Boots, Rumplestilksin, Goldilocks, Gulliver, The Nightengale, Ugly Ducking, and the Fox and the Grapes.",Q +"Hypnosis is so relaxing that I try to listen to one CD a day. In colder weather I feel it works best to put on a sweater and socks and to lay in a comfortable bed with a heated mattress pad and blanket or duvet. I say this because it can get cold laying still for a while.This CD has two hypnosis sessions. One is a waking hypnosis where at the end you are back to reality and alert afterwards. The second is a hypnosis that leads you into a sleepy state so you can drift off to sleep with music and ocean waves.This CD helps you feel positive about leaving the house and promotes a more confident and calm attitude so you don't fear new experiences and unfamiliar places.Rachael Eccles has a real talent for visualization and takes you to a peaceful beach. Her soothing voice will help you relax and solve your issues too! Give this CD some time to work. Being afraid of going outside is a real problem and takes some psychological work to undo. I am happy going to familiar places but still struggle with going to new stores, parks etc. This summer I'm going to start to be more adventurous so planning ahead!~The Rebecca Review",b +"I'm fairly new to hypnosis CDs. Mostly I've used them in the past to get to sleep faster and found them effective. I prayed and prayed that I would lose some weight and the answer seems to be to get some hypnosis CDs and retrain my thinking about eating and exercising.Since I've been using weight loss hypnosis CDs I've exercised nearly every day and changed some of my eating habits. It therefore seems worth it to devote time to hypnosis instead of reading more books about weight loss because really what you are dealing with is the subconscious mind and how it has been programed. We all know what is good and bad in the food world, but often we just don't have the self control to make the right choices.This CD is excellent for its deep relaxation visualizations. If you can darken your room, turn off lights, get into bed with a warm comforter or heating blanket you will find it much easier to fall deeply into hypnosis. Make sure that no one disturbs you. Tell people in your home that you are going to be unavailable for as long as it takes to do one, two or three of these hypnosis sessions. I like doing all three because I find that the most effective way to relax.The first hypnosis session on this CD is a waking hypnosis which means you will go deep into relaxation, then have helpful thoughts put into your subconscious and will be told to awaken at the end. The second session is just for 5 minutes and can be used when you are short on time but need some positive reinforcement. The last session is a hypnosis which guides you through similar images to the first one, with a little change here or there. You are left to fall asleep to calming music at the end. So perfect to listen to in the evening when you are going to bed.Rachael Eccles really has a gift for helping people solve all their problems. I've been trusting her to solve a lot of mine lately! She truly wants to guide you to a place where you are more in control of your life and are not leaving things just up to chance. I think a lot of overeating is stress related and this helps you feel a sense of calm and serenity so you can be more aware of your choices each day.I found this CD has some really beautiful visualizations especially the garden and hammock that take you deep into a world of relaxing hypnosis. I've tried another weight loss hypnosis CD and they all help, but this one is my current favorite because it makes me also feel way less stress. After listening I feel centered and motivated to lose weight.~The Rebecca Review",[ +"A friend of mine in Australia told me he was listening to exotica from the 60s and I found this as a result. It is very relaxing and a total travel trip back in time. There are vocals but no lyrics and it is more classical. I do think the album cover is pretty neat too. So I'm listening to this while baking banana cake with cream cheese icing. I hear mostly piano, drums, flute, strings and other orchestral sounds. So if you want to listen to something from the 60s that is classical this is a good choice.~The Rebecca Review",a +It is difficult to imagine how two people who could create such ecstasy with their voices would no longer be talking. Fortunately this CD captures a time of spontaneity and playful chemistry. This is worth owning if you are a real fan. There are some original songs and three covers which are very well done.~The Rebecca Review,p +"A treasured collection for the New Age and World Music fan. This music captures your mind and spirit and is an evocative and exciting collection which will open your mind and take you above the plain of ordinary consciousness.""Yeha Noha"" takes you into the mind of an eagle as it soars high above the earth swirling to the sky. You can feel the energy in the sky and see the sun rising. They say if you see an eagle soaring up to the sky, it means good things will happen to you. This title of this song means: ""Wishes of Happiness and Prosperity.""Sweet Lullaby makes you think of the scent of rain and fresh earth, perhaps a walk through an African veld in the afternoon while you listen to the beat of drums somewhere in the distance.A good set of speakers or headphones is a must so you can enjoy all the nuances of this collection. This music will engulf a room and promotes peace, joy and a feeling of harmony with life. You can literally drift away to another world by listening to this music. It is very uplifting and evokes images of peaceful places and blissful journeys.""Chi Mai"" is an incredibly sad piece of music so I guess this is not all joy and happiness but there is a certain poignancy in the music which speaks to future happiness and an feeling of being engulfed in love no matter what happens in life.""Prelude"" will take you to India. Yanni's music was my first introduction to mood music and it is fitting that his music is on this CD.""Cavatina"" is a beautiful piece which is extremely evocative of romance and country picnics. Happiness and Hope are the feelings this music will evoke in you. Perhaps the start of a new day.On the second CD, ""Children"" is a deep water piece which reminds me of whales playing in the ocean and of ships sailing in the wind. You can almost hear the cries of whales. The beat is very up tempo and is in contrast with the first CD which seems to be a slower pace. You never want the song to end and it does seem to have the feeling of eternity.""Chacharpaya"" has South American/Mexican flair and is filled with flutes and drums. ""Protection"" was more light smooth Jazz. ""The Sun Rising"" is perhaps the best song on this CD as far as mood music goes.""Sadness Part 1"" will be one of the most recognizable songs. ""Brideshead"" is classic music with violin and harp and is followed by ""Pie Jesu"" which is ethereal. ""Schindler's List"" is a haunting piece with a sad violin and a sense of unfulfilled dreams.There were a few pieces I fast forwarded over, but overall, I think you will enjoy this set. I would choose this imported selection over the other CD's because you just get so many more selections. I think you will play the first CD more than the second, but it depends on your mood. Return to your senses with this beautiful compilation.~The Rebecca Review",d +Great gospel big band recording. Great charts that are well-played. Brentwood-Benson sells the actual arrangements that you can play.,D +"I discovered Say Hello To Sunshine through one of my favorite American rock bands awhile ago (Finch) and instantly fell in love with their sound. Listening to this EP gave me flashbacks of my late teen years, just after high school filling basements with music, jamming with my friends, and trying to form a screamo band. These guys have really done it, you can hear a range of influences from Finch, to Deaftones, and even some Juliana Theory influence in this EP. I love this band, and I sincerely hope they come on tour in the U.S. one day, but until then... Buy this album, you will thank me later.",C +"very nice . easy to listen to and very restful, thank you",` +"great music, one of my favorite thanks",S +Beautifully done- both uplifting and relaxing. Kino has a lovely voice - clear and sweet. Thank you Kino!,o +I have no idea how this could be called 'essentials'. It's just JF's idea of what an orchestrated practice should be. Not my favorite. There's no brochure that describes the postures or the procedures.,I +"Belleruth is a pioneer, and her audios are all first-rate. I have not only used all her titles with my therapy clients and my friends - everything from stress to cancer therapy - but they have also been immensely helpful to me personally. Everyone has experienced heartbreak, abandonment or betrayal, so this title speaks to us all. With deep power as you sink into unconscious mind trance, you gain resources to counteract what circumstances or other people have yanked out of you. Highly recommended.",D +Ready listening,[ +"Great CD, nice collection of their best",^ +"I bought this CD when I was 10, I particular enjoy Music Box Dancer, Chariots of Fire and Last Date. I never would have gotten into these classics if it wasn't for this collection that I came across from an infomercial.I wish my friends could be interested in this kind of music!",j +SO wonderfully calming and soothing at night. One of my favorites. Perfect for background dinner music.,B +Buying 2 more just to have !!,f +I got this as a Christmas gift for my husband and he loves it.,\ +"I finally crossed Hedwig off of my "To Watch" list this week, and I instantly became obsessed with it. I keep thinking about how much I enjoyed it, which means that, of course, I now have to have copies of the soundtrack and DVD to add to my collection. JCM's costumes and make-up are positively fantastic, and I love how Hedwig can convey emotion without having to say anything at all. These songs have rightfully earned Hedwig a spot as one of the top rock musicals of all time. I just wish I had gotten into this earlier so I could have seen NPH or Michael C. Hall perform as Hedwig.",g +Great anthem for strong women,Q +Special stash level beats,l +This is another live release but I bought it becasue I need to have everything that Hal;ford releases. Judas Priest rocked but Halford rocks harder.,Z +I ordered 4 CD. 3 had cracked cases. This is the replacement cd. It too is cracked. Disappointed.,M +I was glad to find this. I lost my original copy several years ago and this cd is a connection to my dad since we used to listen to it together,` +Being a fan(atic) of Mr. Mathis I collect many of his beautiful voice albums. The sound on these are pristine. All of the albums are available individually but this is an excellent price especially for the quality.,J +Great upbeat tempo of some classic songs. He can sing any style of song and own it.,E +This proves that JM can sing anything in any style. I have "I Love My Lady" on one of his CD's and another song by him on a Chic CD. This is so joyous; full of rhythm and spunk. Creative and artistic vinyl.Would like a CD version for the car.,S +"Have a copy of the 1957 vinyl. Bought the CD for the remastering and convenience. So love the first song, Will I Find My Love Today. Able to play it repeatedly in the car. His young and entrancing voice!",M +A wonderfully clean sound.,k +"It was exactly was I was looking for, had the right songs on it and sounds just like the movie.",] +To bad these are discontinued now and verrryyyyy hard to find. After I bought one of these and listened to it I ordered as many more as I could get. I gave most of them to my Pastor to give to folks going through serious illness or diagnosis. So comforting and it is 2 disc!!,^ +Great!,^ +"Positive presentation, shipping, availability - hits high notes across keyboard.",h +"Now I know why I didn't relate to my daughter's (& her friends') admiration of this group. I'm not a good observer, for this isn't my genre. I bought this song as it was a favorite of my recent 43 yo daughter's sudden death...a star who's gone to fly with angels...true star power (we are the stuff of stars).",l +"Much better than the depressing Hip Today outing of the grunge era. God I hated Grunge !This is a mature, FuNkY Extreme which takes up where Extreme III leaves off. This is my Favorite Extreme Album. The band doing what they do best. If you don't like what you see here.....get the Funk out.",e +Great quality sound for a used CD. Roy Orbison never sounded so great as he did on this CD.,n +Came a bit damaged but nothing to write home about.,N +Well worth the wait! Excellent collaboration! Thank you!,U +GAVE AS A GIFT,A +"Ah Elton! I had wanted the CDs with double the songs, but it was outrageously priced. I'm super happy with this one... my favorites are on it and the price was very nice.",D +No exaggeration. Incredible stuff. Too bad this CD is so hard to find now.,D +"I love Prince and couldn't wait to get this set ,I am Very happy with it. The unheard songs are greatly done the Prince way and just great to listen to. Most of the songs are 5 to 9 minutes long that's awesome",X +Only had one song I was looking for.,B +This is just an amazing CD! Love the music,a +"This movie tells it like it is absolutely no hiccups and no lies about racial bigotry and hatred in the worse way imagine in the South! I first saw the movie as a child of 7 and right smack in the middle of segregation, Jim crow and what have you! The director happened to be ahead of the times in so many ways! The truth and some people can't or don't want to accept the truth, if you don't deal with reality, reality will deal with you!",[ +"Absolutely my favorite of all of Wendi Friesen's CD programs! Worked immediately! I use her CDs when lying down and relaxing, cleaning the house, getting ready for work, or just as background noise. This really worked for me right away!",d +only problem is it could have been longer,c +It's a great read. Quick and insightful. I like Bruce Liptons take on things. He's excited about sharing his knowledge and it comes thru in this book. It has become the book I gift to friends.,M +"I thought this was sung my Dottie Rambo, Didn't like it since it wasn't. I was after hers not someone elses.",u +Amazing music. Great mix of classics and Kary's own elevation jazz. Cannot say enough great things about this CD. Highly recommend.,Z +"I've heard this band live now and this music is great. As a live band they really rate as good as I have heard. Vocals are strong and well done, by pretty much every one in the band. The Horns are awesome, right there with any of the best out there and end to end, these guys do rock. When I saw them July 5th, 2014 they had the crowd following them around the arena. Danced and sang along with them all night, it was just as good as seeing the high dollar bands I've gone to see over the years, loud and proud, just not too loud. They come out of New Orleans, cover all the genres of music, from Armstrong to ZZ and Beatles to about whoever you might want to hear. If you get a chance to see them, take it and enjoy. Great sounds!",I +"Jung Ho Seok aka J-Hope, Hobi, *literal* Sunshine has been holding out on us.I mean, the signs were there, if we cared to look: BTS is perhaps the most extensively documented band in modern history, and there are plenty of glimpses of Hobi irked and maybe a little sad, but only when the cameras are focused on someone else. But he, and probably Jin, carry an outsized share of the positivity weight.But this is what happens when you keep turning that little crank on the side of the box. Keep winding. Go ahead.It all comes down. All of it. And it's not all pretty. Although, let's face it: Ho Seok is one of the prettiest humans ever to walk the planet. I digress. Sorry. What I mean is - Jack In The Box has moments of delicate beauty. Equal Sign and Future are lovely, the first for humanity in general, the second for one special love. But there's a conflagration going on. MORE and Arson in particular hit you hard with sides of Jung Ho Seok that J-Hope keeps mostly hidden.It's all out now, though, and it is freaking glorious.MORE is a straightforward headbanger. Arson stays closer to hip hop. But they both FEEL great. Ho Seok's got a gift for rhythm and sound - his lyrics are percussive and convert information more by how they feel than by what they are overtly saying. The beats and embellishments are divine booty shakers. You get the message because you listen to the songs over and over - they feel that good even though the tone is dark.Well, in fairness, Arson is both dark and blindingly bright. The album's closer ponders whether, now that everything is on fire, he should put it out or let it burn. You can decide for yourself, but it seemed to me like he decided to pour gas on it.",x +"Woo sung has made an ep that encapsulates this particular earthbound moment with humor, irony, and a little regret - all wrapped up in a 12 minute nonstop sing and dance-along. !!! this modern life, indeed.Seriously. In Phase Me, he taunts an ex-lover or someone he wishes were an ex. The melody and lyrics are less bombastic/more seductive than I Will Survive, but it’s a similar emotional vein. The narrator is at maximum incandescence and In the video is getting a LOT of attention from uh people. Many people. And you kind of think, good for you- until you notice that the title is spelled PHASE. Which, if you like Woosung, takes you straight to Moon. And phases. And the message immediately changes. He’s great now, but it’s inevitable that the next phases will tend towards darkness before the light comes back. Which pulls you into the shadows despite how upbeat and catchy the song is on its face.Lightness and dark weave all through Moth; every song touches on both and on the need to protect one’s personal light from people and circumstances that would diminish or demolish it.And all things taken together, the dominant message seems to to one of defiance. In spite of past sadness. In spite of what’s after you. In spite of what is on the way.Highly recommend.",g +bootleg and sounds like it; collectors only,N +bootleg and sounds like it; collectors only,K +The greatest. Buy it immediately. "YouSet The Scene" is the greatest rock song ever. (No exaggeration.) This and "Pet Sounds" along with"Bo Diddley's Beach Party" are my three desert island records.,Z +"""Jazz Café CD: 1930s, 40s Vintage Blends. Cool, Smooth, Swing Jazz, Bebop, Ballads"" is a nice collection with good variety. The sound quality is very good. It makes for soothing music for the work day.",W +""Speedster" is Generations from Exile Tribe's 3rd release and that I wanted first and foremost because there are so many songs on it that I love, but "Generation Ex" which is their 2nd release and I believe came out in 2015(?) is still in my 5-star love range. There are a few songs that I skip but there is a chance that those few may end up becoming favorites later."Sing it Loud" is the opener and it is a solid get-the-hype going song. It dips a little with "Everlasting" but then goes into a great trio of songs with "Higher," "Always with You" and "My Only Love" which all are very listenable with great pop sensibilities. I really like "Revolver" although I'm not sure why. The last 5 songs though are really, really good. In fact, my two favorite Gene songs are the last two on this disc (STORY and Hana). Those two songs are just perfect as far as I'm concerned and definitely worth the money I spent for this CD.",R +"I don't know how I came across Generations from Exile Tribe, but once I heard one song from them ("Ageha"), I wanted to hear more. I actually opened an account with iTunes and Spotify so I could hear more of their music and so far, I haven't been disappointed. In fact, I'm rather addicted to the music.I don't really know much about them except that they are made up of 2 vocalists and 5 dancers which is an interesting set up to me, but apparently is something of a standard with the company they are affiliated with. Outside of that, I don't know much more about them except that their songs are top grade pop-dance-r&b music. The really good stuff that makes you want to smile and move. "Evergreen", which is on this CD, is about as good a pop song as I've ever heard. I can imagine that 10 years from now, it'll sound as good then as it did on the first listen. Other songs on this CD are solid. I really like "Loadstar" and "Gimme!" but I have the softest of soft spots for "I Believe In Miracles" which sounds straight out of 1977 and I mean that in the best way possible.It doesn't look like their CDs and/or DVDs are available from Amazon US except via third party sellers and thank goodness, the third party seller that I buy from is super reliable and I'm quite happy with them. Generations from Exile are available on Spotify and iTunes so if anyone wants to check them out, that's a good place to start. While I like listening via iTunes and Spotify, I still like having the hard copy CD in my hands and this is my first CD that I've bought for this group. It was actually released a few years ago so I've got some catching up to do and I look forward to collecting them all.",k +"I've been wanting this song in 'hard copy' for awhile so when I saw this CD/DVD combo (and it was my birthday), I snapped it up. The seller was great. The product came in good shape and in good time all the way from Japan so I'm really happy. I was a little worried about region but I was able to play it with the Power DVD with no issues. The song is one of my favorites from Exile the Second as is the music video so it is nice to have the hard copy for my library.",F +One of his beat albums. Just love the whole thing.,F +If you like Hoyt you should have this album. Just buy it. Really.,D +"A little bit of Bowie history, but the recording is not all that great. The bass on I've Got You Babe is ridiculously high and overpowers the vocals, I don't know what they were thinking on that. The "and friends" portions are mostly skip-worthy, especially since the transitions are so jarring at times. If you've never heard it before, it's maybe worth picking up, but I was underwhelmed. It seems likely there was only so much they could do with the source material, but the end result is just not that exciting.",p +This less known record adds another dimension to Mark Linkous's unworldly talent. A must-have for diehard Sparklehorse fans like myself.,I +"Very great album! From the moment I first heard this album, I loved it. 20 years later, a few of the songs seems somehow, melodically dated, but most are just as wonderful as I remember.",p +"This is an excellent CD to listen to, while making coffee (or tea), especially on a crisp morning. I find that listening to the CD during my daily commute also helps with the crazy urban traffic. It's upbeat enough not to put you to sleep, but soothing enough to keep me out of the road rage zone. Some if it is a bit cheesy 90's contemporary jazz, but overall it was very much worth the price. As I've pretty much said, the album is a great autumn one to wake up to.",m +Truly one of the most outstanding CD's ever. Can't say enough about this heavenly music.,f +I LOVE THIS MOVIE AND I LOVE THIS SOUNDTRACK!!!! this has to be my favorite live action Japanese movie of all time!!!,A +"Love this cd! Good quality, came w case and booklet, just one crack in the case, which is fine by me.",I +"Nice french press. Pretty good quality for the price. Kind of tacky vinyl sleeve, but it can be removed.*I received this item at a reduced price in exchange for a review.",b +"Purchased this because of the songs offered when purchasing a SoundBlaster music and video card back in the late 80’s. At that time, it was the best product on the market. SoundBlaster is now gone, but the music and isn’t and now my grandchildren get to experience what my children did, way back when! Will need a Dos emulation software to run the complete program with the videos to the music. This includes setting up a virtual drive. But if you just want to hear the songs, they will play on Windows 10 back.",s +"The CD contains a wide ranging collection of classical musical of a more soulful nature, hence the title of the CD. I really like the performance of Pavanne (a quicker tempo than other versions I've heard) and Chopin's Etude. Perfect for a rainy afternoon with candles lit and a warm cup of my favorite beverage.",m +"I love the movie and the tone that the music sets. Christopher Young is one of my favorite composers, I first noticed his music in the movie Rounders. He really knows how to set the mood of the scene. I had waited for a bit because this price was a little high. Once it got within range I got it and very pleased with my purchase.",e +I had this album on tape growing up and went to get it again so that I can still enjoy it.,z +"I had been looking for this cd for a while and I remember at one point-this cd was about $400 on Amazon, they must have found more or something, because I couldn't believe that I had gotten this for 12.99. I have always enjoyed the music to this as well as the movie and listened to it over & over again. I really have enjoyed Amazon, even more when they developed deals with used cd & book places. Very pleased with this purchase.",L +"Hole's Unplugged stands amongst the long running series best and yet, remains unreleased officially. This CD has been produced by a fly by night label looking to cash in on a loop hole of broadcasted material of a certain age and unfortunately, the quality shows. While the original broadcast had been bootlegged a few dozen times throughout the 1990's in varying quality, the majority of those CDs left much to be desired in sound. This CD runs in the middle pack of those old bootlegs. The CDs blips, drops sound, skips in audio and only a third of it will rip into a media library. Mid 90's bootleg entitled Unglued remains the pinnacle of this recording and even that is not up to par with officially released Unplugged recordings from other artists.Hole's Valentine's Day Unplugged deserves an official release and from one fan to another, this recording certainly isn't it. May the buyer beware.Hopefully Erlandson or Auf Der Maur will make good on their promises to keep Hole's legacy alive and officially release this set one day, in entirety (as of the date of this review, only ""Softer, Softest"" and ""Season Of The Witch"" have seen release in an official format).",C +"For fans of Cher, the ""Stars"" album has been a sticking point. There have been petitions, online campaigns and out right begging of anyone who will listen - including Cher herself - for a physical CD release of the album. There has been false starts, confusion as to owns the rights, and just time passing by without any sign of it on the horizon. Seems as though Cher & Co. have been beaten to the punch by some Argentine bootleggers who have taken the wide open opportunity to release it on their own with the two albums that came afterwards for this package.Is it unofficial? Yes. If the grammatical errors weren't enough to give it away, the flimsy packaging will (make sure to have a protective sleeve handy for safe keeping, otherwise this CD set will fall apart pretty quickly). The artwork is grainy, dull, poorly presented and pixelated.Sound quality? The sound quality quickly makes up for the shady packaging. Outside of time intensive, fan produced remastering, this is the best sound quality I have heard of these three albums outside of their original vinyl pressings.What is it? This isn't Married Cher. This isn't Disco Cher. This isn't big hair, fishnets, Rock Cher. This isn't Showgirl Cher. This is Cher in the 70's taking a more low-key, soft rock approach. Her voice has never been better or more present than it is on ""Stars"". ""These Days"" is truly a long lost gem that has been buried in time. ""I'd Rather Believe In You"" is split across the two discs and very much continues the sounds began on ""Stars"". Her version of ""Knock On Wood"" is not to be missed. Musically, the only pitfall to this set is the ""Cherished"" album that closes out the second disc. Cher picks up Storytime Cher (""Half-Breed"", ""Gypsy Tramps & Thieves"", etc.) and the results are very mixed. ""Cherished"" is definitely for fans only.Hopefully, Cher and Co. will come around to releasing these albums officially one day. They certainly deserve to be heard once again outside of bootlegged discs and fan forums.",J +Me encanta este álbum. Es poderoso! Tiene pistas al final. Lo Recomiendo,l +"It's only 2 pages, but there is good stuff here. It also backs up some of the stuff I've read about praise the the Gallup books and The One Minute Manager. One of the simple ideas is to ask your employee what kind of recognition is meaningful to them (of course based on their personality that can be a challenge on its own, especially when you are dealing with engineers).",q +Gift for a family member,w +"Who doesn't love Toby!I still like to buy music on a disc, rather than just downloading it.",M +"I love Jamila! She knows SO much and is a wonderful teacher. I got this CD and it is PACKED with all of the rhythmic variations ever used in Belly Dance. Get a copy and then focus on one rhythm at a time. Work your way through the whole CD (don't rush it; learn it,) and I guarantee, you will be the hottest zill player like you never thought possible.",m +"The packaging is good no harmful. This album is awesome, the songs are really catchy. Seohyu is my love ♡",\ +"My niece went crazy when I gave her this. She liked the cards/pictures and booklets that come with it. I’m not sure if it comes with the option to download the digital album, but it should, most people don’t have cd players/writers anymore",V +I was fortunate enough to see this group playing at the Orlando West walk way ; their music is so beautiful yet holds a beat that is energetic. My grandkids say they do not like classical music but when I play some of these music pieces I hear them say; hey this stuff is great.I also work out and kickbox and bike ride and this music has been a great addition to every day chores and work outs.I am so glad I bumped into this group; and how wonderful to see them as well; they were adorned in costumes representing the era of when classical music was introduced and then you hear them and look around and everyone listening is moving with the beats. Loved it !!!,m +Beautiful songs!,O +"This album filled my teen years, and I still search for it hoping it will be redone in digital format. Wish I had the means to do it myself.",f +This waa a wonderful Christmas gift for my mother in law.,C +"These songs may not have been originally recorded by Jackie Gleason's orchestra, but they are some of America's most beloved melodies. No vocals, just the soft instrumentals of music that should be preserved for generations to come.",k +This is a amazing story of courage and faith! You will not be disappointed. It is well made and the content of the story is exceptional!,\ +"When I got the package, I was stunned. I thought it was going to be a DVD! I even went back to the original ad, and there it was, the word "vinyl". So I played it, and since I had been in the last Armor OCS class, it was evocative and interesting. I'm looking forward to sharing it at our next reunion.",m +unique,U +My first venture into J Geils albums. Great collection. I was surprised to find how many I knew from the radio.,A +"I had heard of pianist George Winston but wasn't really familiar with this work. However, when I saw this CD at a used book sale, I thought it would be perfect for using when I teach yoga, and so I purchased it. As I had hoped, this music is soothing and peaceful. ""Color/Dance"" is calming, starting slow with a bit of build up (perfect for early in yoga class!). I also like the even-tempo of ""Stars."" Some of the tunes are a bit more upbeat, such as ""Moon"" and yet ""Too Much Between Us"" has a very quiet feel. Overall, I'm definitely glad to have acquired this album and would consider purchasing others in the series.",K +"I found this CD at a used book sale and purchased it to use when teaching yoga. The peaceful wave sounds are perfect for any type of relaxation, from yoga to massage to simply chilling out on your own. My only reason for giving it 4 stars is that the various tunes aren't really distinction from one another--but that will work perfectly for anyone looking for more homogeneous, uninterrupted background music.",` +"I am a yoga teacher, and I am always looking for peaceful, non-instrumental music to use in my classes. Native American flute tends to work well, so when I came across this CD of flutes, guitar, and keyboard at a resale store, I thought it would be perfect for my needs. Unfortunately no times are given either on the CD case or on the disc itself. When I played the CD for the firs time, I was pleased to discover that most of the songs are in the 5-7 minute range, which works well for my yoga playlists. I have listed the 12 tracks on the CD below along with times for each.1. Open Heart, 6:282. Above the Clouds, 6:053. On The Open Water, 3:404. Still Small Voice, 6:435. A Light Flickers Inside, 7:276. Mountain Song, 5:417. Silent Wings, 4:488. Everlasting Love, 6:469. Breath Of Life, 5:1110. Pacific, 6:3011. The Way Home, 5:4412. Wind Makes Waves Of Grass, 4:37The album begins slowly and gently with ""Open Heart."" In addition to the main instruments, soft chimes can be heard in the background at times. The next two tracks continue this methodical pace, with ""On The Open Water"" adding more guitar. ""Still Small Voice"" sounded similar to another artist's tune, although I couldn't quite place it. With very little sound other than the flute, ""Silent Wings"" was one of the quietest tracks, whereas I found ""Everlasting Love"" to be a bit more resonant. The one song I didn't like quite as much was ""Pacific""; the louder, sustained notes felt more discordant to me. Almost any of these tunes would be ideally soothing for shavasana, or final relaxation in yoga, and I particularly liked ""The Way Home.""I would recommend this CD for yoga, meditation, massage, or just easy listening at home/in the office.",I +"As the title suggests, this 60-minute practice by Stephanie Keach consists of a gently flowing series of yoga postures. Performed entirely on the floor, the poses offer a combination of restorative work and mild stretches. Stephanie starts you off in a supported cobbler's pose (using a blanket as a bolster), where she guides you through some basic pranayama (yogic breathing) work. Removing the bolster but continuing to lie on your back, you will perform knee hugs, hip opening work (including half dead bug pose), and a reclined twist. Next it's on to hands and knees for cat rolls, child's pose, and cobra, working up to a flowing series which seamlessly combines these three postures (Stephanie's study under Kali Ray is apparent here). Moving to a seated position, Stephanie performs neck rolls, a simple seated twist, cow seat, and then full cow pose (unfortunately, you may find that your upper body is not sufficiently warmed up for this posture). The final seated posture is head-to-knee pose using a strap; Stephanie has you flow in and out of this pose several times before holding it wherever you are able to do so. Returning to lying on your back, you will work your abs with knee drops and then perform bridge and crossed leg twist. Finally, Stephanie sets you up for a long (about 10 minutes) savasana, with only the sound of gentle music playing in the background until a soft bell calls you to finish the practice.This video reminded me very much of Rainbeau Mars' Pure Tranquility, partly because both practices are performed entirely on the floor, and partly because Stephanie's teaching style is very reminiscent of Rainbeau's, particularly with her frequent focus on the breath. Stephanie also talks about our overactive ""monkey mind,"" encouraging you to let go of thoughts and to concentrate on your body instead. Overall, this is a very nice, calming, relaxing practice accessible to all levels of yoga practitioners.",] +"NOTE: This review is based on reading the book, not listening to the audio version.In the second chapter of this book (actually labelled as ""3"" because this is the second prime number), we are introduced to Christopher, the narrator, who is writing a mystery novel as a class project. Christopher is an extremely intelligent 15-year old who just happens to be autistic. He begins his tale by relating his discovery of a neighbor's dog, which had been murdered with a garden fork. Because he likes both dogs and mysteries, Christopher sets off to find the killer, but what starts as a story about a dog becomes a story about Christopher's life. Not only do we learn about Christopher's many quirks, but also we learn the explanations for his seemingly irrational behavior, making it seem almost logical. The book is surprising funny, as Christopher, with his limited emotional range, makes the perfect straight man. As a psychologist, I found that the depiction of Christopher rang true, and overall, I found this novel to be a delightful, different, and quick little read.",r +"DescriptionThe country singer's first five albums on one 2-CD set plus a non-album B-side. Ten hit singles including the US no. 1 ""Harper Valley P.T.A."" as featured in 'Mad Men' Season 6.Disc 11. Harper Valley PTA2. Widow Jones3. No Brass Band4. Mr Harper5. Run Jeannie Run6. Shed Me No Tears7. The Cotton Patch8. Sippin' Shirley Thompson9. The Little Town Square10. The Ballad Of Louise11. Satan Place12. Yesterday All Day Long Today13. Yearbooks And Yesterdays14. What Was Her Name?15. Edna Burgoo16. My Scrapbook17. The Part Of Honey18. Whatever Happened To Charlie Brown?19. The Girl Most Likely20. Back To School21. That's How It Is With Him And Me22. Teardrops On Page Forty Three23. Box Of Memories24. A Taste Of Tears25. The Rib26. Sunday After Church27. The Artist28. I'm Only A WomanDisc 21. A Real Woman2. There Never Was A Time3. Things Go Better With Love4. I'm The Woman5. The Wedding Cake6. Thin Ribbon Of Smoke7. Our Minniev8. The Back Side Of Dallas9. Country Girl10. A Change Of Heart11. Am I That Easy To Forget?12. That's A No No13. We Were Raised On Love14. I Almost Called Your Name15. Darling Days16. Wish You Were Here17. Your Old Love Letters18. I Love Him19. Wherever You Are20. The Generation Gap21. Fine Feathered Fowl22. Words Names Faces23. My Man24. He Made A Woman Out Of Me25. Duty Not Desire26. Games People Play27. Darkness Falls28. Holdin' On29. Okie From Muskogee30. To The Other Woman",q +"Antonio Bribiesca had a long, long career and played with all the greats of Old Mexico. Listening to this takes you back in time to the great music of the 1950's to the 1970's. This CD has some tracks solo guitar, some tracks have Mariachi and some tracks are solo. I recommend this wholeheartedly!",B +excellent CD... love it..,g +I like it,n +"I had this recording on vinyl when I was a teenager, and we learned most of the songs and sketches by heart. I still find them hilarious, but I can't guarantee that you will if you have no appreciation for the mimicry of Sellers and the skill of his writers. This is where I first heard 'All the Things you Are' sung by an old Londoner in his bath, with the Percy Faith orchestra there in the tub with him. Great stuff which brings back happy hours with the Philips reel-to-reel...",c +Love them,J +Reminiscing the good ole days. So long Kenny. Thank you for the music. Glad to find this!,\ +I love just about everything Vikki Carr sings. The rating is not about her but the way a fragile item like this cd was packaged and shipped. Delivered with several cracks on the jewel case. Fortunately the cd was not damaged. Amazon needs to review its shipping policies.,V +"Honest and straightforward! On a vein with King's Record Shop, this album is one of her overlooked gems! Glad I finally found and bought it.",c +"Bought because I loved Patsy Cline even though I wasn't that big of a country fan. What a surprise! Three great stars for one low price! And 9 awesome recordings!Got it home, played it and it immediately became one of our family's go to favorites!No, it's not all their big hits, but they very well should have been as there's not a bad cut to be had in the bunch.T. Texas Tyler, Webb Pierce and if course, Patsy are all well represented here with cuts that are engineered well and showcase each artists range of talents.I went into the Navy and when I came home my poor album had seen many visits to the turntable and was essentially worn out! Cover was falling apart and there was no inner sleeve to be found. It's taken me 35 years to find it again in decent condition.This time it will be covered!",A +"I've looked and looked for a complete ""best of"" or anthology collection of his works for years. This, so far is as close as it gets. Nice compilation (so far) with only a few missing. One day they'll get it right. But for now, this is the best you'll get.Unfortunately, Johnny was pushed in directions he shouldn't have taken. And forced, at times, to put out less than stellar collections of songs or arrangements that while pleasant, were not the best he could have done. Johnny is a class act with a gift! This collection offers the most of his best in one package. A great investment in worthwhile listening!",E +"It's a complete, classic, hit riddled slice of Bobby's recording career. We were so pleased at the quality, quantity, and complete collection of must have offerings from someone taken too soon. We miss you Bobby!",l +Always was and probably always will be one great party band! I really enjoy this album! Another awesome addition to my collection!,Q +"A friend of mine got this for Christmas the year.it came out. I wanted it but didn't want to buy the entire collection for this one disc.Our Day Will Come, Tell Him, I'm Leaving It Up Yo You, and Easier Said Than Done are all hard to find! Love this CD! Great year! Great music!",u +"It's really very costly these days to get what you pay for but this collection delivers. Made from original masters and does not include any re-recorded material to my knowledge, unlike some other greatest hits albums. Excellent buy!",f +This is a classic! My third pressing! Just play it till I need a new one! Was so glad I found this gem on new vinyl!!!It has been available on CD and I have one of those. Don't know if you can still get it on disc.If you're a classic country western fan or just love country swing this is the album you've been looking for!Rest in peace Ray and Bob. At least we have this awesome recording to remember and honor them by.,W +Great Collection! I only bought this because my vinyl is in storage at the moment. I was presently surprised by the songs included and the order the songs are in. Makes for a great cross section of the career of this underrated incredible band.,k +"Gotta admit I was prepared NOT to like this LP. I got it for Christmas as a gift. Was I surprised.For his age, Neil's voice is in fine form and this is one of the better albums he's done is some time.It's as good as his earlier stuff when you could put on the album and the entire thing was great to listen to.I loved or liked every track and was surprised to find it in my player so often since the holidays. Out of allthe CD's I recieved as gifts, this one turned out to be the one I liked the best! And I ASKED for the otherones!!Good solid songs, good quality arrangements, with quality production from the folks at Capitol. Listeningto it makes me want to see Neil again in concert. I haven't been since the late 90's but, even then, he puton an awesome show!",a +An incredibly diverse collection of some older Christmas Classics as well as some newer soon to be Christmas Classic by current contemporary artists like Aimee Mann & Dianna Krall. A collection that is a great Value for the Money and well worth adding to your Christmas Collection of Holiday Tunes to make the season bright!,g +"Having been unable to find this issued on CD, I'm forced to listen to either a recording of this on a cassette of my vinyl (to keep from wearing out my vinyl) or the vinyl it's self. I'm not a big fan of downloads or manufactured on demand CD's either so...while I did download this, I don't think I'll be doing much more of it.I purchased the LP when it was first issued. The sound on the vinyl is 10 times better than the MP3 download but, it's better than nothing.After almost 40 years of listening, this is still one of my favorites by Steve Miller. While this isn't a ""top 40"" release, there are some great cuts on this LP. One half the LP is live, the other studio. The title cut (Rock Love) could easily been released as a single and probably would have done well for the time. I love all the blues cuts on the LP and while this is a pared down unit for Steve Miller, this LP will suprise you. With repeated listenings, I find it to be more and more one of my favorites of his and the only thing I find wrong with it is that it's one of his shorter LP's. With only 8 total cuts. Harbor Lights is probably my favorite cut but, honestly, there are NO BAD CUTS on this LP.",w +Received this and it is a very good CD.,J +"This CD is just wonderful. I had a tape long time ago and have been looking for it for quite some time, and here it is. The CD was recorded in 1965 but the music is so beautiful. I love Gregorian and Orthodox chant. The monks from Chevetogne sing both. This one is a recording of the traditional orchodox vespers, and in the prayers you can hear the names of the king and queen of Belgium at the time.",G +"This is a reproduction of an older CD. It contains work from Fuenllana, and pieces he rewrote for vihuela. I love the voices of Nuria Rial and Carlos Mena, had seen the video on YouTube, and then found this CD. Most were very expensive, but this was at a decent price so I bought it immediately. An absolute winner !",E +Very little to say - I just adore Orthodox service music and this is absolute favorite !,Q +Bought as a gift and I believe it was well recieved,S +Amazing song on an awesome translucent blue vinyl single. Pressing was very good quality. Pretty typical for colored pressings.,Q +Love it,a +These 4 individuals are sooo talented and have such great presence- hope for more music from them in the years to come!,_ +These are all amazing. I use them to wind down for sleep and the combination of guitar and other calming sounds works well for me!,k +"I've own this set since I seen it broadcasted on TV several years ago. To the best of my knowledge the 2 cd set is no longer in print. The single cd just doesn't do the ""old dogs"" justice. The songs are hilarious and you have to listen to the lyrics to catch all of the innuendos in the songs. Such as ""ain't it hard when it ain't"" or ""she rather be homeless then be with me"". Then there are songs that ring true to life, ""music is hard on the living, but sure speak well of the dead"". It's about how everyone seems to critize the lifestyle of the famous but when they pass on they praise the artists.",y +Speaker rambles. Too many "stories" which make this drag on. His accent makes it hard to listen to.,w +"I was expecting this to be similar to Peter Furler's first solo album, but it is totally different. I guess I just miss his old Newsboys days--the Newsboys now are not the same without him, but I know things change. This is, however, a nice, slower, worship album,",S +"I searched previously for meditative music; I have insomnia. Thought it would help. Just got it today, but I have heard it and really think this CD will help. More later.",B +I love this group! Everyone needs to give them a chance!,F +This is a gorgeous CD. Highly recommend. Service was great.,` +I am looking for a song that Charlie Zahm sings named the ad on of Canada. Is the song on this CD?,w +"I love the music, and this is the one and only CD I have purchased through the mail that arrived without damage to the case. The packing for this terrific CD was exceptional, and the case was not cracked, as had been my experience in the past.",o +Absolutely Love. Comes great packaged.,T +Amazing album. Had it on cassette years ago and still love these tunes.,Z +Moms is drowned out by the crowd and she also sometimes was not right at the microphone.I recommend a studio recording. Some of the jokes are out of date- like the gay jokes. Otherwise Moms Mabley is very funny.,e +Only got the bonus CD and it was scratched and case cracked.,\ +No dislikes . Love the music of Janice Martin she has great music great sound beautiful voice first female Elvis I love her CDs and I ordered a few more can’t wait to get those,r +I loved all the songs on the CD and the music sounded great and I enjoyed every moment of listening to Cat Stevens.I had a big crush on him and I still do he was such a hunk all day girls were crazy about him.,x +"My ballet teacher used this music in class today. If you love classical-type music play with skill and expression, this is for you. Beautiful and a joy to dance to!",X +The CD arrived perfectly in great condition.,r +"This album is great! Has pictures of the band, and other things along with the actual CD. It looked like a book when it arrived, but is much better than I expected!",v +"I'm not going to go into much detail on the actual World Domination CD, which is absolutely amazing. But talk about the blu-ray this version comes with. It's a live performance from their show at Shinkiba Studio Coast on November 23, 2017. The set list are: "Choose Me", "Don't You Tell Me", "Moratorium", "Puzzle", "Cross", "Matchless Gum", "Awkward", "Daydreaming", "YOLO", "Alone", "Real Existence", "Take Me Higher", "Play", "Secret My Lips", and ending with "The Non-Fiction Days". Now... I understand you could simply overlook this band as some J-Pop gimmick niche group. Something along the lines of Babymetal, which I actually also am a huge fan of but for different reasons. But don't let the cute personas and maid uniforms fool you. Every one of the band members are extremely talented. I grew up listening to the hard rock and heavy metal of the 80s and 90s, and I associate their sound to the hard rock from back then. Especially in Kanami Tono's guitar, Misa's bass, and Akane Hirose's drums. Once you're able to look past the cute girls in maid costumes, these girls rock. That's not to say that Saiki Atsumi's vocals and Miku Kobato's rhythm guitar and backing vocals don't, but the real meat and potatoes of the band's rock sound are Kanami, Misa, and Akane. From Akane who pounds the skins with her double bass pedal/single bass drumming, to Kanami who shreds as good as most guitarists from my 80s rock bands, to Misa who plays her five-string bass like it's a damn lead guitar. I believe one thing that can throw people off from getting into Band-Maid is the feminine vocals, but I grew up with there being girl rock bands so I'm not effected by it, but I can understand. Besides, Saiki can go from j-pop sounding, to driving rock, to low and sultry. What's not to like? As for the concert.... if you're a fan of Band-Maid, the live performance is going to blow you away. Getting back to Saiki's vocals.... they were powerful and spot on. I think that's what I take most from this blu-ray is how incredible her voice is live. Also, if you're a fan, you're used to every track giving each member a chance to stand out throughout the song. No one takes a back seat in this band. And this concert was no different. Most songs have an extra little intro before the traditional song begins, from Misa and Akane driving in the next song's tempo on tracks like "Real Existence" and "Play", to Kanami playing the piano before "Daydreaming" (unfortunately a pre-recorded intro. She wasn't actually playing it on stage but it was her) and having a back and forth between herself and Misa on "Take Me Higher".... every song has a reason to make you glad you paid a little extra for the Limited Edition. That's not to say I don't have a couple complaints. Miku's mic.... it was way too low. You could barely hear her. And on a song like "Alone" that's known for her duet with Saiki it really stuck out. Saiki had to practically whisper to not drown her out. And there were a couple parts in the blu-ray where it seemed to fade out and then in between a couple songs. I have a feeling it cut out Miku's usual crowd interactions where she pumps up the crowd and is her usual kawaii Miku self. They may have cut those scenes to keep the flow of the blu-ray faster. If that is in fact what was cut I would have liked to have seen it regardless. But those are small gripes in the full spectrum of an AMAZING live show. Half the reason I bought this was because, living in the states, I knew I'd probably never get to actually see them live in person. More American venues really need to open up to these Japanese bands. You'd think they'd get the hint when seeing these bands sell out in record numbers wherever they do actually play.... but I digress. If you're a Band-Maid fan, and Amazon actually has these in stock.... I had to buy mine from Band-Maid's website... which was also $20 cheaper... BUY THIS LIMITED EDITION. You won't be disappointed.",R +Just love this CD I have four of their CD's and love everyone of them. Completely enjoy their music. Well worth the money.,V +Nice music. Arrived quickly.,i +"Came quickly, packaged well. Great music.",e +Love the songs.,] +"These are beautifully made and came with a nice quality chaplet card as well. This is a powerful, fairly quick chaplet to pray and much needed today! The price is so much better than elsewhere and I am very pleased with my purchase!",R +Flanders & Swann are very entertaining . I am delighted with my purchase.,q +These women can really sing - I had heard a clip from them on a page I follow and wanted to hear more so I was elated to find this cd!,U +"What a refreshing change from the routine music that is being performed today. Great rhythms, lyrics, and vocals. Who knew electronic dance music could be so versatile?",N +What was expected.,X +Good quality good music,o +"We've done this drive with a number of different audio tours - this one is by far the most entertaining. The music is wonderful, and we bought a copy just for the songs. I love maui!",Z +Love it great live performance from the great Tom Waits. There is not one bad track on the cd,C +"If you want of real taste of Alaska, buy this album. Alaska folk music at its best. It reminds us of our Alaskan vacation whenever we play it.",G +This was the age of the greatest group ever to hit the stage! I purchased this music fir the memories of the past! The music can’t be matched!,l +Alternative mix of Brainville and I love it,w +"I listened to this LP as a child, over and over again. I can still remember some of the songs and I am near 60 years old! These are just cute little songs for children without being tied to merchandise or comic books or T.V. shows.I would love to find this on CD for my grandchildren.",c +LOVE this CD! great music from very talented performers. Great gift idea for music fans.,e +"I have played this CD over multiple times and it has never failed to point my spirit and mind upwards (like all his other CDs, too). I am often reminded of how the evil spirits left Saul when David sang unto the Lord whenever I hear Marty sing. Marty is one of the very few singers out there who truly sings from HIS HEART unto the Lord, and not unto men -- and there truly is a big difference.",a +This CD has a wonderful collection of overovertures I am very pleased with my purchase. Never tire of listening to this great music.,_ +Excellent condition,z +This was a gift for my husband. He was happy and enjoys listening to the CD. Great price on the CD also.,l +"First heard the title track on the radio in Washington DC in 1997, two years after it dropped. Immediately tracked down the CD and listened to it on the train all the way back to Southern California. Melodic, calming, evocative. I still listen to it.",l +"If you like the REAL music of the REAL Sons of The Pioneers, don’t buy this piece of over-produced crap! The Sons are drowned out by the soaring violins and the over-orchestration chosen by someone who is madly, stupidly in love with ELEVATOR MUSIC . There is NOTHING REAL in this waste of plastic, money and time.",c +Playing soon,P +"The sound was very bad. I have a new sound system, but this surprised me.",a +"Playing soon, new system",m +Getting ready to use.. music system just arrived,u +I had forgotten what I was missing in my music repertoire. Thank you Ms. Mitchell!,j +The CD ministers to me and love Babbie Mason's music.,d +"Great music. Performers are impeccable, wonderful jazz solos, "Sinatra" big band style arrangements of Christian songs. FABULOUS chords and rhythms. This cd will lift your spirits.",G +"I appreciate Michael Faircloth's work. The hymn arrangements included are creative, varied, exhibiting immaculate pianistic technique and are recorded on a fine piano. The clarity of the sound is great. There are few pianists whose work is offered without being covered or supported by other instruments. Michael's pianistic expression is enough in itself. I would buy any new CD's if he offers them.",J +"Very appealing piano solo arrangements, very well played. Michael has the clean technique of a well trained artist, but is also very creative.",y +"For the listener, the music is creative, a different take on hymn tunes. I found it enjoyable. I also have the printed piano book, and some of the tunes are written as chord charts, which I can use, but don't work well with my students.",\ +"I get that they don't want to overwhelm Suzanne Giesemann's voice with the music, but it's too soft relative to her voice and since it's Hemi-sync music isn't part of the gain to be able to hear the music so you gain the benefits of Hemi-sync? If you turn up the volume to hear the music then Suzanne Giesemann's voice will be too loud. There was no sample track available on Amazon but you can go the Hemi-Sync website and listen to a short sample there. I would have liked just a Hemi-sync music track to meditate to as after awhile I image you will remember the meditations so the option of just the music would have been nice. This is part of a 4 CD set by Giesemann to support mediumship development. The 4 CDs are The Training Ground, Building the Power, Working with Your Guides, Making the Connection. This CD, Making the Connection, has 2 tracks slightly under 30 minutes each. At the Hemi-sync website you can purchase this and the other recordings as digital downloads for less than a physical CD and you can also purchase as a set. I didn't see the digital download available on Amazon. I decided to purchase the CD as I can pass it along to a friend when I am done using it.There is a simple guide booklet included, but you can get the same info and maybe a bit more watching some of the Suzanne Giesemann's videos and interviews at YouTube and she also explains her BLESS ME method on a video on YouTube as well. As with many CDs these days the CD does not come in a polystyrene case to prevent more trash in a the landfill that doesn't break down easily. It will arrive on a mostly paper case.You can read about the mediumship series at the websites for Suzanne Giesemann and Heni-Sync.Here is a copy of the description of this CD from Suzanne Giesemann's website as none was posted on this page:This 4th recording in the Mediumship Series by Suzanne Giesemann takes listeners to unlimited levels of connection in the spirit realms. The recording features two tracks: Track #1 is a guided meditation to help you learn to easily shift your awareness and vibration to meet those in alternate realities exactly where they are. Track #2 is mostly non-verbal, allowing you to have your own experience as the Hemi-Sync® frequencies assist you in reaching an expanded state.I hope this review helps you decide if this or any other of Suzanne Giesemann's CDs/digital downloads are for you. There are also some free materials at her website that do not require you to put in an email. I suggest visiting the Hemi-Sync website and listening to samples as well as reading the descriptions. Personally I am not interested in learning to do the traditional mediumship/channeling contact. I want to use some of the info expand my consciousness/awareness. I don't recommend or have an interest in contacting or channeling spirits or entities.",n +This is a great album and it sounds great especially in my car which has a pretty good sound system...it's an oldie but a goodie..rock on!!!,k +"Arrived in perfect condition,Wonderful singer",D +I wanted one of the songs on the album and was fortunate to find this cd with its music and then other gems,m +Good songs,C +Love this cd!,Y +I liked this one....he is my favorite trumpet player!!,[ +I played this on my boom box that's in my bedroom and I should have played it earlier as it kept me awake from this guy's great sense of humor. I got it wrapped in cellophane and earlier than expected I have no beef with this seller.,\ +Sting is hands down my favorite musician and he never disappoints.,E +Have been listening to my cassette recording for years. Glad to have it now as a CD.,C +Loved it,H +I wish online sheet music sellers would post the key the music's in or the first page of the music so I could see if it's something I can probably play. Not everyone is a virtuoso.,A +No me canso de escucharlo,o +Bellas canciones,^ +for me this cd is timeless. i listen to it every month and love it just like i did when i first bought it. i'm glad this one is in my collection. i love lalah's voice. i hope to see her in concert.,[ +My daughter loves these catchy kid-friendly songs that also teach her scripture. We listen to it in the car.,a +Gloriously recorded and beautifully sung to the glory of God! You won't be disappointed!,W +Love these. Raising my son bilingual and this compilation of short stories are perfect exposing him to my mother language while traveling in the car.,Z +Brought back so many memories of my teenage years in London. Couldn't listen to it for too long as I was unable to keep back the tears. Fabulous CD!,w +"I've loved music all my life and have collected vinyl since the early 70's. Later, when vinyl dwindled, it was CD's. Lastly, it was digital versions that got added to the library. As I've gotten older, I seem to drift back to earlier days and recently, Alice Cooper songs kept running through my head. So....I wanted to add the CD version of their earlier albums. No way to do that without including Billion Dollar Babies! I was completed pleased with the quality of this recording and found the tracks to be clear with robust sound! It was worth adding to the library and I've enjoyed listening to it.....LOUD!!",q +Love ALL of Paul Wilburs worship music. He is a favorite,] +"The music is good, vocals are good, as usual with The Who this cd is different than anything previously released. Case was damaged (unusable).",r +"Mailed straight from Athens, this dreamy CD arrived. The singing and bouzouki playing are just gorgeousand the real McCoy. If you like Greek songs or the Greek language (I am studying it) you must have thisrecording !",A +"An outstanding LP showcasing the best of Rita Coolidge. If you can find a copy, buy it!",V +"Was listed as new, but when I got it, the CDs were scratched up badly and several tracks wouldn't play.",V +I thought I was ordering the whole CD not just single CD,r +"I enjoyed this book. It has a lot of useful advice and some good reminders. A lot of the content is covered in his other books. This book is short, to the point, and nearly all the information is relevant to the topic.",R +Love it!,l +Rich Little is great.,P +"This album is Grammy material. The worship is awesome and she has a voice just as powerful if not more powerful than Celine Dione. Love the live strings through out, gives the entire worship a different feel. Keep it up!!! can't wait to see what's next!",q +Love the album got Solar’s photocard in mine,Y +Everything I wanted I also got Ken’s photocard in mine,F +wonderful,P +I love Ronnie Milsap and have been a fan of his for many years. This is a rare version on vinyl and all of his songs are great.,D +"This only includes a smattering of Ronnie's greatest hits. It has a few of his more obscure songs that probably should have been hits. There are 36 songs here, so you get quite a bit of listening time. Most fans of his early country work more than likely already have these many times over, but still, you can't go wrong with Ronnie. He has always had a wonderful, soothing voice and this set is no exception. If you happen to be a collector of his work like I am, this is a must-have for your collection regardless.",n +This is yet another spectacular country album that Ronnie Milsap recorded for RCA. The title track is full of his signature vocal crescendos. Absolutely beautiful!,M +"This is a three CD set of Ronnie Milsap's greatest hits. It's a great addition to the collection of any Ronnie Milsap or classic country fan. You get 35 (out of 40) of Ronnie's top country favorites here, all of which went to number one on the charts.",Q +Anything Sandi sings is beautiful ♥️Her voice is so clear and you understandevery word!! ♥️,K +Heard this album many years ago and was sad to have lost it. Well now I have a new one.,T +Bought this as a baby shower gift for my Beatles loving son and his wife. Apparently newborn Penny likes it as much as they do!,M +"Talented artist. I love the Beatles Covers, and he has picked out most of my favorites. Can I request "Yesterday" make it on the next CD?",E +"The narrator is simply BRILLIANT.He is very, very good.He gets into the mind of each character he narrates and talks as they were experiencing it.Wow !",q +Best album ever.,u +Good to hear "real" country music again! A real bargain!,E +Now my collection is complete.,p +an awesome collection of his greatest hits..,q +Great music,O +"I like this guy's guitar work - so the hard feelings aren't there. BUT...This is a TWELVE INCH EXTENDED PLAY 45.The two concerts the material comes from have not been issued on CD or vinyl before - I will be buying the material again to get the entire (or more complete versions) of the concerts.The segue into Hendrix's "Third Stone From The Sun" was what I bought the record for - I may have paid a bit too much for four tunes, but I got what I wanted (for now).",J +"CSN is better than 4 Way CD, which is a live concert.",A +A gift,N +Just discovered these guys a week ago. Such infectious grooves! I wish i would have found this band sooner. This CD is packed with great jamming music. I cant stop listening!! This import's a little pricey but OMG Worth EVERY Penny. I have not been so instantly addicted by a band in i can't remember how long. So good.,O +"What can I say - they are rubber tips, there are four of them, and they fit.",y +"[[VIDEOID:66d6b825fffbb69e86f95ae570ba4a79]] My children are like me and like to sleep with some white noise. We all sleep with a box fan going, but the kids have a tough time falling asleep some nights. I put a small music player in the room and put in this CD. My youngest has been listening to it at every nap time and every bed time and loves it. The CD he had before was a bit loud at times so he either missed the music when we turned it down or got scared when the waves would crash a bit loud. This CD seems to have a good balance.It starts off with ocean waves and moves into soft music. It's soothing and goes on long enough that I simply press play and let the CD run through each time my little one needs to sleep. It has helped him sleep better than without music and he asks me to "hit play!" every time. We are happy with it.*I RECEIVED THIS PRODUCT AT A DISCOUNTED PRICE (IN ORDER TO TRY IT AND REVIEW) THIS DOES NOT GUARANTEE A GOOD REVIEW OR CHANGE MY OPINION ON THE PRODUCT, I AM HONEST AND UNBIASED, ALL OPINIONS ON IT ARE MY OWN AND BASED ON THE PRODUCT ITSELF. I HAVE TRIED THIS AND EVALUATED IT ACCORDINGLY",] +"I'm not a big fan of novels about spies, secret operatives or hit men, but David Baldacci is one author who hasn't disappointed me so I decided to give his most recent book, The Innocent, a try in audio book format. The book was read by Ron McLarty and Orlagh Cassidy -- both did a great job.In this story Will Robie, is a hit man for the US government. His targets typically are terrorists who are planning to harm or threaten the safety and security of others. He is very good at what he does, but there is something about a particular assignment he's given that doesn't sit right with him. He refuses to do what is expected, making him the hunted by those he works for.While Robie is on the run he meets a fourteen-year old girl named Julie Getty who is also running for her life Julie's parents have been murdered and she may now be a target herself. Against his better judgment, Robie can't run away from her, but he has no idea what he is getting into.This story is fast paced and takes the reader on a wild ride. Numerous twists and turns had me paying extra close attention to what was going on. I found myself having to listen to a bit more and a bit more, finishing the audiobook much sooner than I expected to. There are some heart-pumping moments, and several sub-plots as well. It's one of the books where you can almost imagine yourself watching the action. The sound effects used in the audiobook intensified this effect, and readers who read this one will be anxious to see how the story lines all tie in.",l +"Doing more by doing less is a seductive concept. But is it possible? Yes, says this how-to manual on essentialism. The formula for doing more by doing less is to discern what is absolutely essential, eliminate the rest, and get those things done with as little effort as possible writes author Greg McKeown. McKeown is CEO of a strategy company in Silicon Valley, co-created a course at Stanford titled "Designing Life, Essentially" and speaks at companies including Apple, Google, Twitter, Facebook, Salesforce, and LinkedIn.This book may not be for everybody. If your life is manageable, filled with satisfying activities, and you're progressing at the pace you want, you may not need this book. But for those who feel overloaded, distracted, stuck in the mire of doing a lot but not progressing on what matters to you, you might find it of interest. Although there are time and life management books by Stephen Covey, Brian Tracey, Julia Morgenstern, David Allen etc., this book approaches life management from a fresh angle: essentialism. It is filled with contemporary examples which are relevant in 2014.Four E's constitute the process of essentialism says McKeown: Essence, Explore, Eliminate and Execute. The goal is to do less, but better writes McKeown. It's a disciplined pursuit of less he writes. "If you don't prioritize your life, someone else will," McKeown says. He recommends asking yourself continually: "Is this the very most important thing I should be doing with my time and resources right now?" Or, to discern what is essential to you, how about this question: "If you could do one thing with your life right now, what would it be?" The aim is to live by design, not default. You practice distinguishing between the trivial many and the vital few.Under the umbrella of each of the four E's of essence, explore, eliminate and execute, McKeown lists mindsets and actions to live more essentially. Take execute, one of my favorite sections, McKeown outlines: buffering - prepare contingencies and expect the unexpected, subtracting - bring forth more by removing obstacles, progress - the power of small wins which harnesses the power of steadiness and repetition, flow - capture the genius of the best routines, focus - figure out what's important now and be - the essentialist life of more clarity, more control, and more joy in the journey.Threaded throughout are abundant examples of individuals who live by the principles and actions described in this book. Warren Buffet seems to practice essentialism in his approach to investing about which Buffet says humorously: "Our investment philosophy borders on lethargy." Doing more by doing less. There's the example of business prophet Peter Drucker who is quoted forecasting: "In a few hundred years, when the history of our time will be written from a long-term perspective, it is likely that the most important event historians will see is not technology, not the Internet, not e-commerce. It is an unprecedented change in the human condition. For the first time - literally - substantial and rapidly growing numbers of people have choices. For the first time they will have to manage themselves. And society is totally unprepared for it." This book offers ideas on how to manage yourself and what is essential to you.Clarity = success promises this book. It makes sense that if you practice essentialist principles, it will revolutionize your life. You will create more of what you want, and eliminate more of what you don't want, enabling you to do more by doing less. This is life transforming, and one of the best books I have read recently in which the message is potentially life-changing.",w +"I am a big fan of Andrew Lloyd Webber's musicals. Cats contains the very well-known song "Memory." Otherwise, there aren't many memorable songs in this musical. It also is a review, which means that there is no real plot.",n +"First of all, not read by Clarkson. Second, not all that interesting. Read the books by him, watch the show(s) he's on, but you may want to pass on this one.",D +Glad I bought the cassette. Bassey sings great but the background accompaniment really hurts her performance. More of a novelty than something to pursue unless you HAVE to HAVE it.,H +"I had dinner at Harris' and enjoyed Susan's group and her singing. She has a passion and love for the music that has stayed with me over the years. I always watch for her when I go to Harris'. I'm getting older and I see that she is too, but the flame of her passion for jazz does not dim. My favorite: "Slow Boat to China"",c +"AIURa was a 4-frame comic series transformed into a 4 minute anime series. The OP and ED songs were great, with the ED sounding like a well done school anthem. This CD has four tracks, and only one of them is one I was interested in.Track 1. Extended version of the AIURa OP.Track 2. Unrelated song. Title 限りなく.... which my dictionary tells me is something unlimitedTrack 3. Karaoke version of Track 1Track 4. Karaoke version of Track 2If this had only two tracks, the OP and the ED, and no karaoke or extras, I'd have given it four stars, at least.As it is, Track 1 was very good, and maybe worth the import price if you _have_ to have it.",c +"this is one of the best albums I ever heard. also a lot of songs in this album are great for running, like "been told".",W +"It's a treat to play year after year December after December. I love the order of the songs, the lavish orchestral acompaniment, and the boy choir supporting Jessye Norman's astonishing voice. A beautifully produced CD, by far my favorite Christmas album.",O +Very happy with this Great Score!! Thank You very much.....wish these girls would come over and beat 'Me' like a drum!!,m +"These "secret tapes" are live performance recording that were either lost or never meant for distribution. First thing to know- though Sony has remastered the Tapes- they are old, of poor quality, meaning lots of hiss and noise.The Bach Concerto nr. 1 is familiar stuff to Gould fans. Sonically this recording with Mitropoulos and the Concergebouw is not much better than the Ottawa videos that are available on Youtube. This is a live recording from the 1958 Saltzburg Festival.The Beethoven Concerto #5 op. 73 was recorded live with the Buffalo Philharmonic under Jospeh Krips. This often under-rated orchestra struts it's goods in a fine, sensitive performance. Though Gould is a bit noisy- singing a bit loudly, even for him, I think it's one of Gould's best Beethoven's. There's even a bit of tenderness in Gould's playing- an approach not usually part of his playing. I wish the recording were technically better.Last is Schoenberg's Klavierkonzert op. 42. While Gould is best known for his Bach, he was a great advocate of Modernists like Schoenberg and Webern (he made at least 5 additional Schoenberg recordings).Though the live recording, made with the New York Philharmonic in 1958 under the baton (again) of Dimitri Mitropoulos is very noisy(tape hiss and audience coughing), it's worth hearing for his very sophisticated interpretation of a difficult work. He was 26 when this recording was made.",D +"I was so excited for this series, because the Earworms website promises, ""brain optimized, accelerated learning."" I was really hoping for something revolutionary, like the [[ASIN:0596007124 Head First]] series of books. But, unfortunately, this is just a very mediocre introduction to Japanese. I was so disappointed.I bought the [[ASIN:1886447209 Rapanese]] cassette tapes over ten years ago, and they were goofy, but they taught me a little Japanese. I bought this CD excited to see what leaps Japanese instruction had made in the last decade. Sadly, that dated, wacky audio cassette was much more advanced than this CD.The 'music' is super cheap stock music, or elevator music. The voice actors talk in a little bit of sing-songy voices, but definitely not on beat with the music (the music was mixed in afterwards). You'll probably pick up more Japanese by reading the Wikipedia article on Learning Japanese, than you'll get by listening to this CD.The voice actors are by far the best part of this. With every other part of this, you'll wish you bought some other method of learning Japanese.Pros:+I liked both of the voice actors on this CD. The male actor has a nice British accent, and the female actor has a nice Japanese accent.Cons:-If you're looking for something that's revolutionary in any way, you'll be super disappointed-Super annoying music-Super annoying sound effects-Really bad sound mixing. The background music is too loud-The dialog is a rehash of every other beginning Japanese course ever written",A +"I purchased this record before leaving Phan Rang, Vietnam in October of 1967. It is a documentary record which covers the ground war, air war and river war. It is a ride back in time for me and lets me reflect on my life at the old age of 18 and 19 years.",h +"The first couplet of the first song begins:You've got a smile so bright, you know you could've been a candle.I'm holding you so tight, you know you could've been a handle.Anyone who has heard of the Temptations knows those lines from their classic ""The Way You Do the Things You Do."" It is crisply sung, almost doo-wop. It never loses its poise or falls into cliche.Most of these songs were hits. If the titles don't jar your memory, listen to the samples. Expect to know at least 10 of these.A couple weak links are present in ""Treat Her Like a Lady"" and ""Error of Our Ways."" In these two songs, there's no funk, no soul, and no depth. Both are acceptable, as pop R&B, but are musically pallid when compared to ""Papa Was a Rolling Stone"" and ""Ball of Confusion.""The liner notes provide a puffy history, and credits song-by-song. That's not why this is worth buying. It is the sheer cool class and sway of the Temptations greatest hits. A couple songs might be left off, but most are here. Be sure to find the best price.1. Way You Do the Things You Do2. My Girl3. It's Growing4. Since I Lost My Baby5. Don't Look Back6. Get Ready7. Ain't Too Proud to Beg8. (I Know) I'm Losing You9. All I Need10. You're My Everything11. Angel Doll12. I Wish It Would Rain13. Cloud Nine14. I Can't Get Next to You15. Ball of Confusion (That's What the World Is Today)16. Just My Imagination (Running Away With Me)17. Papa Was a Rolling Stone18. Shakey Ground19. Treat Her Like a Lady20. Error of Our Ways21. My Girl [Acappella]I fully recommend ""The Ultimate Collection"" by the Temptations.Anthony Trendlanthonytrendl.com",d +"I have a number of The Residents albums, but this one is my favorite. It's a collection, and they did a very nice job of selection and order of the songs.Note that some of the songs are shortened from the originals, which is fine if you aren't expecting the original. So this is a great introductory album.",m +Not what I expected,T +Yeah Baby!,t +"Carrie Jean Wharton has done a beautiful job composing these stories. The stories are wonderful for all ages. Anyone who has lost someone special and is grieving will find these stories very helpful. The transitions from one track to the next are very well done. Each story leads to another aspect of "letting go" of grief. The grief process is not a simple thing, this CD offers a gentle approach to aid children in working through a lot of the issues that come with grief.",r +"Just a remarkable and moving album in every way.. One of the darkest, bleakest, blackest albums of all times.. The 40+ minute 'Mass Suicide' sermon is breathtaking.. Otherwise, a great Sunday morning brunch record.",L +Sometimes you just want to hear some music that makes you feel young again! I appreciate the music by the original artists and this cd is great!,o +Very nice,a +"Over 30 solid hours of British history, all very listenable and accessible to even the non-historian. An excellent resource for anyone looking for an immersive British history experience. If you're travelling to Britain in the near future, and want to know more about the amazing land you'll be visiting, this is a great way to learn a lot whilst driving, working out, etc.",u +"I LOVE this music. It's not just "Beatles' songs", it's John, George, Paul and Ringo CREATING their songs. Earlier takes of their famous songs where they played around with different backup vocals or instruments. It's like you're in the studio with them. I'm definitely ordering Anthology 1 & 3 of this set. Can't wait to enjoy them!",V +"This was advertised as ""the shadow"" but every single show (293 of them) were all about Sherlock Holmes. I got sent the wrong item.Update**** so 0TR has realized that they sent me the wrong flash drive and they are sending me a new one. I am very pleased with this seller (OTR) 100% to deal with and do recommend !!! They are very professional and I appreciate them.",T +"This is a lovely, relaxing CD.",T +"While I love Akram's music, I was very disappointed that a lot of the songs on this album were repeats from other albums. If I had known, I would not have purchased this particular CD. While it does contain new material, the overwhelming tracks are from his other albums. Disappointed!!!",M +good product but did not receive in original cd case but that's ok I will still order products from Amazon in the future.,P +Wonderful music and great lyrics. Highly recommend.,w +excellent music,W +"So many great classics, not a typical, cheaply thrown together mix of mostly Bach and Mozart. This has many more artists that everyone should enjoy.",y +"Anything by Louise Hay is top notch and I have clients who love these 2 meditations, and find the evening one especially helpful in calming down and getting to sleep.",Y +"The life one is cool, she love Boyzone, just the price a little bit higher than other stores",f +Good,P +"This, IMHO, is Joanne Shenandoah's finest work. The melodies are lovely and uplifting and the story they relate is inspiring.",S +Anyone from that era will love this!,Q +"Budgie rocks. To me this is the most under rated band in history. They are like a mix of Rush and Black Sabbath, but really they have their own sound. Amazing to me that they were never adopted by mainstream US audiences, but you can discover them on Pandora to find out for yourself.",Y +I gave this to my mom for Christmas and she loves it!!,y +Love this music. CD arrived in great shape.,o +"was highly recommended by a guy on trip advisor, gave it to the kids to take on their honeymoon, they didn't wind up going there, so no idea if it is any good or not!",[ +"David Cooks voice is like a rose among thorns. On this acoustic EP he takes several songs from his ""This Loud Morning"" album and does an acoustic version. When David does acoustic his voice is front and center and you get to experience the beauty of it full on. I love this ep, the only thing that would have made it better is if he had included ""Fade Into Me"" and ""Hard to Believe"" on it.If you don't have it by all means buy it. If it sells out here you can get it from that wal store or that tuney place.",E +So worth the price.,X +"You don't have to be from the Hampton Roads area to appreciate Waterworks. But I guarantee, it helps. 1994's release, Soul Divining, Waterworks provides 10 cuts of fairly decent music. Leading off with Sarah, not to be confused by Sara from Starship's [[ASIN:B000YMOA64 Knee Deep In The Hoopla]], tells of the story of how she, Sarah, ripped out his heart and as any country song goes, "stomped that sucker flat".Other songs such as Miles Away, the second cut, is a song of wistful yearning of wishing to be with the one they love to What a Fool, the seventh cut, talks about that fateful night he took away his girls virginity. Soul Divining is a look inwards. It's not pretty in a dreamy, lovey way but beautiful in the way introspective thinking is.This may not be for everybody and likely it's not. It is for me and that's why I have it.",k +"If you knew nothing of Alpert, this might suffice. However, knowing what he is capable of and hearing this, they aren't one and the same. This isn't Herb's best at all!While the sound is definitely the master, the arrangements aren't at their best. Alpert is much better than this. However, his treatment of Sting's Fragile is a fantastic cover. Having heard this from a variety of musicians, this is very, very good.Go find some Herb Alpert and the Tijuana Brass to hear him at the top of his game.",o +Space Oddity is a MUST HAVE in just about any audiophile's library. David Bowie pulls out all the stops in this album. There's not a thing bad about it.,` +Birds of a Feather is exactly that when the listener understands this disc. Everybody who plays on it is acquainted one way or another musically. I got this when I was in my Larry Carlton mode. There are definitely times when a disc like this is exactly what your CD player wants.Birds of a Feather is worth owning just for the mellowness it offers.,m +"Pink Floyd's Dark Side of the Moon gives your stereo a workout. This is what music is all about. This is where compact disc has the serious advantage over vinyl. You get to hear those high notes, those very quiet notes, the reverberation within a note.Dark Side of the Moon is a classic "Must Have" in just about any rocker's library. The '94 "upgrade" to the original '73 release is a step forward. Originally peaking at #1 in Billboard's Top Tapes and LPs, it again topped the chart in 1994 upon this rerelease.Dark Side of the Moon is a monumental accomplishment by one of the leading progressive rock bands of the time.",M +"Elemental is listed as McKennitt's first album originally released in 1985. The copyright date of my CD is 1994.Elemental is, as one reviewer notes, purely Celtic in style. McKennitt had yet to spread her wings and move into other styles of music to meld together as she is most known. Elemental is elemental. The element is the Celtic style. My favorite, Track 3, Yeats' Stolen Child is a song that locks in the mind in both sound of the notes and lyrics as the refrain calls away the child to the faeries.McKennitt's vocals are wonderful. She has the knack of making even the most mundane sound wonderful.",w +"You have to be in the mood to listen to this. It's not bad at all, but it's likely not anything you'll just reach out and get unless you're in the mood for, as they said on Monty Python, "... something totally different".Stephan Mikés is a sitar player who was based out of Miami and frequented the small venues in that area. In 1996 I was down in Miami for school and happened to catch him at a coffee shop called Cool Beans Café. He's interesting, approachable and I thoroughly enjoy the night and the music. In fact, I bought both this and his other disc[[ASIN:B000CA7KWY Good the Bad & The Karmic]] that night.It takes something special to make this kind of music approachable to mainstream America. I do believe Mikés was attempting to get it that way.",J +"I'd have to agree with the other reviewer. Colors carries some weight based on track 6, Helios. Beyond that, it's an interesting disc with an ambient sound that just could work for a quiet night.",_ +"Few outside of Nova Scotia know of the group Signal Hill. These guys, four of them, started out in Halifax playing covers of major hits. They didn't do them identically, fortunately, as that really wouldn't have shown off their talent. These guys are much like Crosby, Stills and Nash. They have VERY tight harmonies. And speaking of CSN, one of their covers on this disc is Suite: Judy Blue Eyes.Musical line up includes:Greg Tobin, Jason Campbell, Jason Meadus and Terry Chaplin.1. Turn Up the Sunshine*,2. Video Killed the Radio Star,3. You Don't Have to Explain*,4. Crackerjack Box,*5. Suite: Judy Blue Eyes,6. All Together*,7. Clarity,*8. Black Water,9. Thank God I'm a Country Boy,10. The Sun Will Shine** - Original music by Signal HillIt was always quite a treat for me to see these guys play when I was in Halifax. The Lower Deck, Halifax, NS, mostly hosted these guys. Live recordings are out there and they convey the freedom and fun of the group. They have a large local following who sing along with most songs. It is a very entertaining thing, indeed.",` +"These songs are done decently enough that it could provide background music for a dinner party or such. If you were in the mood for music while reading, this would also do the trick.These are all remakes, just as it says in the title, so if you do buy it, don't complain that "these aren't the original songs". That's because they AREN'T.",v +"The Manhattan Transfer is probably one of the best vocal groups - ever. Their tight harmonies and honest camaraderie shows well through any performance or audio disc.Tonin' is a fun disc with Manhattan Transfer backing big names such as Frankie Valli with his hit, Let's Hang On. Then there's Smokey Robinson and his phenomenal hit, I Second That Emotion. The best, however, comes with James Taylor. His treatment of Dream Lover is such that to hear it is to wonder why and how that wasn't the way writer and singer Bobby Darren had wanted it to begin with.Of all the Manhattan Transfer discs I own, and they number a lot, this is definitely my favorite.",G +"Shortly before his untimely death in February 1989, Roy Orbison completed this album, the 22nd of his career. Orbison, no stranger to working with others, shares this album with the likes of Elvis Costello, U2, Albert Hammond, longtime collaborator, Bill Dees and his friends from the Traveling Wilburys, Tom Petty and Jeff Lynne.Orbison's widow, Barbara, played executive producer upon her husband's death, ensuring both this album sounded the way he wished and making sure it was released. There are no missteps here. This is an album that was truly a labor of love and done with friends.Mystery Girl is as much as homage to his wife as it was just bits from the line of the song, She's a Mystery to Me.",L +"CTI (Creed Taylor Inc.) Masters of the Guitar, and yes, this is different cover from that other one offered, is a testament to guitarwork. This was issued in 1988 with artist recordings ranging from 1974-1988 through CBS Records. Creed Taylor, the label owner is also the producer of this album.This is okay music. There's nothing here that's going to get the listener too excited. It is decently constructed, produced and executed. There are 11 tracks from 9 different artists. George Benson and Kenny Burrell both share the honor of having two tracks each.If you're looking for something to use in the background for work or a party or even a gathering of sorts, this might well be a viable option.",o +"The Mamas & The Papas were a staple of California radio in the 1960s and 1970s. 16 of Their Greatest Hits represents pretty much anything you might want to hear from Cass Elliot, John Phillips, Denny Doherty, and Michelle Phillips.Folk music was the forte of John Phillips. However, soft rock/pop rock was what the rest of the group were interested in doing. Cass Elliot's voice proved to be one of the selling points of the group. Her appearance also said, "You don't have to be one of the 'pretty people' to be successful".Of all their music, and there is VERY MUCH that has been played for years, Creeque Alley is my favorite. Creeque Alley, besides being a very harmonious song, is the story of the Mamas & the Papas. The lyrics are a historical dialog of the who's who and how's and why's of the group.This is a fantastic album to own. If you have nothing more than this, this is probably just enough to enjoy from the Mamas & the Papas.",D +"Johann Strauss is justifiably known for his waltzes. Kids know the Blue Danube, even if they don't know its name. Hum a few bars and they'll pick it right up.This disc, with a playing time of just over an hour, offers six of Strauss' most famous waltzes. The disc starts with Blue Danube (10:54) and ends with Tales from the Vienna Woods (12:31). The Vienna Opera Orchestra plays with Peter Falk - no, not THAT Peter Falk - conducting both Blue Danube and Tales from the Vienna Woods. Carlo Rottini and Alfred Scholz also conduct.The music is exciting, happy and gay (and no, not THAT gay). Waltzes have a habit of changing one's mood. It's hard to be angry listening to one.",Z +"Unless you absolutely hate jazz, there's no reason you shouldn't give this a listen. This is a great primer to the genre. Duke Ellington, Lionel Hampton, Benny Goodman and Peggy Lee are just some of the names on this 15 cut disc.With the exception of one cut, Sweet Georgia Brown by Benny Goodman, the album was recorded, mastered, mixed and released in full digital sound. CDs at this time were hip to catching all those high and low notes which had been lost in the analog world. This is the perfect way to get an idea of what jazz sounds like without throwing out a lot of cash.",T +"What do you get when you throw five of the greatest musicians of our lifetime into a recording studio? You get "The Traveling Wilburys". As a lark and after recording a B-side song for George Harrison, in May and April 1988, Harrison, Jeff Lynne, Tom Petty, George Harrison and Bob Dylan shared writing and singing responsibilities for what would become The Traveling Wilburys.With Jeff Lynne's deft hand on the controls, ten songs were quickly recorded in the home of Dave Stewart (Eurythmics) and eventually released with Warner Brother Records. You can hear Lynne's influence in many places with several songs seeming to be more at home with ELO than with this collection of artists.One of the things that was best about the Traveling Wilburys was they didn't really take themselves seriously. They weren't there trying to pump out music in order to fulfill some contract. Instead, what you had here were five guys who knew and respected each other having fun.",j +"Traveling Wilburys, Vol 3 is a huge mockery of a lot of things. First, there was really no group formed as the Traveling Wilburys. This was a joke in and of itself. Second, and one of my favorites, I wonder how many people spent a lot of time trying to find Volume II? There IS no Volume II.It is actually rather easy to mix up Vol I and Vol III. The albums seem to have the same feel and some of the songs seem to belong on one album but not necessarily the other and that other might be the one it's actually on! Vol III offers some really good cuts in the form of 7 Deadly Sins, Where Were You Last Night and Wilbury Twist. The latter pokes fun at themselves, and in a small way, honors Roy Orbison.Having lost Roy Orbison in 1988, the Wilbury's changed up their pseudonyms and mixed up life a bit. Dylan, Harrison, Lynne and Petty do their best to carry on. However, I'm sure what started out as a lark became less fun with the death of their bandmate and friend.",s +"I could probably have lived without owning this disc. However, have it I do. I think the reason I did get it was because so much nostalgia was packed into these 21 songs. Anka's young life definitely affected and shaped his music. Who was Anka's babysitter? Diana. What was happening to him? Puppy Love.Anka was a gifted song writer and musician. That worked well for him and, really, the rest of North America. He was "every guy" and, like it or not, those guys knew the words to these songs.Anka's 21 Golden Hits are certainly hits. If you were to own only one Anka compilation, this would be it. I can vouch for the one and only.",_ +"This review is for the original Bona Drag, not the 20th Anniversary Issue nor the Remastered version. Bona Drag gives us a good handful of Morrissey's early works. Excellent songs like Interesting Drug, Last of the Famous International Playboys, Everyday is Like Sunday and Suedehead are all here. If you've never listened to Morrissey, this is about as good a place to start as any.Bona Drag has 14 cuts on the original. I think there was something between 17 and 20 on the Remastered version. Well, that's remastered. This is the way it was.",B +"Morrissey's Kill Uncle was Morrissey's second solo album. This was not the music to which Morrissey was best known, but unlike the basher review, I've no malice towards either Steven Patrick Morrissey or this album. (By the way, if you don't like him, why the hell'd you buy the album?)Morrissey, who I often refer to as the King of Gloom and Doom, is hardly such in this album. That's partially why this album is so different than most. Most people know Morrissey as the guy that could sing about death in a poppy manner.Kill Uncle is not bad. But it's definitely not Morrissey's best work.",\ +"Many years ago I was learning to copy Morse Code courtesy of the U.S. Navy. I had known of Kraftwerk because of their first U.S. album release, Autobahn. Radio Activity had Morse Code scattered throughout. It was a "must have" for me because of it.However, it wasn't the only thing that kept me harnessed to the stereo. Kraftwerk SOUNDED good! They were a tight group of folks playing electronic music long before it was popular. In fact, many give a nod to Krafwerk as pioneers of Electronika and, subsequently, techno-dance music.Perhaps my favorite album by Kraftwerk, and I HAVE been listening to them since the 1970s, Radio Activity always sounds fresh to me, regardless when I play it or whether it be vinyl or compact disc.If you like dance/club music, you'll like Kraftwerk. If you like Electronika, you'll like Kraftwerk. If you like Techo-pop, you'll like Kraftwerk.",j +"I used to live in Virginia Beach, VA. I was the sous chef of a restaurant called Zia Marie when it first opened up in the 90's. Among the staff was a guy named Mike Pio. Mike was a member of OFS. It was him who got me listening to OFS.Honestly, I can't remember much about the disc other than, well, I KNEW somebody on it! Sadly, about 5 years later, the disc was in my Fiero at my apartment in Watertown, NY when it was broken into and a handful of CDs and my chef knife kit were stolen.Rock on, Pio, my sauté bro.",w +"In 1993 remember fondly waiting for this album to come out. The radio station I listened to, 93.7, The Coast, WKOC (Norfolk, VA) told me of its release. This station wasn't known for following the herd or leading ""Sheeple,"" it was ahead of the curve.My first listen to Truganini sold me on the album. It told me the Oils weren't yet done with their work. It told me offenses still waited to be set right. It told me they, at least, would try to raise the conscienceness of a self engrossed people who generations earlier thought nothing of wholesale slaughter of others in the name of European white men and differences in the color of one's skin.I don't know how I knew at the time it also signaled the end of Midnight Oil outside of Australia with only the album Breathe to follow in 1996.Sadly, the Oils never received the worldwide success they so honestly deserved. Hopefully during the years after their disbanding people occasionally think of the good they did through their music. Yes, it was political. Yes, it made ME think. And yes, I miss them.",g +"Love this Artist the MOST, Love both CD's and the Music.I have them both in my car and listen to them over and over.They keep me calm while driving, and that is what I like the most~",u +"I haven't listened to this record as I bought one that hadn't been opened & now I feel badly about opening something that's been undisturbed since 1981. I know it's good as this is pat mAcdonald & everything he does is good, though most of his work is fantastic. I do open some stuff.",E +"Why is this CD so good? Well one of the greatest soundtrack artist, Craig Safan, composed it. Safan is especially good at adventures like John Williams. Safan made this great movie even better. I am a big fan of the 1980s movies. They were full of hope, light adventure, and a touch of comedy. I enjoy current movies too, but I must admit they are mostly darker themes, with an apocalypse, zombies, foul language, and gore. So if you haven't seen this movie be sure to set aside some time for a rousing adventure in space. This reprint is excellent. Excellent quality CD, great price, fast delivery, well packaged, and reliable seller. I highly recommend getting this CD now. When this goes out of print it will double or triple in price. Being a fan of Remo Williams I had to pay more than double this price to get the reprint Remo Willams: The Adventure Begins CD because I waited to long. [[ASIN:B000EZ91EC Remo Williams: The Adventure Begins]]",C +"This not the original soundtrack songs on this CD, just the background music, by Alan Silvestri, that is good, just not the songs by artists. The original songs for Fanare: We're Going Somewhere [0:32] 2.Elton John - Saturday Night's Alright For Fighting [4:54] 3.We've got to dig up Dom [0:24] 4.Cream - Badge [2:47] 5.You called off your wedding [0:40] 6.Carole King - It's Too Late [3:54] 7.Pat Metheny And Lyle Mays - September Fifteenth [7:39] That car's afflicted [1:12] 9.Classics IV - Spooky [2:52] 10.There's nothing wrong [0:44] 11.Pat Metheny - Farmer's Trust [6:24] 12.Here's to us, by God [0:43] 13.Steppenwolf - Born To Be Wild [3:31]14.Pat Metheny And Lyle Mays - It's For You [8:14]15.Goodbye friend [1:12] 16.Blind Faith - Can't Find My Way Home [3:15]. One source I know of is at [...]",v +"Excellent and came with little pics of the artist, my daughter loved it! Thank you!",q +"Classic George Harrison. He was a gifted musician who clearly found his place in life. This music is evidence of this. Personally, I'll always treasure this album. I've never once had any regrets and the fact that it's on CD makes it convenient when traveling in my personal vehicles. Appreciated the fast shipping and overall protection of the item during transit.",X +"You know the old saying, "Loved her, hated him?" Well, I don't actually hate him, but in my opinion Mark Lindsay's voice has not withstood the test of time. Susan Cowsill has been a valued session musician for about 40 years and does a great job of carrying this CD. She is really good and gets better every time she records. Nostalgia is nice and sometimes people hear what they want to hear, but as a fan of Mark's for over 50 years I am very disappointed with his ho-hum performance on this CD. Again, this is just one musician's opinion.",A +"Again, I needed this CD set for my Wagner classes this summer.Excellent collection.",H +similar,l +"This collection has all the Monkee songs I remember. A nice CD to have, since only one member of the group survives todsy.",_ +This is Gregory Porter's signature style in full bodied sound. His meaningful lyrics are punctuated with jazz riffs and beautifully syncopated tunes that make this a truly memorable sound experience. I saw Porter at a one man show several years ago and he sang many of the songs on this CD. This CD will make you a Gregory Porter fan if you are not one already!! Really good one!,G +A very good collection of Peruvian traditional music.,A +"Good but ""16 biggest hits"" is more bang for your buck.",J +"Good mix . Since I’m new to their music, I can’t say how many are songs from their debut era. Does include some of their more well know songs like ‘save me’, ‘run’ and ‘fire’.",c +Have to hear it to believe it. So much talent.,X +William Kealoha record album is great!,i +EXCELLENT !!!,E +Great product... smooth transaction... A+++++++,^ +Great collection of MJ's songs. Love it!,v +RIP JACKA! This Album was the DOPEST,k +This is a great yoga dvd. Rodney Yee is wonderful.,h +Gotta' love the walking song,[ +Love the album and I would of loved the delivery except it was really long over 3 weeks.,u +He is my new favorite worship singer,v +Good Music,` +This was in great condition and quickly shipped. AAAA++++,m +"I was so happy to find the soundtrack to one of my favorite movies. Mancini was a genius, and I love listening to this record.",t +"This is a wonderful CD! I first heard these tracks a few days ago on YouTube, and decided that I wanted my own copy. Beautifully sung and orchestrated! These favorite hymns will get you in the mood for worship and bible study. Helps you keep your mind on God, and uplifts your spirit, faith and trust in HIM only! And I found the cd right here on Amazon! :)))",K +"Guided by God's Loving Holy Spirit through our Dear Lord and Savior Jesus Christ, Pastor Doug Batchelor teaches on bringing your family, neighbors, friends, questions, doubts, complexities, resources (all that we have and all that we are), occupation/job, your sick bodies for healing and your bodies as a living sacrafice, your sins, your hearts, your burdens to Jesus at the foot of the Cross. Whatever your burdens are bring them to Jesus cause HE can bless and change your circumestances. Amen! This is truly a blessed and powerful message. Amen",B +Excellent soundtrack for excellent series.,f +The music is great but it would be nice to be able to interpret the playlist showing on the dash console.,j +Not my favorite...,z +Not what I thought.,c +"The previous reviewer is no doubt much more qualified than I to evaluate the classical discipline of ""Etudes,"" and its value in the ballet studio (both apparently highly regarded).What struck me about the music I heard, were the feelings of modesty, warmth, and kindness conveyed by Ms. Gortiz's performances, particularly on those pieces where she appears to have created her own arrangements. Its as if her personality is sneaking out from behind years of classical training to wink at us, as she gently glides through technically challenging piano music.",i +Metallica is tge greatest fkn metal band on the planet.,` +"I remember when I worked at The Wherehouse Records in the early 80's, my manager always used to talk about Love Forever Changes and how it's this hidden classic. I never believed him and was always too busy to sit down and listen to it. Plus, I was into punk and New Wave and that psychadelic stuff was passe at the time. Well here I am, about to turn 40 and I'm positively transfixed by this new Remaster. I am listening every night when I go to bed with my portable CD player and still moved every time I hear it.The thing that blows me away about Forever Changes is the way it pulls you in. Upon first listen, you assume this is another late 60's peace and love album like Iron Butterfly or something. But as it grows on you, it begins to transcend it's own time. It's almost as if it could have been made in any era since and it would still have the same impact. This is one for the ages and I forsee myself playing it as much as my two other favorite late 60's classics Tommy and Velvet Underground and Nico. Thanks for this magnificent reissue.",j +First Christmas music I heard as a kid. Still the first one out of the box of Christmas stuff every year.,g +Winograd is the old school and these overture performances are fantastic...they got me interested in classical music years ago whenI was in college.,D +Bizets early opera is in a different league than Carmen but has some incredibly beautiful music. Tenor Leopold Simoneau is sublime.,h +"I must say that Toni Morrison is indeed one of my top three favorite authors. When I first saw this movie at the time of its release, I must admit, I didn't really enjoy it. I thought it was hard to follow in that format and the recollections disjointed. The novel, however, especially Ms Morrison's reading of it is totally the opposite. I am thoroughly enjoying it, soaking up the experiences presented, and recommending to anyone who will listen (no pun intended). I will watch the movie again when I'm finished the the audio book, and I've already pre-ordered her upcoming novel. She is an inspirational author!",c +"I own several BigBang albums but this is probably my favorite. It’s non stop fun, fun, fun.Can’t wait for them to be reunited after mandatory military service so I can go to concerts.",F +"I am very pleased with the purchase of Sung Si Kyung's Winter Wonderland CD. The production team, designers, and musicians really put their best efforts and talents in this CD and it shows. I have no doubt, this will become a classic choice for music lovers anywhere in the world. Once again Sung Si Kyung vocals warm the heart and now finally he sings all songs in English. One of my favorite songs in this production is his duet with Ailee: "Baby it's cold outside." Thanks to all artists involved in this project and of course to Jellyfish.",Z +Great packaging that includes a photo album. Totally love the album and I am completely satisfied with this product!,l +"This is a great record with 2 great songs on it, and it was shipped very quickly, two thumbs up!",s +My husband loves the Jackson family. He enjoys all of the songs. It came quickly and all in one piece!,c +Great product at a great price. Buy it and you’ll be glad you did! Makes a great gift too!,Y +Great value. Fast delivery,P +Very relaxing,H +Very relaxing,R +"For years God has used Rhonda’s songs to inspire millions of people. She has used her beautifully gifted voice to praise and worship God, and so many are grateful she travelled the Christian road to the music Hall-of-Fame in our hearts. Honoring Christ with your gifts and talents pleases Him greatly - as He taught us in The Parable of the Talents - Matthew 25 KJV. As a Christian, I testify to God’s amazing forgiveness and grace. All of this is so beautifully expressed through Rhonda’s songs. All her CD’s are a treasure! You can bless others sharing her songs and CD's. May God richly bless our sister in Christ, Rhonda Gunn. The song Almighty God is awesome, friends!",s +"Better than I expected. The songs are beautiful, moving, and inspirational",W +It replaced a vinyl lp.,n +Liked!,g +"This is the heart of the Charlie Parker canon. These are the recordings that made him famous, among them his best recordings like Koko and Warming up a riff. Listening to these on vinyl gives you the experience of all great post bop musicians learning about Bird and bebop.",J +"if you're already a mexicats fan, this is a no brainer. if it so happens, this was suggested to you and you were curious, check them out on youtube. personally, i prefer their more high tempo spanish songs, but you'll get a lot songs in english and in a variety of styles. also, jenny's ability to switch between languages, sometimes even in the same song, is incredible, and her brass playing is very impressive.",G +Ambient Dub masterpiece. Chilled out bass grooves that are timeless. Seller shipped quickly.,f +"I thought this was their DVD, not the VHS version. I am still stoked to get this, but it is hard to watch as not many people use VHS anymore.",L +"I have just recently found Jim Smoak & the Louisiana Honeydrippers. This is a fantastic band which plays old style country music in a ~40s-50s style partly emulating Bill Monroe and his Bluegrass Boys. (Not surprising since Smoak had toured with Monroe for so long.) Unlike the Monroe versions there is a strong fiddle during many of the songs and the mandolin is more subdued.The vocals are more reminiscent of country bands like the Carters than Monroe's bands. In most places the style is much closer to old time country than Monroe's style, Louisiana, Appalachian or Mississippi sounds. The lyrics are more upbeat than the sort of (whining ?) sound you see with some of Monroe's work and much other country music, nothing like the characteristic Stanley sound nor other 'bluesy' types of country music.What I especially like about the album is that there are a number of old songs you seldom find nowadays. These are songs like Liza Jane, Old Dan Tucker, Bill Cheatum, The Fisher's Hornpipe, Mama Don't Allow.",t +"This album is smooth but not at all low energy. The full moon ensemble takes old traditional gaelic music and makes it into modern love songs. Sweet songs are interspersed with light and lively fiddle tunes which would be equally at home in Dublin, the Isle of Skye or Galax Virginia. Track seven, Take Her In Your Arms will take your breath away! And it's just my current favorite,subject to change when track eight comes up.In some songs the fiddle is dominant, in others the vocals stand out, in most there is a nice interweave. Beautiful guitar supports and does not detract; and - a difficult conception in this age of overmixed drums - in none of them does the rhythm section lose control. From time to time, is that a real Irish whistle? Bravo! Five stars!This is traditional music with modern instrumentation and with excellent recording and engineering. Mastering was at Swampers Sound Studio so it is not Lomax sitting in some dark cabin!",S +love it,a +Quick to ship and great item. Thank You!!,R +Great CD with all of his big hits. Well worth the price I paid.,X +Beautiful and relaxing music. Told all my friends.,U +The music is like Louis Armstrong era--I love the base fiddle.,P +"As a longtime admirer of Maestro Corigliano, I purchased this recording quite a few years ago. If you are familiar with his film scores and previous orchestral works, you will not be disappointed with his Symphony #2. It is a rich, dark atmospheric work, written for an all-string orchestra. As always with Corigliano, it ranges from lyrical to discordant. Not an ""easy"" work, but well worthwhile.",Z +"Everything about it, from the sound quality, the Berliners excellence and Levine's sometimes unusual tempi all work beautifully together. An exemplary recording.",k +on time and as described,[ +Enjoy all of his music!!!!!!!,[ +Bocelli as I like him with a great selection of tunes. The man improves with time!,e +;-)),y +"I've had this CD for years and worried about it getting scratched or otherwise unplayable. So I searched high and low to have one in reserve. Finally found it! This is the most soothing, gentle CD I've ever had. It puts me in a very relaxed frame of mind.",q +Such uplifting music.,N +Nice,f +This was a nice CD but I have others that I enjoy more but it is still a nice addition to any collectionand I would still recommend it.,C +I originally bought this CD for two songs but ended up liking the whole CD!,Y +"A fan recommended this group to me' promising original scales. I like Norteno music with traditional accordion waltz backing. These guys have very fine harmony and the tenor lead is among the finest in the genre. Why don't I rate this work higher? Simply because I don't understand Spanish. These are songs that follow a set pattern (for the most part, but with wonderful exceptions). The big deal is to understand the stories and the allusions. Absent that, most of the music is of a sameness. Anglophones probably won't 'get it.'",M +a Great new CD from Na Leo - very cheery and upbeat tunes to drive by! Mahalo!,c +"This CD is awesome, the whole thing through and through!!!",x +Beautiful songs and his voice comes through really well.,D +Great Cd.,p +Nice product,b +This is a great cd.. Paul McCartney was at his best vocally during this period. Love it!!!,p +Many classic hits with NO Neil Young.,U +Live stones . . what else can I say,Q +I really like all of the pictures. They are so cute !,d +This is the most moving and beautiful guitar music I have ever heard. It touches the heart.,N +A great collection of beautiful gospel songs by a talented artist.,t +great book great company,D +Excellent highly emotionally charged talk by Joyce well worth getting.,R +"Typical, great Horner music",C +this album goes through the motions and emotions of a relationship.you meet connect. think it's going to last but nope...break up. ask for patience.variety of upbeat songs like black and white and no judgment to songs that make you cry like put a little love on me.and who hasn't seen thier ex in everyone's faces like in everywhere?WE LOVE NIALL can't wait to see you in person ❤️ 💗 💓,i +Love the group,A +Love the Music,R +I wound up getting more. love the music.,P +Loved the music. Brought back good child hood memories.,W +There's nothing like a Marianne Faithfull CD. I'm happy to have this rare one.,C +Marianne's song is great. The rest is just fair.,R +Fantastic music! I listen all year round.,A +ABSOLUTELY FABULOUS!,j +Love all things Marianne. Nice addition to my collection. Mint condition.,t +"I was fortunate enough to have a neighbor who owned the complete set. I was broke and recovering from bad accident, depressed, and having really tough time. My neighbor said ""Just do it. Won't cost anything but time."" So I did just as it instructed. Wasn't too sure if anything was happening. One day driving long way to specialist, all of a sudden my vision became a complete blur. Scared I was having a stroke, and out of desperation, jerked off glasses. And to my surprise, perfect vision. I had wore strong prescription glasses since age 6. That was all the proof I needed that this program was working and fixing stuff. No more glasses for reading or distance at age 56.",Q +"Despite the big label elbowing of Philips and Gardiner, this is the true first recording of the quite magnificent Messe solennelle. It is a live capture from 1993 at the Basilica or Vézélay, performed by no less than the Cracow Philharmonic and Choir under Jean-Paul Penin and splendid soloists, none particularly well-known, including the mezzo Christa Pfeiler who sings her difficult part admirably. Kudos to the conductor who has complete grasp of the innovative power of this score. The first publicly performed work of the composer (who penned it at the age of 20) saw the “création” in 1824 at the élite church of St Roch (Paris). This means it was contemporaneous with Beethoven’s Missa Solemnis and actually preceded the last masses of Schubert. The Missa Solemnis is of course a work sui generis, but the sheer originality of concept, sound world and variety of expression of Berlioz’s work far outshine the conventional Haydn-flavored works of Schubert.The sound capture of this live session is admirable in all respects, without the stepping into the world of aural calibration of the Philips. Above all, this remarkable issue captures the live frisson of the live event — irreplaceable in my imagination.",W +"Even among the tens of superlative (and increasingly rare) discs Paul van Nevel and the Huelgas have made of early music, this Dufay disc is a standout. Isorhythmic motets are compositions which free melodic lines are overlain on repeated rhythmic patterns (usually in the tenor) -- this increasing freedom points directly to the sixteenth century. Dufay's works carry this concept to their height. As van Nevel eloquently states in his liner notes, these works are a ""symbolic farewell"" to the medieval musical world (ars nova).This irreplaceable disc has complete text and translations. Even more striking, the cover photograph an exquisite image looking upwards into the cupola of Santa Maria Novella in Pesaro. You may humbled to realize that art and music have not made any essential progress after five centuries.",e +"Premières valses (1899) are charming, turn of the century morceaux; the selection from le Rossignol éperdu (completed 1910) shows an ambitious advance in harmony and technique and bears rehearing.Beautifully played by Mme Joly and captured in perfect sound by Accord. Another production of Georges Kisselhoff.",I +"Another excellent recording from the Motette label, here capturing the exquisite Steinfeld (Rheinland) Basilica organ of 35 stops and 1,956 pipes. Many of the tiniest pipes (<2') date from around 1600, while the largest pedal pipe (16') is a restoration of 1934. Craig Cramer of Notre Dame plays an appropriate and unhackneyed Bach program (Prelude and Fugues BWV 531 and 542, the Partita Variations BWV 768, and the three 'Allein Gott in der Hoh sei Ehr' chorale preludes which exemplify his unlimited skills and the greatness of this music.",e +"This is the second volume on CD of recordings made by Spanish Columbia (released later on DeccaLP) and even more difficult to find than the first. But the rewards are immense as this was made at the zenith of the Caballe's power (1974) before her enormous expansion of roles in the lyrico-spinto repertoire at the greatest houses on three continents. As such, the purity and flexibility of the 60s is evident, but also the incipient power and dramatic qualities of her later career.Noteworthy are arias in three roles in she did not appear in on stage: Gilda, Suor Angelica, and Amina (Sonnambula) --- and they are sung to perfection with ideal accompaniment by Gianfranco Masini (who conducted some superb live recordings of Caballe in this period, all imperative listening). Arias from concurrent stage roles include the Trovatore Leonora, Amelia, and the two striking arias of Vepres Siciliennes.Although there is considerable overlap with other aria albums of this period (EMI, RCA etc), the Suor Angelica lament is truly special.",W +"A powerful realization of this masterly work of Puccini — is there any doubt that Trittico is far more difficult to cast and produce in the theater than anything work in the standard Italian repertoire? And its rewards are commensurately grand.This rendition features excellent singers in all the three leading roles — Tokody is superior in sound and manner to the late-captured Price whose voice (if were a barge) would sink. Unfortunately, she failed to fine the voice down for her Angelica in the sister recording. But Casellato-Lamberti is the equal of Domingo and Nimsgern is excellent. Even the Frugola is cast with a real voice instead of a crone’s. Prêtre is completely convincing and the sound is spacious and well-captured. The four language libretto in full sends this to the top of my flagpole of the versions I’ve encountered, including the famous Santini.",Y +"Wagnerians look back to the 50s and 60s —Verdians acclaim the 70s justifiably. Evidence lies in this live capture from 12/18/77 led by the redoubtable Muti, and studded with stars such as Cossotto, Manuguerra, Carlo Cossuta, and in very good broadcast sound from the 2000 seat Teatro Communale. At the head of the ledger is a soprano who is hardly remembered despite starring at the Met for some 158 performances, where she was much appreciated — simply because she was not invited to make any commercial (studio) recordings.What we hear is a voice of lustrous beauty throughout its range (which could hardly be said of many of her rivals such as Price and Scotto); exquisitely fine-grained despite it ability to expand to spinto power, and a technique of flexibility and accuracy (she tried to sing what “è scritto”, and sometime Muti is unforgiving as he’s used to Leonoras who smudge their fiorature). One hopes that when a Mexican soprano anywhere near this calibre appears, the studios will have changed their attitude.So the three key reasons for acquiring this recording (also available on at least one other release) are Cruz-Romo, Cossotto (in typically wonderful form), and Muti (who does not allow any interpolated high notes by the tenor). Carlo Cossuta is quite fine as Manrico (certainly better recalled for this rôle than Otello). Under no circumstances can Muti’s later release (in modern studio sound), with its superannuated principals be considered at all competitive to this rendition.It’s ironic that the Trovatore recordings that give me the most respect for the opera are both live recorded with superb Leonoras: Cruz-Romo and Caballé (a different characterization). I would not dispute another Amazon reviewer of this recording issued on a different label who has called this the finest Trovatore on disc.",i +"This 1986 recording, dedicated to the later organ works of this great composer better known for his large scale works as the Weinen Klagen variations — without which it is difficult to imagine Franck’s creations — is wholly admirable is all respects. The contents include a selection from the inestimable Via Crucis and the Missa Pro Organo (from the same year, 1879) and are played to absolute perfection by the young Bousseau on the St Pithiviers organ outside Paris. This sweet sounding instrument from 1786 was refurbished by Cavaillé-Coll a century later, and by Robert Boisseau in 1962. A marvelous disc with the highest recommendation.",J +"This exquisite disc featuring six Provencal organs features several of the eminent organists of the 6-disc box referenced above, and in slightly duplicative repertoire. At the nominal price and the wingspan of the box (from Switzerland to the Balearic Islands to Spain), I would advise acquiring that set first and supplementing with disc.Or, alternatively, if the set is too expansive, then this disc can serve a standalone window on French Mediterranean organs. Besides the two French organs played by Michel Chapuis (St-Maximin la Sainte Baume) and Rene Saorgin (Malaucene) in the set, this single disc includes four other organs in pieces by Couperin, Roberday and Frescobaldi by, in addition to the organists just named, Francois Chapelet and Lucienne Antonini. A disc to delight those who have traveled to Provence, or those who wish to be transported to the living past.",J +"These two works are early Debussy rarities: l'Enfant prodigue (1884) won the composer the Prix de Rome and la Demoiselle elue (1884) fulfilled a commission related thereto. This disc is also unusual in that the soloists gathered in the former work likely never appeared together in any other recording: Jessye Norman, Jose Carreras and Fischer-Dieskau. L'Enfant contains an oft excerpted work (Air de Lia) but otherwise the 35 minute work is obscure, as is the 20 minute Demoiselle. All soloists (including Cotrubas in Demoiselle) are in fine vocal estate; Carreras shows little evidence of the stylistic proposition he faces, tending to scoop and sob in Italianate mode, but his voice is in its prime. The studio sound is excellent, and full bilingual texts are included (in microfont); unfortunately, the disc is banded in single tracks for each work. An essential recording for fans of significant French fin-de-siecle music interpreted by four luminous singers of the era.",q +"As an owner of many vintage recordings, the sound on this set is marginal. With so little music actually coming through, this interesting (but not altogether convincing) early middle period Verdi is heavily cloaked and perhaps only some of the arias come through with dramatic effect. Extensive cuts to the Paris ballet music. For those unfamiliar with the work, It's recommended to choose a modern recording (Arroyo/Levine, Studer/Muti) to grasp the basic musical detail of Vespri.",G +"This is a live concert performance in London in 1973, duplicated on the Opera d’Oro release (for which there is a fine, in depth review which is worthwhile). I would add that that the CD format is ideal for listening to this work is scarcely warrants a production in our time. The two CD recording omits perhaps 25% of the score, including ballet music (no regrets) but a soprano duet might have had some interest. As a whole, this music is a characteristically melodramatic and bombastic and mostly forgettable (not to speak of the repulsive plot).Tucker is a few years from his untimely death and the sound is solid but typically rather nasal and elderly sounding (suiting his part naturally). The other tenor Leopold (a high flying part) is sung convincingly by Juan Sabate. Perhaps the best reason for a listen is to Hayashi’s Rachel (la juive) — who sings very well with a clean technique and clear timbre. Very little of her career is documented. Michelle Le Bris sings a powerful Eudoxie.",f +"There is almost a surfeit of excellent contemporary renditions of the partitas on the piano (the only way I enjoy hearing them) but this is a distinctive classic. For the connoisseur or pianist, these interpretations of the largely forgotten Carl Seeman (1910-1983) are treasures. The purity, precision, fine rhythmic sense, and sheer fleet, pianistic brilliance on display here provides a window into a different performance tradition. The fingers do it all — no octave emphasis/doublings in the base, minimal pedal, true legato, no détaché — without agogic or rubato effects. This is less “romantic” playing than dominates today’s recording studio, but the uncovering of the composer’s creative world is exceptional and never tires. The recording is close but fine in every way.",c +"This recording encompasses four settings of the five movement 'propre de messe' for the Advent/Christmas period as sung at, and by the monks of the celebrated Monastery of Montserrat (outside Barcelona) with its famed 'Black Madonna', and for whom the late, great soprano Montserrat Caballe was named. A remembrance for those fortunate to have been to this prepossessing site (alas, not me), and otherwise a contribution to the lineage of a precious heritage.Published by the label Jade in 1990, the disc timing is about 52 minutes, and contains fine textual notes in French and Catalan.",I +"This is a treasurable recording (2002) of the complete Durufle organ solo works as played by the superb organist Erik Wm Suter of Washington Cathedral. The audio capture of this 10,650 pipe Skinner instrument is exemplary, as are the notes by Mr Suter. Kudos to all involved in this memorable “On a Sunday Afternoon”.",w +"This remarkable 1972 recording is outstanding on many levels (including the excellent sound recorded in vivo) but above all for the superlative principals. Caballé was in her early prime and connects technically and dramatically with the heart of the role of the queen — even later, Verdi never surpassed the anger, despair and nervous energy of her final aria and finale. While we are fortunate to have Caballé live in 1977 from Aix (with Carreras), her voice here has a unique, youthful beauty (indeed, it is the youngest sounding Elisabeth on record).More surprising to me is the superlative Gianni Raimondi — ten years senior to Caballé) but scorchingly dramatic and perfectly executed with a voice of true, unpressured beauty. A pity the studios ignored him for, to me, he had a finer combination of plangent Mediterranean timbre and power than any of the Big Three who followed him (Pavarotti came closest). Raimondi also sang very difficult roles, including the high-flying heros of Puritani and Guillaume Tell.The mezzo Sara, sung by the (unknown to me) Laura Londi, is very fine — what hidden talent the Italian opera houses of the day cultivated. Bartoletti is somewhat less dramatic than Rudel, but the balance inherent in dramatic lyricism is well guided.",i +"Scherchen is truly incandescent in this astonishing score, and captured in amazing fidelity. I had never particularly liked this music, but listening to Scherchen’s reading has altered my perception and I’m in search of his other Berlioz readings.",x +"This is a review of the pictured recording, made in Bayreuth in situ in 1962. All previous reviews are not for this recording.This Lohengrin carries the special electricity of live performance but also reflects a time where the spiritual qualities of the text and music were respected and carried no 'commercial' prospects.At the heart of Lohengrin is the tragedy of Lohengrin and Elsa. If the protagonists fail to convince, the drama fails. In the youthful Jess Thomas and the 22-year old Anja Silja, the recorded history of this opera reaches its pinnacle. In pure and translucent tones, these singers convey the vision of this mystical opera. The sound of Elisabeth Grummer in the Kempe studio EMI release, widely praised, has nothing like the youthful suppleness and tonal refulgence of Silja -- a matronly sound is far from ideal. Many other praised collaborations (e.g. Domingo/Norman) succeed in musical, but not dramatic, terms. Sawallisch is very fine, the Bayreuth acoustic is well-captured, and a complete [old-style] booklet with text and translation is luxuriously supplied. This is my favorite of my six Lohengrin versions -- and I'm chagrined to say that I bought it last for no clear reason. Although the catalogue is overflowing with fine and very fine Lohengrins -- all worth hearing for some interesting singer, interpretation, conductor -- in a world of unlimited musical operatic masterpieces, and limited time, I'd advise to start here.",l +"A very fine disc in every way that showcases the superb skills of Soviet musicians including excellent but lesser known violin and saxophone soloists; Alexeev is better known through his non-Melodiya recordings. Fine sound, additionally, render this an admirable collection of Glaxunov's concerted work.",i +"The proverbial 'gem', an engaging -- even masterly -- work of German 'verismo'. Though I cannot compare this reading against another recording, and despite the absence of libretto, the score's striking thematic and melodic materials paint an evident dramatic situation that drew me immediately, from the very striking opening bars to the final chords.The first tier soloists (no less than Kollo, Marton, Moll, Weikl) clearly passionately believe in this work and absolute credit is due to Janowski for his leadership of this fine band and cast. Although there is no libretto, there a very fine trilingual notes on the work and the soloists. Recommended strongly for aficionados of operatic history and true music lovers.",d +"This rendition has every virtue and no faults whatsoever, and is unexcelled in modern recordings by baritones, enhanced by the superb plasticity of Ax's pianism.With so many competitors, of whom Goerne is perhaps the most formidable, why so? First, Hagegard's timbre here at its prime is uniquely beautiful, more youthful and with tenor-ish overtones which enhance the reading for me as it binds this last collection to the poignant heroes of Mueller in and Winterreise. Second, his voice, which otherwise resembles the early DFD in luster, has absolute evenness in the lower reaches and in alt -- and expands thrillingly and fearlessly without the least loss in quality. Third, the unaffectedness of Hagegard's reading strike to the central core of this wild, even passionate collection of songs. At this supreme level of musicality, there is no need for the agogic emphases of Terfel, Bostridge or the later DFD. Goerne's very fine reading is darker, and Trekel has not recorded this [yet], this remains surpassed.",r +"Repertoire, performance, engineering and liner notes -- beyond first class. Refer the comprehensive review by g c for details. The essential open air disc for warm weather sally in the countryside with the top or windows down.",z +"This is one of the finest mementos of the late (1994) Richter, playing a very long program of works that were composed within a window of 20 years or so in the late 19th century. As such, it was clearly a program that was close to the heart of this iconic pianist, and we can participate in this live performance in very fine sound (Lugano). As Piero Rattalino points out in his moving program notes, this final stage of Richter played out in minor venues, almost at the spur of the moment, manifesting a withdrawal from public life. There is a concomitant austerity in the performances which have a monumental quality (Rattalino hits the nail on the head when he says they are 'hieratic'). Although these compositions were performed and recorded elsewhere in Richter's long lifespan, this disc is indeed privileged.The Schwetzinger recital recorded by Hanssler disc is the same recital. The Aura has the advantage of Rattalino’s wonderful notes.",a +"Assuming that performances of lieder recitals in general and Wolf specifically are steadily diminishing (at least outside German-speaking countries), this recording could very well represent the golden age of Wolf performances. Very sensitively performed by Seefried and Werba, this misses the fifth star by virtue of my reaction to Seefried's voice -- fine when not under pressure in terms of volume or above the staff -- but colorless and thin at many climatic moments. I suppose this is just the inherent drawback of the 'white' Germanic voice type.",\ +"I bought this because I have all Angel's other CD'S and have worn them out. I attended one of her concerts in Delanco NJ & was elated with praise. Got the concert CD. Now with this CD I am hearing new & some old, songs.What a voice & blessing Angel has been to my life. I first heard her at my church, Christ's Bible Church, where she still sings.If you read this Angel, thanks for all you do & the blessings I still get from your singing. I would love to go to another of your concerts so if you could let me know & I'm familiar with the area, I will try to attend.",e +Beautiful Album-True Spirit of the SW. Rick Brosseau has the voice of an Angel and has recorded several other CD's since this album. You cannot go wrong with this one! (My Husband!) Enjoy!,h +"I was a little hesitant to order the cd because so many of the hymns I was not familiar with. I was most pleasantly surprised after I listened to the cd a couple of times, because almost all of them now are my favorites! My husband and I enjoy this cd very much and play it often at night before bedtime, as we find it not only spirit-uplifting, but relaxing as well.",V +"I purchased this set when it was released in serial format, and this CD was the first one. We loved the music and also the commentary that accompanies the CD. Unfortunately, it was used so much at our house that it eventually disappeared, making it necessary to replace it. I was so glad to find it on Amazon for a reasonable price.I have always liked the sound of classical music but it was a sort of foreign language that I didn't always understand - this set gives some background on each track so that the enjoyment of the music is enhanced.Volume 1 is light, pleasant and a perfect place to begin. The music is enjoyable and the case is colorful and informative.My only criticism is that the CD isn't securely affixed into the booklet/cover and it was constantly falling out when the album was handled. This led to the CD being damaged and eventually being lost completely - that is why I had to buy another one. I have compensated for this situation by downloading the music CDs to my computer and then filing the CDs into a protective album, leaving the booklet/covers accessible when someone wants to refer to them. Now, when we want to listen, we simply plug our computer into our stereo system and read the accompanying booklet.",E +"I love this book, and this program helped me recover from serious disease. You should be careful when buying it though, because third party vendors tend to sell this for TOO MUCH MONEY.ABOUT THE PRICE: This same product is offered on Doug Kaufman's website for $69.95 + shipping. (I think I paid only $10.00 shipping, making the total $79.95.) The paperback print version is exactly the same price as the audio trilogy (and it is worthwhile to own both). I have seen it sold by third parties for up to $150.00 + shipping, and I think that is outrageous! If you shop directly from Doug Kaufman's website, you might save almost 1/2 the cost. If you buy it from Amazon, or another third party vendor, it is possible that you could get a slight discount OR you could pay a mind bogling premium - depending on the seller. Just be aware of the true retail value of this product, and you can make an educated decision about the value of the item being offered.ABOUT THE PROGRAM RESULTS: Having said that, it is a great book and still worth the cost of purchase even if you paid top dollar. After 4 months of using this program, my type 2 diabetes disappeared, and I had lost 40 lbs. The weight loss was shocking because I hadn't weighed in several months - wasn't really trying to lose weight and was only trying to normalize my blood sugar. I began to notice that my clothes were loose, and when I went to buy new ones, I had gone down a size. That is when I decided to weigh myself and see what was up. Very pleasant surprise! Plus, I was free of a myriad of other symptoms.His theories are scoffed at by the mainstream medical community, so if your are a medical conservative, you might want to pass this one by. However, I highly recommend them for anyone who has tried the conventional route and failed. If you are ready for a change, and not afraid of some detailed medical information (with the sources to back them up), then these are good books to read or listen to in their entirety, and to have on hand for future reference. The dry medical data is broken up with real life stories of people who were able to relieve themselves of a host of health issues, simply by changing their diet (and sometimes taking antifungal medications). This makes the books much more accessible and easily readable.ABOUT THE DIET PLAN: This is an easy to understand and easy to follow plan. Although it is lo carb, it is not like Atkins or the other lo carb diets out there. It is specifically engineered to kill fungus in the system. The set includes a program booklet, complete with foods allowed and not allowed on each of the three phases of the diet, and a full week's menu plan. It is not a starvation diet, but very low in carbohydrates, with the carbs increasing gradually as the patient advances in the program. The program advocates freshly juiced veggies in addition to the rest of the diet. I ate a lot and was satisfied - sometimes I got depressed about the amount I was eating because I was already overweight and thought I must be gaining. That is why I avoided weighing myself. The weight loss was a HUGE surprise! I found out that good health and not diets produce weight loss and lean muscle mass.Unfortunately, I slipped out of my good habits, and both the weight and the disease returned. Now, I am back on the program and doing great. This time I know what can happen if I let my guard down, and will never go back to a sugary, carb based, fungus feeding diet again. The lapse back into diabetes and obesity was my own fault and not the fault of the program or it's authors.I really hope this information gets out to the general public, as it doesn't seem to be a well known book. Just be sure to buy it at a fair price.",e +I love this CD. It is soothing and wonderful for anytime the stress level needs to come down. People with children or high stress jobs will love this. A cup of tea and some lotion for massage will round out a good 20 minute stress reducer.,w +"[[ASIN:B01EUC2RG8 Fast Tracks For Men, 2.4 Fl. Oz. / 70 ml - Inspired By Ferrari Cologne]]I was looking for new scents or signature scents for my fiance and wanted to try this cologne. I like and appreciate the scent of Ferrari cologne so I thought maybe my fiance (Bobby) would like to try it. This is a way for us to get an idea of what a scent smells like with Bobby's chemistry without spending a lot of money and then not like the scent of it. If he likes it then we can continue purchasing it. The seller delivered very quickly and we received packaged well. Since this cologne is an Eau De Toilette Spray, it will have slightly more of the oils (10%) that make the spray smell great than just cologne which has about 7% oil. The scent also is delivered in a convenient spray which me and Bobby both like. The cologne actually smells very nice on him. It's a sort of a fresh woodsy, masculine scent and you can definitely smell a hint of lavender and cedar in it. Ferrari describes the scent as " notes of mint and lavender with pepper, black tea, and lebanese cedar to create a memorable scent that weaves distinguished with daring." Some of those things I have no idea what they smell like, but I'd say this not really a musk based scent but it smells great on Bobby. It's still strong enough scent with about three or four sprays being all he needs. As far as wear, it lasts a pretty good while. It's a nice manly in it's scent and he said he received several complements on the cologne. It is also a large bottle at 2.4 ounces delivered to the door for under 11.00. We like the cologne and can recommend it. I purchased this product at a discounted price in exchange for my honest review. The discount I received will not alter my honest opinion in any way. Since I count on honest product reviews to make my selections on Amazon, I will always be forthcoming and sincere in all my reviews. No matter if the product is discounted or not, I will leave honest feedback that I base on my own personal experience with the product I receive. I am in no way affiliated with any supplier or manufacturer of goods.",E +Great CD set. Lots of good old country songs. Nice easy listening background music perfect for any mood you're in.,k +Great music. Song by the original stars. Gets you in the patriotic mood so needed in these troubling times for the USA,b +"Very relaxing, great price for 3 cd's",[ +Thank you Smithereens!!!This version of Rudolph the Red Nosed Reindeer Really Rocks.Get your Christmas started with Bang & get this CD.Fine service from seller.,G +"I love Brian Culbertson's music, so this album is to my liking as well. I love the upbeat party songs, as well as the slower one. He gives you a nice mix of music that always displays his musical talents.",t +Very enjoyable,H +I've always loved Elvis and really enjoy the combination of him and the Royal Philharmonic.,o +"Good Mathis quality, but a bit disappointed in arrangements on some.",y +Great music to a great Movie,y +Very great collection of their early work and cover material! A must!,n +Beautiful! and wonderful quality!,B +I've only managed to listen to the 1st CD so far but so far....I love it!!,H +"Great set-list, not so great recording. It is still worth it for the Massive Attack fans out there.",e +This offers great memories prior to the time these four artists became top entertainers,G +as described,G +"Mr. Wronski had already retired from the performance world when he was in the U.S., where he taught for a few years at Indiana U. in Bloomington. He was a brilliant man, very analytical of technical difficulties, and very inspiring for musical ones. Fingerings and shifting are still way ahead of the times, and provide excellent solutions to thorny pieces like the Berg. While this album is not recorded with the kind of equipment we have come to expect today, it clearly demonstrates that he was a first rate performer, and gives interesting insights as to how these pieces were being interpreted behind what was then the Iron Curtain. He was the Rector of the Warsaw Conservatory, and told us that there, every aspect of the students' lives was guided by their teachers, from how often they went out partying or on dates, to when they had to start abstaining from alcohol before concerts, and so on. Of course, each teacher only 'supervised' three or four students for their work load, and their job was to turn them into marketable artists. Quite a contrast from the 'Henry Ford' approach used in schools in the US today! The recording is a unique and fascinating glimpse into a different world of performance practices.",x +"But my mom, for whom I bought it, absolutely loves it. That is all I was hoping for. So, yeah - five stars.",E +"This was purchased because of special memories of my teenage years. One of ""old blue eyes' ""songs just gives me a warm feeling again after all this time. It's one that isn't heard much but will always be special to me. It's called: ""If You Are But a Dream"" and it still melts my heart!",A +Like the songs - but the CD overall pretty much sounds like the same tune put to each song. Nice overall CD,e +its ok - only 2 songs on the cd I really enjoy.,W +I love the CD but when I got it the case was broken love the CD will buy more again,l +"Excellent cd. Bought two copies,one to give away.",X +"I haven't needed these yet, but good to have for emergencies.",K +"I've long been a fan of Stevie, so the songs on the first disc aren't really anything new to me, but I still think this is worth purchasing for the second disc, which contains videos and behind-the-scenes footage. To me, the best part is Stevie's commentary on the videos... She seems to be having a good time watching them and isn't afraid to make fun of herself. It was a lot of fun to listen to.",s +"the two prior reviews for this album indicate the two types of responses people are likely to have to this album -- some will just hear it as noise, while others will hear it as wonderful layers of beautiful bliss. this isn't standard shoegaze... it's for people who appreciate big gauzy squalls of feedback and wisftul, barely-heard vocals, all held together by a semi-structured sense of melody.",J +Heard it in a boutique - found out what the CD was called and had to purchase.,] +Some old recordings that I have not heard before. You will notice that many have been recorded by some of the greats today who learned from the blues masters of yesteryear. Worth the investment.,c +Was a gift and loved by the receiver,d +She loved it!,P +The five stars are for the cd itself. One of the best I’ve ever heard. But shame on the seller for charging such an inflated price for a cd!,a +Informative. Up to date guide. Well worth it,M +Really enjoy her CD! She has a beautiful voice and even though most of the songs are in Japanese I STILL love to listen to it!,T +You won't want to hear anyone else sing this after hearing Ms. Lynn's version,` +"I got a copy of this single from my parents when I was a child around 1970. I have no idea how they came to own it to pass it to me. Unfortunately it has since been destroyed but I do have a tape recording of it and I managed to get a digital copy from the original 45 I owned before that happened. It's a surreal experience listening to the two tracks, ""In 1967,"" and ""Catwalk."" To this day it's my favorite record. If I could afford this seller's price I would most certainly buy it. It's truly a unique experience.",L +Awesome worship music,I +Love it. Thanks seller!,x +had this album along time ago now I can listen to it over and over,Y +".Scoring:5 stars: Star Wars Fans - MUST SEE!1 star: Non-Star Wars Fans - Don't Bother!0 stars: Younger Children - Mom's be careful, too scary!4 stars: Stuck-in-the-Middle Star Wars ViewersSummary:Simply, another great hit for star wars fans! Filled with great movie technologies, great action, good script-writing, good acting, very good directing, and all those symbolic and instructive details that requires you to watch the movie 20 times to catch the story's subtle underlying themes. Truly another Star Wars movie that changes Hollywood and its technology!Review:While darker than Episode I, it continues to fill in the questions that are left in our minds, including how someone with the promise to do so much good (Anakin Skywalker - Hayden Christensen) can become trapped by evil. (How many times do we fall prey to these same issues?)One aspect of Lucas' genius is that we have waited almost 25 years to learn many of the underlying story drivers from the first movie. Our curiosity drags us back searching for little details of the ""force"", Skywalker's beginning, Yoda, Jedi Knight' motives,Lucas's digital film making and THX are dramatic, eye catching, enthralling, and bring the movie to new levels of entertainment. (Where would Hollywood be today without George Lucas and his technology?)The script has been attacked by many critics, but is a sure winner with the fans!Story:Anakin Skywalker / Darth Vader-to-be (Hayden Christiansen) is sent investigate attacks and protect Senator Padmé Amidala's (Natalie Portman) who struggles against the creation of an army by the Galactic Republic. The Anakin's relationship with Amidala blossoms while it sours with his domineering Master Jedi, Obi-Wan Kenobi (Ewan McGreggor).Negatives:Ok, there are some dialogue weaknesses within Skywalker and Padmé Amidala's (Portman) love story.And, the movie is darker than some movie goers probably want.Scare Factor:Don't bring your younger children to this one: Episode I was made for younger children, this one is darker and therefore more fierce-some for the little ones.",G +Great set!,F +"While I like this joyful song very much, this isn't a version that I especially like. The lyrics seem hard to understand.Juanita Henry",p +Superb!,W +"This is excellent, fun and funny for kids. Kids love the stories and rhyming stories and it doesn't annoy adults. i really don't understand why John Dashney isn't famous. in any event, this is one of those hidden treasures. Discover it.",X +Love the cd.,t +"It's okay--the melody is the same. I'm "old school" and love the old recording by Peter, Paul and Mary! simply okay!",i +"We absolutely love all the songs on these CDs. We have it in the car and the kids love the catchy tunes. They are easy to learn and sometimes I'm singing them around the house myself! We are thinking about having the kids sing a couple of their favorites in church. We have the most fun with "Keep Your Tongue!" haha. Great music, great voices and not cheesy. Thank you for these CDs!",M +Fun!,W +Fun music,W +"This is a great introduction to Mozart with many of his ""greatest hits"", performed with skill and beautifully produced. I keep this disk in my car and use it when I have to drive in blizzards or thunderstorms -- it is very energizing yet focusing. My congratulations to the IMP group for putting this excellent compilation together.",_ +I bought this cd as a gift and they are very happy with it.,] +"Just what I was looking for his Christmas present, he is so hard to buy for...",o +"I absolutely love this CD! From before Miranda was signed/famous. Songs have an old country feel to them. She's amazing even when she was just a teen. If you are a fan, this is a must have.",E +"The western songs, including movie and TV western theme songs, are pleasant to listen to, especially as interpreted by the Boston Pops.",J +"Learned songs in french, german,spanish & and japanese so whole world could enjoy his talent. Learned phonetically and by rote. Sings with soul & you'd never guess he doesn't know words he is singing. Especially love the Spanish CD.",a +"""100 Greatest Kids Songs"" is a little bit of a stretch. There are maybe 50 top kid songs and then a lot of nursery rhymes and random other kids related things that are set to a melody. With that said, with today's technology, one can easily create a CD with your child's favorite songs and leave out the random ones.",Y +"I bought this for a Christmas present for a Coldplay fan. I am told that this cd is awesome and has all of their hits on it. When I was searching for this cd, amazon was the only place that carried it.",J +The selections were for the most part uninspiring. It's going into the donation box.,D +Love love love this cd from Kyle Matthews,w +I never got it but I did eventually purchased it again and it is a game changer for teaching. The price is amazingly affordable too!,^ +Loved the dvd and cd!,E +Another incredible album. Love the middle eastern sound in the first track.,^ +very talented group from ukraine. enjoy listening to their music,e +Like new even though it is 30years old. Well packaged with quick shipment. One of the best of all time string quartets.,U +I love listening to this cd. Great old song that put your mood right.,Y +Arrived faster than expected and is such a great SD staple for those long summer drives to places unknown.,e +"Nice copy, well rated, plays surprisingly quiet for a 30-40 year old vinyl.",N +Everyone loves Nora’s crossover Jazz & country music.,F +Amazing voice,d +none,c +have the record and now both albums on one cd great old 70s music takes me back. came from germany but still made it here in 11 days great job,P +"Yes, I am being just slightly hyperbolic, but only a little. This recording, made in 1957 and re-mastered exactly fifty years later by United Archives, received a Diapason d'Or but does not seem to have caught on all that much with the listening or reviewing public. This is truly unfortunate, because it is an amazingly impressive traversal of Brahms' First, one of the pinnacles of the Germanic symphonic repertoire. In fact, as a great fan of this work, I am inclined to declare it my official choice (as of now) for reference recording. It would make a fantastic introduction to the symphony or the composer, and if you are even a diffident collector of Brahms Firsts, there is no excuse not to acquire this recording.George Szell, as leader of the Cleveland Orchestra, has a reputation for clear, balanced, swift, and "classical" readings of standard repertoire, and while those adjectives apply here, this performance can certainly not be reduced to them. It is also tough, propulsive, precise, dramatic, and even aggressive at moments. Accents are precise and marked in a way that will be familiar to listeners immersed in today's historically-informed approach to Romantic works, but unlike those readings, it never sacrifices long-term architecture for momentary payoff in articulation. Overall, the sense here is of a single line; the tempi are consistent and steady (but never inflexible), and the music is always going somewhere. Szell and his instantaneously responsive Clevelanders never tread water, never waste a phrase: everything is in its place, and the structure of the symphony is beautifully revealed.Above all, to my ears, the trap in Brahms is to indulge in his dense scoring and let everything become soupy and indistinct. In fact, the music will often sound reasonably passable even when presented by a conductor who fails to clarify all of the mid-register lines and subtle rhythmic elements. This is a testament to Brahms's compositional skill, but the effect of the music in the hands of a conductor and ensemble capable of presenting his work with perfect clarity is astonishing and revelatory. Szell/Cleveland, particularly on this recording, is just such a collaboration. If all performances were of this standard, Brahms would not be thought of as a "muddy" composer or orchestrator. This does not, however, mean that Szell's reading of this symphony is "light" or "classicized." The feat is accomplished with no lack of power. Indeed, the driving rhythm of the introduction has rarely sounded so arresting, and the physical impact of the timpani and strings throughout is quite impressive. And while the tempi tend not to linger, Szell is never more than a minute and a half brisker than Herbert von Karajan's reading from his 77-78 cycle (this rather lush, supremely Karajan-esque set was my introduction to the symphonies, and while those performances are not perfect, they do seem to represent a middle path between the lighter Brahms of Toscanini and Szell, and the slower, thicker textures of, say, Celibidache). In fact, he is more than twenty seconds broader in the Andante Sostenuto.Finally, a word must be reserved for the remastering of this recording. I am not a serious audiophile or collector of historic recordings, but I have personally never heard a 1957 recording sound so clear and natural. This early stereo is demonstration-quality stuff, and it sounds worlds better than plenty of later stereo and digital recordings I could name. Never is anything remotely artificial, and Szell's interpretation is well served by an acoustic that allows the timpani to resound where they are often buried in a generalized orchestral slurry, and that bestows upon the strings a rosiny, just slightly gritty bite.In short, you owe it to yourself to pick this up. It is a bit pricey right now on Amazon, but I've seen it here and there in used record shops, being sold new for less than $10. It is, however, well worth full price and more. In my opinion, the only serious competition comes from Fürtwängler, whose apocalyptic live 1952 performance might be the best thing that ever happened to the symphony. However, the sound is far inferior, and Fürtwängler's traversal has the character of a one-off "event," complete with extremely idiosyncratic tempi (the introduction is hammered out at probably less than half Szell's speed) and abrupt gear shifts that make little sense on paper but perfect sense in the moment. The difference should be obvious already: where Fürtwängler's performance emanates directly from his overwhelming personality, while Szell's is far more objective. It is, quite simply, the apotheosis of objective, clear, and propulsive Brahms, and as such, makes for the natural reference recording. It pairs well with the Fürtwängler, but does not suffer by comparison. Buy it.",l +What could be an OK not great album made un listenable by an over amped overzealous audio engineer.Completely blown out low end. Muddy crap.Too bad cause I like these guys.,F +In good shape as described.,r +Old school what real music is,m +Great CD. Well packaged,T +"This is one of my favorite unknown bands, from Champaign, Illinois. They have a captivating, ethereal, relaxing sound that really is unique. The guitarist sounds inspired somewhat by The Edge from U2 (in The Edge's quieter moments like the song ""Bad"") and the lead female singer has a whispy, sweet, but rich sound similar to the lead singer of the Cowboy Junkies. Very excellent, relaxing music. The drummer was a punkish type who really adds a bit of energy and spice to the otherwise lulling sound of the rest of the band. A very good blend of sound.The band never made anything better than this album, which is unfortunate because they are better than their Cowboy Junkie popular counterparts.",y +Fantastic CD,n +"This CS is a no nonsense guide to learning code. It jumps right on in and starts. the narration gets to you so I would only recommend short lessons with frequency. Otherwise, you do really seem to pick up the code fairly easily.",v +"What a wonderful concept and fantastic execution! Bennett still sounds great (Hey he's not 30-get over it), Krall is simply the preeminent jazz vocalist of this era, and the Bill Charlap Trio are simply divine as well. This record is a gift for fans of the Swing era, young and old. I've got a new appreciation for the Gershwins now and want to hear these songs again and again. Just like when I first heard Michael Buble, listening to this record makes not now only happy but also excited to learn more and seek out all the music of the Swing era and its modern interpreters.",k +"I’m shocked this took so long to be released. It’s just astounding. The sound quality is much better than I anticipated for a bootleg- well is it really that? As others have stated, the “sound check” songs suffer in quality but are worth it for the Mean Streets prequel- Voodoo Queen.Wish I had been at this show at the Pasadena Civic auditorium in October 77. The energy is nothing short of tremendous. This stunning set was recorded a couple of months before the band's superb debut album was released (in February 1978), the entire affair was recorded for live FM radio broadcast. According to many, this pivotal early gig remains quite probably the best concert the group put on during this time in their career.This disc is worth buying for Eddie’s performance in Eruption alone. He changed rock forever, has no equal. All four musicians are flawless. Dave’s screaming act is as hilarious as it is entertaining; he really is the best front-man ever. On this set he’s an evangelist of hard rock. VH fans who don’t own this, order it today!",G +"(Like tomorrow is the end of the world) - That exclamation pretty much sums up these discs and Steel Panther in general. This CD/DVD combo is some fantastic over-the-top entertainment, as only Steel Panther could pull off. The music is killer, and the “concert” with Bobbie Brown as bassist Lexxi’s mom is even better.All the songs are great done acoustically: “Fat Girl’” “Community Property,” and “Gloryhole” and the other SP classics. The CD’s lone new track “That’s When You Came In” stands out as one of this disc’s best songs and one of the band’s most melodic- a true double entendre hit single. There are differences between the CD and DVD’s tracks, and I prefer the later.All comedy bravado aside, these guys really excellent musicians who write memorable songs. Guitarist Satchel in particular impresses with his acoustic guitar in this short set.In allowing himself to be the butt of the joke throughout the concert/story, Lexxi must have the greatest sense of humor ever. Spoiler alert: His bedroom, looks nothing like you’d expect-but stop and think about it. Thanks SP for bringing back hair metal icon Bobbie Brown; her return to fame is long overdue.Come along for the “Weenie Ride!”",g +"Overall, I’m happy to at least have the album, but if you’re going to buy to get the whole set as a fan, don’t do it. Pre-order pack came in broken at the bottom, not giving too much reasoning for the seal at the top. Also, pre-order states there should be a poster and I did not receive one. It did come in early, but overall it wasn’t much of a surprise.",J +"All the songs I love on one CD, can't be beat.",N +"Love EVH, and this album was their beginning, so I was glad to find it.",s +"She is SO talented--I am also a singer so I cannot imagine singing all your life and then suddenly not being able to sing anymore. To me, that sounds tragic. I've seen interviews with her, and I guess at this stage she blessed to be to even talk. Luckily, she left us a legacy of music with her beautiful and versatile voice. Wow, this CD was so worth it.",N +Excellent cd in excellent condition,[ +My daughter love it,J +Great gift & fast delivery.,F +I'm a child of the 80's . This disc will take you back.,H +"Bonus tracks galore, so very happy!",r +The Brothers Four could not do a bad album if they tried.,m +"Not bad for a casual listen but there appears to be too much in the way of orchestration and repetition and little in the way of variation and improvisation, which to me is the essence of Jazz. Of course, like abstract art, Jazz appeals to the senses in somewhat mysterious ways. There are two more Indigo CDs on the way but I doubt that there will be little difference in their style.",R +My three year old loves this soundtrack. It is perfect for little kids. We watched the movie too & he loves to sing & dance in the rain now!,E +Love it!!,S +Plays beautifully thanks ! A great collection of Joni Mitchell,N +So happy to get a vintage vinyl inGreat Shape! All sounds great& perfect thank you so much.,V +Love this soundtrack!,z +awesome gift,j +"Esta es una de sus menos conocidas películas, pero en mi opinión una de las mejores. No se realmente como este talentoso director convierte escenas fuertes de violencia de toda clase y encuentra humor en ello. Realmente nos recuerda a no tomarnos tan en serio. Definitivamente un gran profesor y artista. Los actores todos fabulosos(menos el Peter Coyote, que no sé para que diablos lo inyectaron en tal producción, blah!)Todo lo demás perfecto. Bravo!",T +"Richard Cheese takes popular songs and awesomizes them. You can listen to them anytime, anywhere. If you have heard even one of his songs and like his style you will love this album. If you don't like his style, don't buy his albums. They are all great.",L +"I hate to say it, but I was not satisfied with this CD. Perhaps, if you're a tourist with only half a day to drive the Road to Hana, you'd be happy with this as the information is very basic and the CD is rather short. However, I was hoping for some more in-depth information that would give me tips to find the things that the average tourist misses. Suffice it to say that I listened to the CD while still at home and didn't even bother to pack it in my Hawaii-bound suitcase.",` +very cute,M +"Enjoy this workout. It would be better if it came with an instructional sheet showing proper position for various exercises; however, it provides a variety a good workout. I use the Diver waterproof mp3 player to follow along in the water and the time goes by very quickly. It's a good value for the price.",O +Shipping for this album was quick considering it came from South Korea. It was packaged very carefully also. Very good album! Thank you!,T +"This CD is ok. Not exactly what I was expecting. Some songs are movement songs, but the instructions are typically before the song starts, not really integrated into the music. It is a decent CD to ahve in the rotation for my toddler, but wouldn't be a top pick.",[ +"This disc just kicks serious booty!!! With a number of the songs performed in their native language, nothing is lost in translation. It is like listening to an opera, you can tell by the singer’s emotions what the song is about.",n +"This is an awesome little CD that will introduce anyone to many moments of the great works of Mozart.I bought this album originally in a cassette in about 1990, and then got the CD 10 or so years later. This one is a backup of my original CD, if it ever goes bad. Which speaks to its quality, since these are out of print.This CD is a refuge of sanity in the car, lovely at night when you are trying to quiet down and get ready for bed, or taking a long bath - also good background dinner music, even for those who think they don't like classical music.*One note - I almost gave it 4 stars, because these are excerpts of the Mozart movements, not the complete movements. I think when the London Symphony Orchestra did this project with Victoria Secret (when VS was CLASSY, not trashy), maybe they decided to make shorter, simpler versions for brand new classical music listeners, as an intro. I recognize that, now that I'm more familiar with the works. Little disappointment. But these are lovely, nonetheless - and for me, the album that pulled me in so that now I adore Mozart.Another suggestion for a Mozart intro CD I love is the Classical Composers album with the little book. I love it.",] +"Found this one at a thrift store too. It is so fantastic, that i have bought more to share with family and friends! This has been my favorite CD for 3+ years. I adore Mozart, It’s great driving music. The Marriage of Figaro overture (first track) never fails to put me in a good mood, and help me catch up if I’m running late. It’s a great gift for people just learning about Mozart, and for those of us who have left him for years.",x +"As a first release that is self-produced, this is well done...catchy, good lyrics, varied.",T +Does what it is supposed to but could use some heavier gauge wires so you wont need to worry about running 5 Amps across item wires are flimsy and thin soldering on connection not great or insulation. Motorcycles are outside so it should be done with that in mind.,` +baby loves this! great childhood memories for me. first cd had songs from cartoons and the second cd has songs from kid movies. LOVE IT!,H +"Heard a piece from this on PBS and was captivated. I don't know how this soul became one with a violin to produce music for the heart broken but he has done it. If you have ever had a lost dream, a broken spirit, a forgotten promise or the deepest sorrow that your heart could stand then you will love this as I do. I have never heard the violin played to emote such pathos, deep regret and sadness as this. Beautiful in its sorrow.",H +What's not to like about a signed c.d. This was a holiday gift for my mom and she appreciated it. There is a second cover that is signed - so you do have to open the wrapping or just tell whom you give it to to be careful.,j +"I have quite a few of Tom Kenyon's CDs, and have, for the most part, come to expect an enjoyable, out of ordinary and out-of-the-box work and sounds from Tom...However I find this particular CD quite disappointing. I was imagining some unusual, celestial, soothing sounds, but this just feels like he is practicing ""sounding"" (making sounds to express himself).The results are 8 tracks offering very little variations so they all sound pretty much the same - not very intriguing or interesting in the first place, repetitious, quite monotonous, somewhat annoyingly high pitched.I got bored listening to this CD in a hurry, and it will be a while before I go back to it again.Sorry Tom, maybe I just did not `get it'?",O +"Alexander Rybak won the 2009 Eurovision contest, one of the most prestigious European musical manifestations. All his songs, on both CDs, are sensitive, profound and romantic. He is my 16 years old daughter's favorite. We bought all his CDs and are delighted to listen to them incessantly.",N +Schippers was a musical genius. His death at 47 a tragedy.,a +In my opinion this is one of the finest recordings ever made.,` +"Re Pires: The best played, best recorded CD in my entire collection.",X +The cd and the case arrived in perfect condition and the bonus tracks were worth paying a bit more than the original cd that other had without the bonus track.,J +Great flashback album from childhood (in Spanish),M +Just started playing autoharp and just started listening to her music. Where have I been! Makes me want to be a better musician.,G +I love the Janet and Judy album. I hope they come back on lp. Please bring them back on lp.,b +Was a gift for a friend remai s one of my favoritw albums,p +"So soothing. If you like Mozart and music to calm the nerves, this is the CD for you.",^ +:¦:-•:*:•.-:¦:-FAST SHIPPING-BUY WITH CONFIDENCE A+-:¦:-•:*:•.-:¦:-,R +:¦:-•:*:•.-:¦:-FAST SHIPPING-BUY WITH CONFIDENCE A+-:¦:-•:*:•.-:¦:-,t +"Case was cracked upon arrival, but CD was fine.",h +"This band is wonderful, fresh and exciting. The sound is like Afro-Caribbean, soul, folk and dub, blended together and pored out creamy and smooth.",S +"Clive Cussler's Dragon is the best of his books I've read in awhile. Dirk Pitt and friends are back to defend the US from being taken over, not by combat, but by economics, by Japan! Well, actually by a group of Japanese mafia-like group. This a very original plot line that is completely fascinating! I lost a lot of sleep over many weeks listening to this exciting recording.",f +"I have added this to my new HemiSync collection and while I like other CDs more--Gaia, Chakra Journey--I like and will use this one.",Y +"Takes us back to the early years,with their early hits,and their versions during rehearsals,recording sessions to the master recordings",\ +"A very special CD, with the ""BEATLES"" VARIOUS VERSIONS of many of their hits during recording sessions, before they recorded the ""MASTERS"" which were released onto records,and the story behind the recording sessions of these ""masters""",B +Double sided of the same picture. All in all a great buy,[ +This cd is hilarious! Don Rickles is a genius and I'm happy I bought this. It shipped quickly and I love it!,O +A nice CD to play at harvest time.,J +Joni sang with her magical voice while I did some spring cleaning. It was a lovely afternoon.,p +All of it,X +I had lost my album and am so happy to have the music back in my life. This an uplifting production.,T +"my wife so glad to get this CD, she just loved the movie.",^ +"The screwdriver that fits into the plastic body does not latch in at all on mine. It has a little rubber gasket that looks like it is supposed keep it held in by friction, but it doesn't. The screwdriver will fall out if I just turn the tool upside down.",y +My wife liked what she received.,_ +My dad loves it. It brings back memories for him.,B +"Unfortunately, this did not have any sample soundtracks. I was looking for a good CD of just thunderstorms and the description for this gave me the impression this was what I was looking for. Its not. The thunderstorms are wonderful, but their is also music with the storms which I found distracting.",B +"I live about 60 miles from where the horses live, I feel so good to see the horses along the Carson River feeding and with their colts this music reminds me of the wild animals in the area near my home.",\ +"This is an excellent seminar! Entertaining and informative, not a minute is wasted, this series should be mandatory for all of us. I have now listened to both the Development of Men and Understanding Women and found both to be right on, I wish I would have had this information years ago when I was first married... and, especially, as I raised my kids. So many questions would have been answered and so many conflicts would have been avoided. Highly recommend!!!",s +This album is a great representation of the type of music that she has sung so well over the years. It also includes a booklet with pictures and a biography of this wonderful singer/actress.,H +"Please read this review carefully!I have to confess, I didn't shell out $179.00 for this 2-CD set. Instead, I downloaded ""The Brussels Affair"" from Google Music for $4.99. That download contains 15 songs: Brown Sugar, Gimme Shelter, Happy, Tumblin' Dice, Star Star, Dancin' with Mr. D, Heartbreaker (Do Do Do Do Do), Angie, You Can't Always Get What You Want, Midnight Rambler, Honky Tonk Women, All Down the Line, Rip This Joint, JJ Flash and Street Fighting Man.This review is about that MP3 download.The download is a cleaned up, re-mixed and re-ordered (not sure why???) version of something originally broadcast years ago on the King Biscuit Flour Hour. It is (as near as I can tell) culled from several of the performances recorded on the $179.00 CD. But the 15 cuts I downloaded are of excellent, very high quality and mixed perfectly. Every note of Keith and Mick's guitars, Wyman's bass, Ian Stewart's (I assume) piano and Jagger's vocals come through crisp and clean. The performance is extraordinary. Better even that Get Your Ya Yas Out. Both Mick Taylor and Keith Richards are so ""on"" that it's scary. This is the best live playing by either of them separately or together that I have ever heard. Compared to this, the latter day Rolling Stones live recordings with Ron Wood'a wimpy""lead"" playing make me cringe. There is simply no comparison. Taylor's leads on almost everything are near-perfect, and Keith's ferocious rhythm guitar, most notably on Midnight Rambler, is nothing short of awe-inspiring. This was the ""real"" Rolling Stones. So what are you waiting for? Go find the $4.99 download and I bet you never listen to another live recording with Ron Wood playing ""lead"" again.Very Highly Recommended.JMT",y +Plays well..exactly as advertised,] +Love the Lord of The Rings. Nice to listen when there is no time to read. Well narrated.,X +Wonderful group. Great vocalists,g +Great take on Middle Earth. Lends itself nicely to the atmosphere of The Lord of The Rings.,E +This is a very eclectic music style. I really love this music. Too bad they split up!,p +Exactly what I wanted and love getting CD's of my old beat up records.,v +On time and in perfect condition,d +Only 3000 copies of this soundtrack was issued. A very nice soundtrack to a sadly treated SciFi series by Fox.,l +"First off if you want to shake the house with Wanamaker peddle notes this is not the recording for you. What this is, is historical performances by world class organists circa 1940. This type of recording is important in your collection because you are listening to how a particular organist is playing, his style, meter, etc you are not really listening to the organ its self. If you are an organist yourself these recording would be an important thing to use with a musical score to make notes as to how to play something in, as an example, the style of Virgil Fox who said he added 6 notes to what ever J.S.Bach wrote. If you love the music of these gigantic pipe organs, and especially are fans of those who played them in the first half of the last century you must have this CD in your collection.",S +Love Brenda,x +Disappointing. Played once and donated to the library.,F +"I received this CD several days before the actual due date, good job guys! The CD is an import and the sound quality is very good which to be honest, I was a little worried about.I collect music (CD's mostly, Vinyl when I have to)for almost 50 years...yikes! Anyway I bought this CD for one song, the Bobby Russell, ""1432 Franklin Park Circl Hero"" which I first picked up in a 45's sale bin when dinosaurs ruled the earth and Bobby Sherman was popular. I'm so glad to finally hear this song without scratches.",D +Listening puts you in a better mood,_ +"I never can quite find the words for music when its this beautiful. And that's saying a lot, because I tend to thinkthat there are words for everything. This is the kind of music that will shift the furniture in your mind around, let ina lot of fresh air and light, and leave you wanting more. At times, it's like something in a dream, while at others, itis clarity itself.",n +"I adore Italian opera -- particularly Puccini. That makes this CD a real gem. All my favorite arias and duets are here, sung by well-known and talented artists.If you're new to opera, you can't go wrong with this album. Puccini was the Spielberg of his time, and his work still has a lot of popular appeal. (It's much easier to like ""La Boheme"" than Wagner's Ring cycle -- that's for sure!)If you have a lot of opera already, you probably have most of these songs. Still, if you want them all in one place, this is a worthy buy.",c +Oh yeah... good music. Kind of keep looping in your mind even when not listening to it...,e +Excellent CD. A-side is a really good song.B-side is even better.Limited version comes with a DVD,b +Excellent CD Single,r +"Well, I'm a PERFUME fan, so of course it's FIVE STARS.",^ +This is the second time I have purchased from this seller. Fast shipping and item as described.,[ +My children and their friends LOVE this CD. Not sure about this seller since I purchased mine for download.,_ +"Sick as a dog I trudged through the snow at night in New York City to see Betty Buckley perform in Sunset Boulevard, and it was well worth it. This is great memento of the show that I bought at the theatre after the performance, and I treasure it, but please note that the exact same songs are on other CDs of hers such as [[ASIN:B00000K2CQ Betty Buckley's Broadway]]. Only a completist should consider buying this at this point since the price is so high.",i +"Over all I really liked it. Great interpretation of some pop standards like 'Summerwind' , 'Thanks for the Memories"" Even decent vocals by Monty Alexander, particularly on 'Hard Times' But I did not like what happened at the mixing board. Somebody did not like Ray Brown. When there was a Bass solo, the volume dropped way down. On the first track, 'Blues for Groundhog' that Brown was credited for composing, it opened with a long bass solo and I had to crank it way up to hear it at all. Yet with others playing, the bass came through clearly. Wish the liner notes had more information, but it was in German which I could not read.",O +"I bought this when it came out and ended up giving it to a friend who never returned it. If you're a die-hard Lucero friend, it's a fun watch. Filmed during the time leading up and including the recording of Bikeriders (which is my opinion their best album and a real turning point for the band). I've loved this band since I first saw them touring That Much Further West and I'm impressed and amazed at the growth and maturity of the band. They deserve all the success and recognition. This is a fun time time capsule of a young, drunk, ready-to-break Lucero.Watch it for free on YouTube in it's entirety - don't pay $200+ for it here - use that money to prop up artists that have had a tough year due to cancelling shows/tours.",i +"My brother passed away from a very aggressive brain tumor in November. No warning, few symptoms. Went into hospital on a Sat and died the following sat from a seizure. Luckily he was not in pain & really didn't realize how bad things were. He wasn't his normal self, but he still joked around. it was a blessing that he died like he did. I have talked to too many people that know people that have died from Glioblastoma, it is awful and painful. That being said, he was my big brother and the song on this cd was beyond perfect for the memorial service for him. "Jealous of the Angels" is an amazingly beautiful song on this CD. I listen to it all the time in my car. I have come to love the entire CD. After reading Donna Taggarts biography the songs meant even more to me. Amazing woman!",Q +This CD is part of a series that have been my favorite for years. I am thankful to have it (and the others) on CD again!,v +This CD is part of a series that have been my favorite for years. I am thankful to have it (and the others) on CD again!,k +"I discovered Anna Hutto at least 10-years ago. She came to our church in Lubbock for a concert. I'm not sure how that came to be, but was I amazed at her talent with words and loved to listen to her voice. At the time the only CD she had was Journey, which I bought at the concert.I tried to find other recording by her over the years, but without success. A few years ago I heard she was producing a new recording, but I missed it. Fortunately, I discovered the recording, Welcome Here, a couple of weeks ago. I managed to capture a used copy from the Amazon Marketplace. It arrived this afternoon. It's already ripped and stored in my iTunes library and on my iPod. I listened this afternoon and was amazed that her voice sound more mature... a little richer.Her talent with words is unabated as well. It will take me a few more listens to really get the work, but I already know this will be one of my favorite recordings, just like Journey before it.All I can say is that I wish there was more of her work. It's good work.",E +"Great song, arrived in great condition!",R +Very sweet album.,j +"Loved this album years and years ago, although his voice is not the nicest vocal. Haha. But the lyrics and the rhythm suited me just fine. Listening to it now brings back pleasant memories of lifetimes past. Too bad he squandered his career as an artist (I think). But very few people can really withstand so much sudden notoriety, I guess. Anyway, the package was nice, came way ahead of schedule, and I think this is a collectible now.There is another Edward Furlong album (one with poems and such). Anybody selling it?",T +"Excellent music, received as described.",Z +Fun book for kids. I like the positive slant book takes on things that could potentially be frightening for children.,y +"Horrible book. If you are a Prince fan do you want pictures of an empty airport, empty hotel rooms, and spreads of his background dancers? No. There are way too many 'filler' pictures of people and things who are not Prince. Extremely disappointed.",v +Wish I could have heard him play in person! This is a real treat.,o +The 90s ruled music and this is a reminder of that fact. It's a great album that should be in any lover of rock's collection.,Q +The CD arrived in excellent condition. The seller took great care to package the CD so it wouldn't get damaged. It was brand new and sounded great. Brings back old times listening to Bobby Womack.,E +Happy to finally find this. Sadly my phone won't play cassettes.,r +My grandson loves this CD.,x +Pentatonic is a fantastic singing group and I bought this for Christmas … perfect for the holiday season!,v +"Received 3 days early and was well packaged and is exactly as described by the seller. The record is as flat as my turntable platter and tracks and plays beautiful. It is hard to find an original 45 in such good shape. This is the original recording of "YOU" on Fargo Records 1958 and is done so beautifully by Lynne Nixon (original vocalist). After her passing, and in later years, it was recorded by Collette DeLaney with the original Aquatones as backup. Collette's rendition is also totally awesome as both artists sounded so close in vocals. The seller went the extra mile and put it in a nice sleeve with plastic liner covering the holes. I am very satisfied with my purchase and highly recommend this seller, and would not hesitate to order again... Customer satisfaction is a strong point...............5TARS",k +Received 3 days early and was well packaged with no damage. Overall excellent record that plays beautiful with no faults. I am very satisfied with my purchase as well as the sellers description being accurate. Would not hesitate to order again. Went the extra mile to put jacket in vinyl sleeve for added protection. 5 STARS............,L +"I received this album 3 days early which was a nice surprise. The packaging was excellent, as the record was out of it's jacket in the inner sleeve and sandwiched between the jacket and an empty jacket. It was super cushioned to avoid any damage. The sellers concerns that you get a good product that is undamaged is MUCH appreciated. The record was as described by the seller and plays with no pops, ticks, hiss, and no skipping. The original inner sleeve has album advertising from Coral records depicting some early artists which has that nostalgic touch and the original jacket is in excellent condition which is a plus. I received a good product from the seller at a good price and I am totally satisfied. You cannot go wrong here and I plan to order again. Fast service, fast shipping, great product. 5 STARS.......",P +"Beautiful music! The Living Strings was RCA's in house go to orchestra used on various music backgrounds. This spin off became RCA Camden and was considered an economy label for lesser known titles. This is a very International recording due to the fact that RCA's orchestra was from NY playing music from Hawaii and the album was recorded in England in 1970. Now all that being said, let's get to the music. This is the most beautiful Hawaiian album that I have ever heard. It is easy listening, relaxing and romantic. The stereo separation is fantastic and makes the Hawaiian steel guitars really showcase the album. Just put it on, sit back, close your eyes and enjoy a nice trip to the islands. Just key on the sellers and read the info that they provide and then you can decide. There are several who sell good used albums. My copy was pristine in all respects and can't tell it from a new one. Well worth every penny. 5 STARS",P +The vocals on this cd are the best. The audience clapping frequently during the performance is a distraction. However it was a recorded live performance. It was like being in Carnegie Hall.,U +There are so many contemporary Christian musicians who write music with two or three lines of lame lyrics and repeat them throughout the song and count that as talent. This album combines meaningful lyrics with great Irish music. All the lyrics are included as well so you can sing along when no one else is home :-) .,I +"Amazon put cd in with a radio ordered. CD case was totally destroyed during shipping, cd plays just fine. Love George Strait. I put the CD's in a couple of empty cases I had on hand. I am keeping the CD since it luckily was OK.",_ +Love this cd! Takes me back to my youth.,x +"This is an audio CD, NOT a DVD. But, it can come in really handy. I play this CD behind me on my boombox (you remember those!) while I do Leslie Sansone workouts. It gives me such a boost and I don't experience the boredom I usually do with her music. I'm 52 and many of these songs are from my college years or early adulthood, so I really enjoy them.",a +Average,D +Love the quality and music!!!,G +Careless child destroyed the original which was part of a set. This item makes the set whole again !!!! Yeah !!!,a +"NICE LITTLE BRIDGE ..It is not a roller bridge it is old school gibson style style, but pretty well made! The only inexpensive bridge that comes with bushing springs. Problem for me is my hagstrom uses a 52.4mm and this one is between 47 & 48mm and i think that would cause intonation problems,, so since this is a very good bridge i will hold on to it for use another day.. Sorry i could not comment on the sound, But the the workmanship is good well made piece of machinery.",E +This is a great CD and I have waited on it ever since they were on The Voice. They have an amazing harmony and I love their playing and singing together. Hope to hear a lot more from these two and hear them often on the radio as people hear and learn more about them. They are a really great brother duo!,B +I've been a fan of Windham Hill artists since the mid-90's. This is my favorite compilation by these fine musicians. So why is this not available on Amazon Music?,` +This guy wants a grand...lmao,q +A friend told me about this cd so I finally ordered it. I absolutely love it .,A +LOCED this 20-years ago when my son was little. Still love it to give as a gift for the next generation.,_ +Such a blessing!,w +Anything Beegie Adair is quality light jazz piano. I have several of her CD's & wake up & go to sleep to her music. Highly recommend any of her CD's.,_ +"Not necessarily my type of music, but my sister loves it!",C +scratches,d +I’ve Loved Moody Blues for 40+ Years Finally Bought an album,H +Love The CD Awesome Music,] +Exactly What I Wanted,u +The product was in good shape just as they said it would be. The music took me back to a good / sad time in my life,f +Incredible worship music cd. Touches God's Throne.,E +"Kidjo is a favorite. Upbeat, stimulating, enjoyable.",_ +BEST Series ever!!! Five Stars*****,c +This is the best series ever! Love everyone of the CD's. Kink only put these out every other year. Started in the late 80's till mid 2000's. Don't miss a single one.,r +Love all the CD's in this series.,i +"Didn't know other VI tunes other than "ICE ICE BABY". Others are ok, not great.",p +"the stories and songs were an important part of my childhood, love it",j +these stories and songs were an important part of my life,` +Love it,Q +I love is cd,V +Well there's only 3 songs I really like .on this CD,b +"Tremendous music, that glorifies God and edifies the soul. Very moving and worshipful.",I +"Soundtrack from the series. Beautiful, soft, unique music.Very soothing and calming.",G +"If you like the Moody Blues, you will treasure this performance.The actor,and watching and waiting,I could go on,they are all top notch.",S +I like that all of the songs are sung by the people who first made these melodies popular in the first place.,^ +"Another great album from Taylor. Her music is fun, clean, and offers many opportunities to sing loud while dance driving.",T +stated cd received cassette tape,\ +I bought a relative a new CD and cassette player and this CD I put in the box with the gift and she absolutely adores it of course Andrea Bocelli is one of the greatest vocalist I have ever heard. He is so good so if you haven’t tried a CD from Andrea Bocelli what are you waiting for beautiful music.,Q +I gave this cd to a relative and they love love it!,s +"I got it but didn,t open it becuse it is for chirstmas",E +"Funny,silly songs : ) I grew up listening to this album and it still makes me smile every time I hear it!",s +Really adds to my collection.,[ +"The haunting beauty of these songs really hits home. The artist does a great job of expressing how Inuit people are as mistreated in Canada as US Native Americans are. Mixing of the languages provides a neatly done effect. Even those who don’t speak her native language can understand the basics.The beauty of her songs and voice makes me hit replay over and over.There’s also a really wonderful mix of painful and joyous emotions in the songs that if you cannot grok to it, well...your loss I guess",t +Listen while you work :),W +"I love Ronnie Dunn's voice but this cd had too many of other people's songs on it. I would love to have some of his songs from the past, even some with Brooks. They were a great duo. But I am enjoying the cd.",C +The only song on the gold cd that I want is the shoop shoop song. Over produced on both.,Q +Blessed are and farewell Angelina are much better. This one not really worth it,_ +Beautiful music,_ +Vacations are more enjoyable by playing music as you go andthis CD is full of familiar tunes with the words included on a fold-out page. So keep the gang's spirits up by singing.,^ +Great CD. Perfect. Has most of his greatest hits. Great sound. His voice is clear and perfect. Original recordings!Outstanding!,P +"Excellent performance, excellent sound quality.",_ +"Two CD's in one package, for an excellent price. Buddy's stuff always the BEST! Cannot go wrong when buying anything with his name on it!",m +this gave Mia bragging rights!,I +i like this kind of music -owens,L +well i hear the songs so it works on blue ray player,h +"Great CD, for all true Allman Bros fans. Made me feel that I was back at the Fillmore East listening to Duane and the band playing live. Not a studio album or professionally recorded but a kid with an old fashioned tape recorder so don't expect, ""Eat a Peach"" etc. Important historical recording for your Allman Bros collection, especially if you never had a chance to hear the great Duane Allman sing/play live. Heartily recommend.",T +If you like The Inspirations you will love this CD!!,k +SO PERFECT! Mine came with a gift and two pieces of candy. <3It came VERY fast. Like five days earlier than expected.THANK YOU <3,Z +Wonderful music!,\ +"I had this on cassette tape many years ago and it has always been my favorite. I was very happy when we found it on cd. Kenny sings very trational and his voice is amazing. It just isn't Christmas in our house unless I can hear Kenny Roger's Carol of the bells. This is a must to add to your Christmas music, it's right up there with Bing and Perry.",M +An amazing album an amazing band hugely recommended stop.,n +Excellent!,Z +"The music from this movie is fabulous. However, the piece "Mountain Dance' also by Dave G., is not part of this CD. Mountain Dance is played right at the beginning of the movie and played again when she is 'changing clothes'. A bit disappointing; however, the music is lovely.",_ +"Zbigniew Presiner is a favorite composer of mine. This is a Christmas album, but, for me, since it is in Polish, I do play is all year. Well, I play a lot of Christmas all year, but this is 1 I like. He is a very talented composer & his music is melodic & soothing on this cd.",p +"IF YOU LOVE ENGELBERT, YOU WILL ENJOY THIS CD.A WELCOME ADDITION TO THE OTHER CDS YOU MAY HAVE OF HIS.",m +The first note of Iz's singing took my breath away. I listen to this tape over and over while commuting. I arrive at my destination relaxed in spite of the traffic.,^ +"I purchased two of these, one for my sister and one for me. My sister had an injury and this was purchased to comfort her. She played the flute in school and was first chair. It's a fantastic album!",H +"Gerald was not only a superior representation of Abraham Lincoln but he insisted on only playing Patriotic music and Hymns that honored God. This is probably his best CD. I knew Gerald personally. It was I who helped him launch his career as an Abraham Lincoln Presenter. He gave me enough newspaper articles to fill a scrapbook. He also gave me all the music he recorded. This one is a must to add to your collection of Patriotic and Christian music. Gerald went to be with our Lord and Savior this past spring. I am proud to own his recordings. photos, and news articles.",W +My three year old grandson loves this,V +"Love it! Everything was packed very well and arrived without any bumps or bends. I am super pleased considering I luckily got my favorite, Jackson! The poster is on nice thick paper and I got the little Mark photocard. Long live Markson:) only downside is I can't read hangul in the lyric sections :(",c +Received a special edition which was still sealed. I am very please with quality and packaging.,y +DVD ONLY; no CD,i +Album came as described - and we are very pleased to have this vintage album that brings back such great childhood memories!,J +"I bought this for a gift for my daughter. She has been collecting the Christmas Story Village pieces. I was quite disappointed in the price though. I ordered it and by the time the shipping was added, it ended up costing me $72.00. That was much more than anticipated. Quality good. Looks very nice with the rest of her village.",J +"Another beautiful collection of meditations from Rupert Spira! Listening to these "yoga meditations" really does help you lose the feeling of being a separate person located in a body. He uses interesting visualizations, like imagining the breath coming not just through your nose but through the entire surface of the body -- and you understand in a physical way that there's really no boundary between yourself and the space around you. I highly recommend this set to anyone who (like me) loves the teaching of nonduality but needs help with experiencing it as a reality.Unlike the other reviewer, I had no trouble loading the mp3s onto my phone, so the CDs obviously aren't copyright protected.",x +"A great work of music. It won't suit the tastes of everyone, but it's a bit of history and you can feel it. If you are a true music lover, you will want to own this CD set. And while I don't get anything for saying this, you can find this album at amchitka-concert, and not pay $150.00. All proceeds to Greenpeace.",o +"nice, relaxing music",e +Wonderful trombone sounds!,G +"Loved this program series and especially liked the music associated with it. The CD has all the cuts from the program sung by original artist. The insert has photos of the actors/actresses and lyrics (in Hangeul-Korean language -only) in the insert. If you liked the music from the series, you will want to have this CD",M +"Concertino de Camera is so mesmerizing . I play it when I just want to relax and chill out, and that is Daily! I find myself playing this piece everday in the car on my way to work. It's like medication.",u +I probably own all of Michael Lington's music and I have to say he did his best work on "Second Nature".,r +DO NOT BUY. You're paying for a fake item. It's a fake vinyl.,k +"Damn bottlenosed molesters!No seriously. I found this CD the first time I heard it in a bargin bin for $.88 and it was the best not dollar I ever spent. After the CD broke I had to rebuy it.Just... amazing.You need to have a specific sense of humor, but if it's what you're into you'll be belly laughing for hours.",c +"This pillow has the perfect support for me. I sleep mostly ony back and sometimes on my side. In either sleeping position the pillow works great. I love how your head sinks in the middle and has support on each side. I used to have to use my hands or lay the edge on top of another pillow to try to get this support, now it's just in the pillow. I couldn't be happier with my purchase!",[ +"Always enjoy these songs, likable by most",c +"The album came, B version, with the photobook, photocards, poster, CD etc. Everything came as listed in product description. High quality. My sister and I are pleased.",G +"The album came, B version, with the photobook, photocards, poster, CD etc. Everything came as listed in product description. High quality. My sister and I are pleased.",E +Enjoyed my purchase,t +LOVE MY GIRLS !!!!,r +"Love westlife, hard to find cd's,so this is appreciated. Thank you",U +Good album!,[ +I love the collection of songs and glad I bought it. It has a wide variety of different songs that are all great.,G +"I had sort of forgotten (HOW COULD I??) these great songs, but what a great gift to myself this is! I was dancing and singing along from the first to the last song!!!",R +A great collection for my Taeyeon's album!,V +This CD is a great collection of Donald Fagen at work. Such a talent.,a +This is an excellent audio book--I recommend it highly to anyone who wants to understand more about how the food industry really works.,x +We ordered this for a cardio drumming great beat,r +Another live performance that is worth a listen but not well done.,P +Wide variety but limited hits,m +Elvis sings a wonderful medley of great gospel song. What is not to like.,V +"Okay, the main reason I bought this 5-disc live set (Kansas has a bunch of live recordings) was for the 1982 recording with John Elefante on lead vocals. There are no other recordings with him on vocals, and he has a fantastic voice.First the negative: The 1977 and 1980 discs have poor (but not overly poor) recording quality. ""Two for the Show"" is a much better recording of the ""Point of no Return"" tour. The 1980 recording has many songs from ""Audio Visions"" that have not been heard live before.The 1976 recording is good, and it has much of the old stuff that you would want to hear live. That leaves the 1982 ""Vinyl Confessions"" tour. (I got to see the ""Drastic Measures"" tour, and I still want to have a live recording of that one!). The 1982 disc sounds the best of all. ""Yes, but it's not Steve Walsh,"" you may say. And I will tell you, when Elefante sings the Walsh songs, he sounds fantastic! I have heard about 7 live versions of Carry on Wayward Son, and they just don't sound good. Elefante kills it on this one! It sounds great! He also does a great job with No One Together, Hold On, and Portrait (He Knew).Buy this collection for the 1982 recording.",W +"This is an extremely raw recording. It's not bad, but it's not great either. The background vocals are off key at times. I've seen Journey 3 times--twice with Perry and once with Arnel. Arnel sounds much better live. There are some highlights here though: Solid recordings of Rubicon, Edge of the Blade, and an okay version of No More Lies from Schon's solo album. One last note. This is the "Frontiers" tour. Where is Separate Ways, the biggest hit from that album?",g +"Love this magazine and this particular issue has the cutest embossing folder, dies and paper. These will make great Christmas cards for next year.",O +"great live album, spirit filled",g +Cd worked great,` +Plastic was not on the case when I received the CD but it works fine.,L +Great group. I have been following them for many years.,f +"This is a great group. I really enjoy their music. Twenty years ago I visited Halifax. I heard this group singing. I contacted their managed, trying to get a CD. He did send me several from other groups but unfortunately, this group did not have one. As the years past I still remembered how much I had enjoyed them. A few months ago I thought I would google them. I was surprised to find that they now had become more well known. I ordered all I could by them and am enjoying all of their music very much.",w +Delivered intact and plays well thank you,n +Excellent condition and delivered on time,f +"I’v Been waiting for decades for IN A ROMAN MOOD to be available on CD. A few years ago I burned my 35 year old LP to CD but couldn’t get rid of the snaps, crackles and pops. The sound of the CD is excellent and I can’t tell you how happy I am to finally have a good copy of this album! Keep up the good work.",Q +Nice mix,y +"Excellent collection of Polish 80's pop by Urszula. Considering the setting & time--a country behind the Iron Curtain, under a military dictatorship since '81, US sanctions imposed--this is top notch music. Easily comparable to Kim Wilde or Pat Benetar with more synths thrown in.",u +"Great concert, all DMB live trax are with the money if you are a fan. Next best thing to being there",v +"Fast shipping, Girlfriend loves it!",W +Can’t believe I found this!!! My husband loved this band back in the day. He was in shock that I found this CD for him since he can’t find streaming music for them anywhere.,s +Love it !,R +Every major Piano rendition for the 4 the Artists represented. It's worth the purchase.,E +He's a fair/good singer.,n +Just only two songs but with excellent sound and quality.,q +So glad Roy made his 4th album. He puts his heart and soul into every song.,o +Great collection of live music. Nicely recorded.,D +I read somewhere that there were only 500 copies printed. It took me years to find the album and it's one of my all-time favorites.,u +Just delightful... Great service too,o +"I have seen all four in person. It was a joy to have PJ give her Dad's entry, I was not prepared for it and old mourning surfaced but recovery was very quick. It is a treasure to have available many songs that I have missed hearing and re-listening to favorites as I drive on the long distances treks I enjoy doing. The open highway through these lovely lands in the US with the Bee Gees along for the ride. Thank you for featuring this in your collection for sale.",B +the best,s +"Beautiful music, beautiful voice.",x +Beautiful music,C +Not what I expected.,Q +Not what l expected.,o +Nice CD.,H +"I remember buying this album when I was in my early 20s, back in the 70s. I was broke so the price alone was an incentive to buy it (I'm thinking around $2-3). Of course, there were a few musicians I was familiar with, as well. Much to my surprise I think I enjoyed every cut. I believe there was a second album put out that I purchased and it was also very good. I've always wished I could find both albums recorded as a CD.",W +So glad I found this. I love the music from this movie and it's not what I could find on the soundtrack itself.,I +Very motivating terrific music.,I +Great music for workouts. Used this when it first came out am planning on getting back into shape,l +Great instrumental classics (mostly country) from the master of stringed instruments. Roy also explains the difference between an instrumental and a vocal.,x +Get it. Just...get it! Those who say that Christianity and science don't go together may find this to be interesting.,i +"Symphony No. 2 is Johannes Brahms's most lyrical symphony, and it has received a host of superb recordings. The present 1981 DG disc presents the account by Carlo Maria Giulini and the Los Angeles Philharmonic Orchestra.Giulini immediately conveys that his will be an intense, emotional interpretation but not at the expenses of the lyrical element. The 1st movement is typical of Giulini's approach. Forte tutti's are forceful but the softer passages are sublime. The 2nd movement is beautifully played and sweet as can be. Yet, Giulini always supports the interpretation with an intense undercurrent of emotion. The sublime is never far from the tension, and the final two minutes reveal the inner substance to be the juxtaposition of lyricism and strength. The 3rd movement is a walk, simple and innocent, then a serious, intense dance, but one of joy. The 4th movement begins softly with the glorious explosion of full orchestra confirming the intensity that joy can contain. I have heard some beautiful accounts of this symphony, such as the surprisingly robust recording of Celibidache, but I can't think of one that tops this in any meaningful way. The final minute of the movement is the greatest I have ever heard. It is truly astonishing!When you listen to Giulini's masterful account of this wonderful symphony, you get the feeling that this must be what the master intended. It feel that right. I would not complain if this was the only account I could ever listen to again. The sound is fully supportive of the majesty of the reading. Despite the 1981 recording date, this early digital recording bears few if any of the faults so common in recordings from that period. The brass may be the slightest bit grainy, but there is no early digital over-brightness. Soundstage and center fill are superb. This one is capable of evoking tears and that's what makes it magical. It easily gets my ABSOLUTE HIGHEST RECOMMENDATION!!!!!",l +"Eugene List was a pupil of Olga Samaroff at Juilliard who also taught Roslyn Tureck. He had an illustrious concert career but his recordings are hard to find, now. This Vanguard disc presents him in performances of the piano music of Louis Moreau Gottschalk. The first disc is devoted to two-hand piano music, while the second features one-piano four-hands and two-piano works.Disc 1 begins with one of Gottschalk’s most well-known works, The Banjo. While I have heard faster accounts, I have heard none with such rounded tone. Nor has the piano been recorded so well. If you have never heard The Banjo, you are in for a real treat. This is the finest performance of the virtuoso piece ever! I will mention a few other pieces of note beginning with Souvenir de Porto Rico which receives a caring performance and perfect idiomatic playing. This is a rather sultry piece, and List gives it plenty of sway. You won’t fail to be impressed. Bamboula is another of popular pieces. It is the forerunner to the cakewalk. The reading is luscious. The Last Hope is a mournful piece of great beauty. Tournament is a gallop that is just plain fun to hear.Disc 2 is comprised mostly of arrangements by List and Cary Lewis. They are, of course, performed with distinction, but I have little interest in arrangements as they are not the intent of the composer. Only Tremolo is the original Gottschalk version and it receives a superb performance by List and assistant. This is a very lively work with a wonderful theme. The four hands almost play as one.This is a superb album of Gottschalk piano music. The second disc is equally enjoyable but largely consisting of arrangements. The piano sound is excellent but sounded like a different instrument on Disc 2. Curious. VERY HIGHLY RECOMMENDED for Disc 1!!!",e +"From the high quality of the sonics, it is difficult to believe that Sir George Solti and the Chicago Symphony Orchestra, Vienna Philharmonic Orchestra, and London Symphony and Philharmonic Orchestras recorded these 19 Famous Overtures for Decca between 1959 and 1994. These are classic works most of which are known to serious and casual classical music lovers alike. What is even more surprising, however, is that the two from 1959, Suppe's Light Calvary and Pique Dame Overtures are two of the best sounding of the group. Both were performed by the VPO.Every overture on these two discs receives a committed, convincing, and exciting performance. I found none whose tempo was excessively fast or slow. The inconsistency of sound was in keeping with the range of recording dates, but earlier recording dates did not predict the quality of sound. This 2-disc album affords the listener a generous sampling of well-known overtures interpreted by a world-class conductor leading world-class orchestras with generally excellent sound. If the sound had been more consistent, I would have awarded 5-Stars. As it stands, the album is a safe recommendation!",b +"Frederic Chopin modeled his 24 Preludes after Bach's Well-Tempered Clavier. Before a concert, Chopin would seclude himself with a piano and play through the WTC for memory. Like Bach, Chopin wrote a Prelude in each major and minor key. Each one is a masterpiece with each its own prevailing mood. In the present album, Ivan Moravec performs the set of Preludes and the Ballade No. 4 on the Supraphon label, recorded in 1963.In Moravec's hands, the Preludes take on new lives. He plays them with a fresh approach. When you listen to them, you can hear him feeling his way through each of them. Take No. 6 in B minor, for example. If No. 15 is the ""Raindrop Prelude,"" then No. 6 should be the Teardrop Prelude. Its sublime melancholic melody is unparalleled in the piano literature. In contrast, No. 7 in A major is a sweet but brief smile played. No. 9 in E major is performed with serious majesty, but I wish it was less disjointed and more flowing. Moravec demonstrates his superb technique and light touch with No. 10. Unlike the interruptions of No. 9, the Prelude in G-sharp minor which can sound disjointed is flowing. Odd. No. 15 in D-flat major receives the finest performance I have heard. The emotion flows throughout but it never becomes thick and overly dramatic. I have always loved this Prelude, and Moravec more than any pianist conveys perfectly its exquisite and anguished feeling state. No. 20 in C minor was played at Chopin's funeral. It can sound ponderous in some hands but not in Moravec's. The F minor Ballade concludes the disc. It is given a heavenly performance by Moravec who conveys perfectly the intense moods of the work.This is the finest set of Preludes I have been privileged to hear. The treble of Moravec's piano can sound hard in loud passages but not so much that it diminishes the pure joy of his magnificent interpretations and overall performance. For me, this represents the definitive ""modern"" recording of the Preludes. I give it my ABSOLUTE HIGHEST RECOMMENDATION!!!!ADDENDUM: The 2009 reissue of this recording has tamed the aforementioned treble hardness. The reissue is now the top account of these Preludes. I reviewed it at http://www.amazon.com/Chopin-24-Preludes-Ballade-No4/dp/B000003521/ref=cm_cr-mr-title",[ +"When it comes to recordings of the complete Brahms symphonies, we really have an embarrassment of riches - superb accounts by Karajan, Klemperer, Dohnanyi, and Levine, to name those whom I consider the top-shelf Brahms conductors in the last 50 years. You can't go wrong if you chose a cycle by any of them. The question here is if Lorin Maazel should be added to those esteemed few? Maybe. Maazel conducts the Cleveland Symphony Orchestra in 1975 and 1976 recordings of the four symphonies plus the Academic Festival and Tragic Overtures, and the Haydn Variations.Maazel has never been considered a member of the elite circle of conductors, but he also seems never to turn in less than average performances, either. His Brahms cycle here begins with a superb reading of Symphony 1 that plays to both the profound and lyrical elements equally well. If it lacks the absolute nth degree of excitement, the emotional warmth of his account, as demonstrated in the dolce and animato theme in the 4th movement, may completely atone for the slight sin of omission. The Academic Festival Overture and Haydn Variations demonstrate a superb performance less hampered by slight sonic deficiencies.The Symphony 2 greatly benefits by Mazeel's great ability to convey the lyricism of a work. Symphony 2 has often been compared to Beethoven's Pastoral Symphony a linkage that Brahms would have been very touched by, since it was Beethoven's preeminence that intimidated Brahms and prevented him from bringing his first symphony out. The 3rd movement is sculpted superbly from the contrasting slow and agitated sections. The 4th movement with its long, winding primary and secondary themes and extensive development demonstrates Maazel's long view that maintains interest as Maazel carefully brings the movement to its smiling close.The Third Symphony is given a finely etched performance. In the 1st movement, Maazel once again Maazel seems to take the contrasts in such easy stride. The 2nd movement is shaped perfectly, and the 3rd movement's melancholic theme is treated with sublime tenderness. The final movement seems to lack that previously nth degree of excitement again. Brahms' Symphony No. 4 calls for passion, and Maazel responds passionately and intensely. The 2nd movement's opening horn sets the stage for successive increases in the number of horns participating in a musical meditation performed beautifully by the Cleveland brass. For me, the 4th movement has always seemed Brahms's least successful symphonic movement. And Maazel is unable to change that. The Tragic Overture that follows would have made a better 4th movement.The discs have superb sound for an analogue recording, but I wish the woodwinds would have been given a bit more brilliance by the engineers. As a whole, Maazel's Brahms symphony cycle is a winner but just not an award winner. Only a few issues keep him off the top rung of the ladder, but he is only a step away. Recommended with a few reservations regarding excitement/intensity and dynamics.",e +"There are in essence two types of wind music. The first type is comprised of Baroque brass music and transcriptions, especially by Gabrieli, and the purpose is for worship. The second type is comprised of military marches the purpose of which includes celebratory but principally for the purpose of keeping cadence for marching troops. This 2-disc album is of the second type and features Herbert von Karajan leading the Brass of the Berlin Philharmonic (plus some other wind instruments). Now, for the purpose of this review, please join me in suspending any political overtones there might be in some of these pieces and their possible use in the past by the German Army of WW II. I am speaking only to the music itself.If you love band, wind ensemble, or brass music, then you will love this album. Each march is superbly played. I am a veteran myself, and I frequently became aware of an irresistible urge to get up and start marching! A few were composed by Beethoven, Johann Strauss and Johann Strauss the Elder. One of my favorites is the Florentine March by Julius Fucik. I first heard it on a Frederick Fennell disc. The march that concludes the album is the well-known Radetzky March.The marches were recorded in 1966 and 1973, but the sonics are pristine. The recording venue for all was Berlin's Jesus-Christus-Kirche, so the recording has wonderful ambience. There are no real criticisms that can be levied against the music with the minor exception that Karajan was obviously never a marching soldier, because he strays occasionally from the strict tempo that military marches have at their very foundation. He is actually trying to make music! The nerve!These marches are performed superbly and the sound is stellar. It is absolutely enjoyable from beginning to end. This one gets my highest recommendation. Forward March!",E +"Discs like these really irritate me. These are wonderful accounts of the three tone poems on this disc which is a poorly-performed digital remaster of an earlier Supraphon disc. The Supraphon disc also included an excellent account of The Noonday Witch.The digital remastering has the tympani so poorly defined that you hear only a rumble. Also, it sounds as though the engineer decided to add some reverberation - TOO much.So what you get for your money is an over-reverberant edition of THREE tone poems ONLY. For better accounts, try Chalabala's original version. Even better, get the superb Kertesz account on Decca Eloquence that in addition includes the Carnival Overture, Scherzo capriccioso, Symphonic Variations, Hussite Overture, My Home Overture, Othello Overture, and In Nature's Realm Overture.",h +"This is an SACD compilation of 12 short works for organ, brass and timpani and arrangements of works for such ensembles. It is unique in two ways. First, the selections are not just blockbusters but also serious music. Second, you might assume that an SACD of Organ, Brass, and Timpani would be an astonishing disc. It is not.The Strauss work with which the album begins is not widely known but is nonetheless a work of art. Some of the selections are well-known but primarily among classical music lovers such as the Baroque period Overture from Handel's Royal Fireworks Music. Gabrieli, another Baroque composer, is well represented by several of his sensational Canzons. Bach selections include music from three of his Cantatas. Even an arrangement of Mussorgsky's The Great Gate of Kiev from his Pictures at an Exhibition is included to bring the final curtain down on the disc and the two people who bought it. In fact, I may be the only one!This disc suffers from selections that are not the most popular and sound that is good at best. I have heard few organs more poorly recorded. The organ is as unexciting as is most of the disc. If you are looking for spectacular sonics or selections fairly well-known outside serious classical music circles, look elsewhere. Two stars may be generous - and I'm a classical music lover.",H +"4 1/2 Stars. This is a Japanese edition and most of what's written is in Japanese and German. It is rather interesting that Igor Markevitch should conduct Mussorgsky's Pictures at an Exhibition as it concludes with the Great Gate of Kiev, the city in which he was born. Kiev is only the conclusion of a unique and thrilling account of Pictures - thrilling at least until the end.Markevitch is definitely his own man. To begin with, he is not afraid to take a retard in the concluding measures of an individual picture more than you usually hear with other conductors - and it makes you ask, ""Why don't all conductors do it?"" I found myself asking that question quite often. Markevitch's sense of contrasts and dynamics is extraordinary. It is as though you are hearing the work for the first time. This is a truly fresh and remarkably successful approach to an old warhorse.The Great Gate of Kiev, actually, was the least impressive of the pictures (I don't think Markevitch lived there long!). Frankly, I think that Ravel should have beefed up The Great Gate of Kiev more. (I want TWO orchestras AND a concert band to bring the piece to a close! Too much???) Seriously, I expected more from the final tutti section than is heard. The recording was made in 1973, and we have had many great recordings from the 1970's. I just don't have an explanation. The Great Gate of Kiev begins well, and you expect even better at the conclusion. Interpretively, Markevitch keeps a strict tempo just before the conclusion when you desperately want a retard. Downer. But at least the tempo isn't as schizophrenic as in Serebrier's Naxos disc of Stokowski's orchestration. Fantastic Great Gate of Kiev's are heard in Neeme Jarvi/CSO, Kuchar/Ukraine, Reiner/CSO, and my pick for the absolute best Picture - Ormandy/Philadelphia (the Japanese-produced disc).Sonics are stunning on this 24-bit processed disc. Each instrument is clearly heard and very natural-sounding whether it is a trumpet, violin, bassoon, flute, or bass. Yet you hear the ensemble. As mentioned above, only the Great Gate of Kiev proved to be less than I expected.So what you have here is a superb sonics with the most original interpretation of Pictures that I have ever heard. And does it ever work! Highly recommended (but get somebody else for The Great Gate of Kiev).",U +"This is not the cover art of the album I purchased, but the disc does appear to have originated in Japan as the majority of the print is in Japanese. I have purchased some splendid Japanese remasterings. This one isn't improved enough to justify the bother.The original EMI recording was remastered in 1992 as part of a several-volume set. I think that Japanese engineers successfully cleaned up the treble but were able to do less for the bass. The result is a distorted balance that favors the upper 2/3 of the frequency range with deep bass definition better defined but lacking in weight. The tympani are recessed but better defined than in the 1992 reissued set. The organ is well heard but poorly defined. The upper registers are really quite good.But let's face it: what makes this a classic is Kempe, who had amazing interpretive powers in Strauss. This is one I really wish I could have heard in person. Kempe took a perfect tempo for the work - not so slow that you feel lost and not so quick that you have little time to savor the scenery. Kempe's account with the Staatskapelle Dresden is very moving with a clear vision of what lies ahead. As such, the performance is highly recommended, but I would keep thing simple and stay with the domestic EMI release that I reviewed separately. Come on, EMI. Let's give this a state of the art remastering that it deserves!",w +"Ormandy and the Philadephia Orchestra made two recordings of Mussorgsky's Pictures at an Exhibition, one for Columbia (now, Sony) and a later one for RCA. I recently reviewed the RCA disc on Amazon. This is the earlier, prized account and it is easy to see why. It is phenomenal! Unfortunately, Sony released it on CD only in Japan. The good news is that you can still pick it up from sellers in Japan. Oh, and don't expect to read much of the trifold ""booklet"" that it comes with -- it's in Japanese except for the front.It is hard to believe that the pictures by artist, Hartmann, that served as the inspiration for this classic composition were simple little sketches! Ravel's orchestration of the original piano pieces remains magnificent, and this is the recording that gives you all of the excitement and color you could wish for. Ormandy gives us a well thought-out interpretation with great sound. Pacing is perfect for each picture. Dynamics are remarkable. Bass is prolific and well-defined. Instrument timbres are natural-sounding. Brass couldn't be more thrilling. Belying its age, background noise is nonexistent. Ormandy chose tempi that never dragged the music which flows like a meandering stream, sometimes a river. The soundstage is full but not very deep. Middle fill is fine. The perspective is rather forward. The trumpets blare with a raspy quality that is, frankly, is over the top and to the point of being uncomfortable to listen to at times. This is a problem that I have encountered before on Japanese discs. I can't help but wonder if there was some remastering to highlight the upper end. I have experienced this problems with Japanese remasterings, as well. Otherwise, the disc is fantastic and confirms why this performance is so highly touted.I have compared this disc to the French and Great Performances discs. I do think that the top end on the Japanese disc may have undergone additional remastering. I discuss this further in my review of the French disc, if you would like further comments on the issue.This is just an exceptional performance, and aside from the high recorded level of the trumpets, is the leader in the field in most every way. There are some that may approach it in some way or another, but none can top it!",H +"The 5th Symphony by Beethoven may be the most well-known of all symphonies ever composed. The first four notes have been envisioned as fate knocking at the door, the symbol for victory (dit-dit-dit dah is the Morse code for the letter ""V""), or as Leonard Bernstein once said ""three quarter notes and a half."" Whichever your interpretation, it is the most famous motif in the history of music.The current performance is powerful, majestic, and for some, plodding. Hey, it's Klemperer. While the tempo is slower than most in each movement and the 1960 recording is a bit harsh during loud passages, the interpretation is played with great conviction. But there are few who can match the absolute power and majesty of the 4th Movement. While I imagine that LVB would raise an eyebrow to the tempi, he would have been less likely to have found fault with Klemperer's handling of the final movement. It is simply glorious even at the slightly slower tempo chosen by the conductor. The final few measures are up to speed, celebratory, and place an exclamation mark on the effort.The Symphony No. 8 fares better with tempi but don't expect a light-footed reading. Hello. It's Klemperer. The Eighth was written while Beethoven was composing its numeric predecessor. It represents the pause that refreshes. No so here. The symphony receives a fine reading nonetheless, and it is less gargantuan in concept than Klemperer's Fifth.Klemperer's 5th is certainly stands in contrast to the white-hot reading of Carlos Klieber who commands 1st place in almost every reviewer's list. But is there room for another recording at the top? For as important a symphony as Beethoven's 5th, the answer must be yes. But Klemperer, as unique as his account is, won't be there for me. I would look elsewhere for a companion to Kleiber, maybe Szell.",T +"This is a very well-recorded Mahler First that is conducted by a man with three Mahler cycles under his belt. It is clear that Lorin Maazel has a special relationship with Mahler, as who else has conducted three complete cycles. So we should expect an exciting First. Right? Well, there are moments where that is true. With all of the Mahler Firsts out there, however, moments are not enough, for almost every recording has those moments. The bar must be higher or else we would include the majority of recordings. For a recording of this symphony to pass muster, most of each movement should be worth listening to in repeat hearings. Unfortunately, this recording is not one of the chosen few.The 4th Movement is the most successful of this recording and perhaps one of the finest of all recordings of the First Symphony. The other movements, however, are less successful and are marred by tempi that drag or many moments that drag. The orchestra is of course great and they have as much experience with Mahler as any around. So, don't think that the sound has any problems. It doesn't. The interpretation, however, does not vie for top honors. As great as the 4th Movement is, even it drags at times.In honor of the moments of joyful listening -- especially in the 4th Movement, I can only say "Thanks for the memories" as I file this one back in the drawer. There are other recordings that are contenders on all counts.",Z +Love the songs! Not quite as good of a Nebuchadnezzar as the original but still OK.,I +"This was used by two relatives of my during and after cancer treatment was was, by their description, very helpful, when they were initially quite skeptical of how useful it might be. There are many, many acute and chronic pain conditions that this could be so helpful with.",k +"I have used these CDs for several years now to shorten the amount of time it takes for me to get into the zone for writing creatively. They work. I put the track for creativity on endless repeat. In fact I have it loaded an old iPod whose sole purpose is to play this for me to write to. I can tune out the sounds of the household and focus on writing. Without it or with other meditative style tracks, I'm just not nearly as productive and don't get into the writing process nearly as quickly. I will not go without this CD in my writing toolbox.",h +"3 of the 14 discs in this collection did not play in multiple CD players and would either not play at all or skip entirely.CD cover art and programs full of misspelling and/or incorrect information on mulitple albums Box set was torn and I ordered as new. One CD had copied meta data so even though I was playing ""Please Please Me"" Album all the songs titles and album art displayed were from the ""Rubber Sole"" Album.I truly believe the is a fake reproduction of the box set because there are just so many errors throughout this Beatles Box Set.",c +Well made. Good singing. Inspiring. Useful. Helpful. I hope you try it and enjoy this album. You can find the songs on YouTube to try before purchasing. It was well worth what I paid for this.,Y +Feels conforting to listen to these CD''s I bought 3 and this one for STRESS is for my mother who is 82 years.I enjoy it too!,C +"I was thrilled to find a cd of "Promises, Promises"!",j +excellent!,p +Great CD!,S +"I'm not thrilled with the CD. The first two tracks seemed more about promoting the therapist and not about why I purchased the cd, which was fear. Mr Bowden suggested that the dentist was my friend. That was not my issue. My fear came from having my teeth drilled without novicane and now the drill sound is torture. I thought this cd would do something to calm that down. It did not. I have several great cd's on hypnotherapy. My favorite is by Melissa Rose. To each his own. I gave the CD to my dentist, maybe it will help someone.",q +"Loved this shamanic guided tour into my heart and into my wounds. Cd is not really long, but just long enough for me to get one issue and then journal and reflect on it.",\ +"If you are reading this, you should know that the music is now available as a package set: this CD and a DVD for a better price than is currently being offered for the CD alone! Check out Amazon's separate listing. Now for why you might want to make this your own."Cover" is the term for an artist performing a song either originally performed by or made a hit by another artist. This CD is all "covers" including Dion re-doing his own song, "The Wanderer." The liner notes explain why Dion has such veneration for the artists and these songs. There is a big element of respect in this....as there was when other artists such as The Rolling Stones, Elvis Presley, and The Beatles did "covers" as well. Some people actually prefer their versions of Walkin' the Dog, Hound Dog, and Twist and Shout....some actually think that these "covers" are the originals....but I digress.This is what comes through on each song: Dion respects the material; he doesn't go to great lengths to fake the vocal of the original artist; he absolutely respects the instrumentation; the sound quality and production values are superb; the notes are helpful; and, you may find yourself liking some of these better than the originals.Dion has been engaged as a creative artist for decades --- he isn't getting along by doing the "oldies circuit." He is a wide-ranging vocal talent and he has made this album both with love and with some of the best studio musicians available. If you like the Telecaster sound, you will really appreciate what he has put into each cut. Both the songs selected and the performances have now stood the test of time. I listen to this CD frequently and often find a phrase or a riff that strikes (all over again) as super pleasing. I hope you do, too.",L +"Great disc - Reflection Club sounds like lost recordings by Jethro Tull from the 70's. This is a superb production that has all the style and nuances of classic Jethro Tull. As noted the high resolution files on the DVD are PAL encoded and will NOT play in most U.S. players so buyer beware! Highly recommended for fans of Jethro Tull and if you can play the DVD, high resolution and surround sound enthusiasts.Message from seller Madvedge:Dear Customer,Unfortunately we have only now noticed that there are problems in America with the playback of our DVD from the Mediabook ""REFLECTION CLUB - Still Thick as a Brick"". This is due to the fact that the DVD uses the PAL method for color reproduction. American players often only use the NTSC method and cannot play PAL DVDs. We were not aware of this problem, since European players can usually play both methods without any problems nowadays. Therefore we assumed that this is also the case in the USA. Now we have been proven wrong.Therefore, we will have the DVD converted to NTSC. When this process is complete, we will send you a download link through this channel, where you can download the NTSC image of the DVD. You should be able to play this image with a software media player, such as the free VLC player, or burn it directly to a blank DVD.We hope this won't cause you too much trouble.",n +"I went to a concert at a Unity Church and heard The Harringtons and Cindy Lou Harrington and bought her CDs! Their harmony is so beautiful! I love how she sings! Her music includes many songs she and her daughters wrote. They are meaningful songs and are danceable songs with a strong beat, harmony (mentioned before) and uplifting. No dirges. I went to a women's college and we had harmony and beautiful sounds in our step singing with 350+ women on the steps singing without instruments. So I get into this sound of harmony and beauty.",v +"I like the cd ,it's good old bluegrass musicI WOULD RECOMMEND THIS CD",` +"It's ok, good for the price3",Y +"Love the many styles of the song, however, I still have yet, to find hers with the hip hop phrase. I was hoping this had it but it doesn't.",n +"This takes me back to my youth in the lakes region of New Hampshire and helps enhance memories of my grandparents. #2 is set to better known tunes. It is clearly superior to the first, and it is getting more and more difficult to find this, despite the fact that it is quite blissfully relaxing.",X +Takes me back to the 60’s - 70’s ... music of today doesn’t come close to this.,X +"Album is great, very few scratches, took a while to arrive from Korea but was packaged very well.",N +I love Got7. Their albums are always beautiful and pretty. It just so happened that this seller sold me a damaged product so I had to return it.,A +good music,s +The CD worked fine and we enjoyed it.,a +"Fabulous! Had a bad spot in my range from coughing due to the flu, and this CD put it back together. Very well thought out!",M +"I looked for a couple years for Stephen King's The Stand in an audiobook format. FINALLY!! The Stand was my introduction to Stephen King and I loved it! Still my favorite after all of those I have read by Stephen King (The Shining, a close 2nd!) My husband liked the movie version....me not so much. He had never read the book, so I faithfully kept checking with Amazon for several years and it paid off. My husband finally got to "read" The Stand, audio version, as he drove his big truck cross country.",S +Amazing cd. Quality of cd and case is too notch. No scratches or marks to complain about. Came brand spanking new.,\ +If you role with the RATTPACK or your a BOBBYSOXER get this amazing mixtape that marks the end of the Young Sinatra series.,b +"These guys were a great Genesis tribute band. When they played Genesis, it sounded just like them. I wish I could get a CD of them playing Genesis music.",` +RELIABLE,C +This video is perfect for all ages especially teens or young adults. Another excellent video from Louie Giglio. Worth watching several times!,s +"Every Christian should watch this video. It will encourage and bless you. Every one, at some point, will go through difficult times and this video is perfect for someone who is discouraged or needs to hear from the Lord. I ordered 4. I've sent them to family and loaned the rest. It's a video worth watching more than one time.",O +"The live CD is excellent but has annoying gaps between songs. Not sure why they did this. I edited mp3s with Audacity and that helps a lot. Otherwise, love everything by the Cranberries.",m +Fabulous music. Entracing exotic sweet. Idan is a talent.,d +Fabulous music. Entracing exotic sweet. Idan is a talent.,c +"Get this series, it will change your life.",\ +Everything HW. this compilation has a unique mix.....,[ +Excellent..... Great Services.....Blast furnace....,m +My husband and I both enjoyed this album. It's mellow music and it's great for easy relaxed listening and dinner music.,d +This has good musicians and proficient bluegrass but I found it while looking for Russell Smith (of Amazing Rhythm Aces fame) and the description of this cd indicated that he was the singer. There is a Russell Smith singing bass on a couple of songs but it sure doesn't sound like the Russell I was looking for. Bad research on my part. Still worth having if you're a bluegrass fan.,_ +Great record. Parents loved it!,Y +"Although I'm a huge Robert Palmer fan, his official live CDs leave me kind of cold. One of my favorite live Robert Palmer CDs is actually a bootleg called "Feel the Heat." "At The BBC" is a close second. No booklet and the liner notes consist of a track list.",J +very nice record,M +"We have the old sound track of Dark Side of the Moon. My husband know music and is in fact a composer in his own right. He listened to this trice so far and put his stamp of approval on it. It always was an excellent recording but this new one is wonderful, you can hear things you do not hear on the original.",` +Good falling asleep or Dinner music.,Z +"I really liked the movie in general but some of dialogue was unnatural or too formal. So as my title says, I try to accept it, even though I believe it needs improvement. A possible reason for Anakin and Padme having too much formal dialogue (as I would put it) is that they both grew up very fast.We see Anakin in Ep.1 beyond his years as a boy and taken away from his mom to start his Jedi training because he's the *Chosen One*. Big load on anyone's shoulders. Padme was queen and now senator. Can you say, no personal life? I say this because I like giving people the benefit of the doubt and I understand that movies should be watched with an open mind instead of often exagerated expectations.Someone else critized Natalie Portman's acting when she didn't react strongly to Anakin's slaughter of the sandpeople because it was horrible and over-exaggerated justice, which took him one big step towards the dark side. It would be believeable to say that Padme would think, in light of his mom's death and Anakin's anguish, they all deserved to die because they're a menace and a threat to peaceful people. On top of that, it wouldn't be appropriate to start scolding Anakin at that moment for killing all of them. She is a skilled negotiator and knows when to hold her tongue and when to speak.I get frustrated when people become harsh critics of decent movies because most of the time they fail to understand characters' motives, the movement of the plot, etc.. In this case, Ep.2, we have dialogue that was stiff at times and given to young actors who did the best they could with it. I'm sure Sir Alec Guiness would've done a better job with the same lines but he's got decades of life and job experience over Hayden. Cut the kid some slack and try to enjoy the movies people. Just assume that Anakin spends too much time with fixing machines to know how be smooth with the ladies. HAHA!Lastly, yeah Jar Jar was annoying but that doesn't mean he shouldn't exist! The Star Wars movies aren't a perfect universe where no annoying people ever exist. People should realize that you need to watch a movie not as a person in a theatre but as if you were a bystander in the story. Immersing yourself in the movie and not being critical allows you to enjoy the movie a lot more. Accept Jar Jar for what he is, an annoying character, but don't go wishing how the movie *should've* been. If everyone would do that you'd have a million different Star Wars movies. Accept them for what George wanted to portray and stop whining. :)",q +Really good hits from the past. Fun SING ALONG tunes from few years ago. Love it!💖,^ +"I have been huge fan of Joni Mitchell forever. Her words, her voice her sound, jus wonderful. Hits you right in your soul.",E +Came in almost perfect condition.,g +"Originally on a record in 1989. Recaptured on a disk now. These are classical music tracks that were ""perked up"" by Louis Clark with the Royal Philharmonic Orchestra. The end result is fantastic. Great original music with great enhancements.",Q +She is amazing!,B +"This album turned me on to Joe Budden. He is the most underrated rapper in the game. His lyrics are on par with anyone. He is so thought provoking and honestly, I listen to a song from this album everyday for like the past 5 months. Best songs are practically every one. Dumb out and Future are pretty extrodinary.",j +Dad loved it n item as described.,o +"Amazing read...confirmed my personal beliefs that we are already living on paradise here on earth, and it will be up to man kind to figure how to love one another in order to maintain this precious ecosystem and continue to flourish and develop as a people. I was a bit disappointed that Bruce did not share a bit more about his personal transformation, but overall an easy read for the non-scientific minded like me!Also I just picked up Suggestible You by Erik Vance which was mentioned in Bruce Lipton's book and this is starting out as an amazing read as well. If anyone has other suggested reading please pass them along",^ +Video not working,v +My wife is from China. She always liked this artist's music but is becoming more westernized and became bored with it very quickly.,y +"I'm not an avid "religious" music appreciator, and this is fairly religious-based music. That being said, this CD is one of the most beautiful albums I own. It's a great selection of songs, but far more importantly, these sisters have some of the greatest voices I have ever heard. All this, and the greatest 3 part harmony my ears have ever feasted upon. Did I mention the amazing recording quality? Buy this and you'll be a happy music lover. ...........(2015) 10 years later, I still second the motion. The recording quality is reason enough to buy this - if it's still available. Many of the songs are without peer.",G +Love me some SG!,w +"2012 SG, Suzuka's last year, great album.",Z +"Twinklestars are bomb! One of the best SG sub-units, and Please! Please! Please! is their best song.",r +"I'm afraid all of my Sakura Gakuin reviews are the same or similar, but it's just that all of their songs are so great and I love each and every one of them! If you like/love SG, then get this!",C +2010 SG FTW!,j +"BABYMETAL Rules, and IDZ is one of their best songs. Excellent addition to my BM collection!",k +"BABYMETAL Rules, and IDZ is one of their best songs. Excellent addition to my BM collection!",i +"BABYMETAL Rules, and IDZ is one of their best songs. Excellent addition to my BM collection!",i +In the early 70’s my sisters and I loved it when Tubular Bells was played on the radio. It was different from the other music played. We all distinctly remember deep bong like bells as part of the piece…it was magnificent. I have some expensive wind chimes that have that same sound although not exactly the right notes. Believe me three hundred dollar wind chimes are not something my budget usually allows but the sound we all loved was so similar that I indulged. When I found I could buy the CD from Amazon my sisters and I were all thrilled. We are all close to 70 years old and after fifty years we were so excited to hear one of our favorite tunes again. OMG…..the disappointment. I am sorry I spent the money. We agree Mike Olfield is a very talented man. If you grew up in a different generation then you will probably be satisfied. We were not!,B +"I'm a geezer, so my grandchildren tell me, but I like Alan Parson's Project. I also like The Wall and other music from 30+ years ago, but they released this album on Blu-ray and they did a great job. I have a 7.2.4 surround system and you wouldn't believe how great this album sounds. WOW!!",i +"It's still one real good album and one so so, but it replaces 2 old favorite cassettes on a budget.",R +GOOD DISC,y +Awesome,l +"Great CD, great price.",_ +great.,Q +Peaceful rendentions of these songs! Very enjoyable!,e +Did not work at all.,W +sound quality not good at all.,s +As expected,h +As expected,K +As expected,J +Good mix of artist with songs,p +"I love this. The material that the cover is made of gets dirty with fingerprints quite easily, but it is a good album",l +"Poor recording, home made copy of the recordings, as the ending of each song does not give proper time for the next one to begin. Poorly recorded. Songs are wonderful.",X +"I have been a fan of Cynthia Clawson since the 80's. Back then, her cassette tapes(!) were easy to find and she was easy to find performing in various venues around Texas; where I was in college. I think this is the best recording she's made. The music is beautifully arranged, and the choice of hymns is great.",Y +"He is very entertaining and systematic. I lowered to 3 stars because of his essentialistic Eurocentric argument that Western music has essentially universal appeal simply because of its qualities and not at all because of political and military domination. Seriously, man. Read a book. Or stick to the topic. I don't listen to lectures to hear your 'gut' feelings.",t +LOVE THIS RECORD. Such a cute story.,X +"We homeschooled our children and these cd's were one of the ways we incorporated scripture memory into our day.We now have grandchildren who are beginning their homeschool journey, so I bought this for my daughter and the moment she opened the box, she shouted, "I remember these, I wanted to get them for the kids!" She then proceeded to sing some of the songs/scriptures she had memorized 20+ years ago.Songs are a great way to memorize scripture and Steve Green's Hide 'Em in Your Heart is perfect for the job!",[ +"I like chuck Berry music,great",F +great product.great music.,s +Like it very much.,y +"This album is very musically dynamic. I enjoyed the changes in "tone" as I moved from each track. There are also quite a few mid tempo ballads on this one, different from what I've heard on their other albums. Duet is my favorite mid tempo track. Some other favorites off of this album are Maximum, I Don't Know, and シアワセ色の花 (one of my all time favorite Toho songs).I really like the photos included in the album booklet too. Inside, I found a small yellow piece of film. I'm assuming the other versions include one as well. Just wondering what I'm supposed to do with it.This was definitely a worthwhile purchase. I recommend this album 100% to jpop lovers/Tohoshinki fans. We are T!-Chi",s +LOVE....Thank you Evanescence!,X +"ok, for those of you that are not familiar with 5nizza (the Russian word for ""Friday"") they are a Russian Reggae group. The sing in Russian with Jamaican accents. Basically its 2 guys, one of which play great jazzy/classical guitar, and they both sing. At the end of the cd is a cover of Bob Marleys ""Jammin"" which is spot on.... If you like laid back raggae/rap/aweseom jazz-classical music and you dont mind that you might not know what theyre singing about, then 5Nizza is for you. Its a great album to play in teh background at a party, or when your just hangin out with friends. I have absolutely nothing bad to say about this album except that it makes me wish i could speak Russian....",I +"This is a nice CD with music composed by a very young man that unfortunately stopped writing and composing for cancer. Buy this CD because you will support charity for testicular cancer awareness, testicular cancer is the most curable cancer if detected early..",g +Great collection of Funk. Sound quality is excellent and the choices for the CD are very good.,` +I just saw TobyMac in concert for the first time. This album has been on my wishlist for TWO years. I opted to buy it at the concert. Wow. Just about every song is something I was familiar with. It's an awesome collection!,h +Very good!,K +Enjoying it!,r +Very cool show but it's not the full show says in booklet.,e +wonderful cd. Karla Bonoff's music is sublime. I wish many more people would realize that than actually do. She should be way more famous than she is.,\ +"I happened upon these guys playing at a farmers market type of thing in the country... they were playing all native instruments, and it was, in a word, majestic.Stumbled upon this at Amazon while headed somwhere else.. seems an opportunity to give a little something back for the great experience ;^)",` +Very sweet songs. This has been helpful to prepare the grandchildren for bedtime.,L +Exactly what I expected and Fast,[ +"I had this cd years ago, and somehow lost it. I was so thrilled to be able to get it again.",N +This is like 15 years old now but has stood the test of time i.m.o.,J +"Beautiful music from a beautiful, soulful, gentle couple. Probably their best album. Again, a good reason to keep the turntable around.",w +"The Trinity Carol Roll (in the Trinity College Library at Cambridge) is a parchment scroll over six feet long and is the earliest source for English polyphonic carols. It is rare document, providing both text and music dating from the early 15th century. It includes a carol written in celebration of Henry V's victory at the Battle of Agincourt - written shortly after that famous battle.This recording brings that music to life. It is a beautiful performance by Alamire, one of the leading vocal consorts in the UK. I highly recommend this for the quality, the musicianship, and the amazing music. If you love history or choral music or the medieval era, you will love this recording. If you are looking for something just a little bit different to add to a collection of Christmas music, this is a great choice for that, too.",I +"Would someone post the song list please?I took one star off because I feel the sound man went way thin with effects... pretty raw. Also, by the way, you can hear a most incredible version of Johnny live (even though it's an amateur recording with a cheap cell phone) by entering ""Johnny Rivers Live In Beverly Hills - Poor Side Of Town"". at youtube. I wish he would cut that live version for us.",L +"This may be the Grass Roots ultimate performance, because after so many years of performing their songs they are bound the just keep improving, but then on this occasion they have the most awesome back up band ever. The brass section on this recording is tighter than any I have ever heard... EVER!! And you don't get a lot of wasted filler with audience applause either, as is the nuisance with most live albums IMO, the songs just start right off and end neatly. It's just too bad this CD is in such short supply. But if you can get a copy I do believe that you will have possession of the best of their best.",J +I can listen to this over and over its wonderful,u +I have the cassette! Great group of her hits!!! Love! Love! Love!,t +"I didn't realize that this was the entire actual shows, so I was a tiny bit disappointed that some things are repeated x 4. But after listening to it a few times, I found it charming. This is fabulous, vintage, real music from days past.",c +"I've tried a half dozen or more different record cleaning solutions over the years and this stuff is the best. Since I discovered L'art Du Son, it's all I use, with some minor caveats. As others have mentioned, for vinyl with heavy soiling or buildup, first cleaning with an enzymatic cleaning solution is very helpful. I like Audio Intelligent's solutions but there are others that I've heard work just as well. The L'art Du Son solution not only gets into the grooves, lifting out grime and debris very effectively, it also leaves the vinyl surface virtually static free. Another added bonus is that if gives off a pleasant mild floral aroma while you're cleaning with it.",j +"Love the music, love the artist.",q +I love Maroon 5 thank you Amazon for the Cd.,M +I heard John sing on TV and I couldn't believe he sang and just as well as speaking the word of God,Q +Awesome CD!,g +Great group of old time favorite jokes from Gene Tracy,A +excellent CD,D +"For those of you who may not know, Black Tulip is actually a label put out by Little Walter Devenne..the famous radio personality who currently resides in New Hampshire. Although he no longer does live broadcasts, his show are still syndicated, if you can find them in your particular neck of the woods.He is also in the production business & has produced a number of pieces, including several box sets. This particular piece, as has been noted, is bootlegged, and the artwork leaves something to be desired; however, the sound quality is good..and what I like about it is that is has the original MONO version of 1962's "Fat Boy", which has been released on several packages, but in what I call "fake" stereo.",w +love it ...how can you not?,[ +"My niece loved it. She wanted the CD, not the photo album.",r +The sound is good and clear and the 1st 3 songs are some of their best songs ever however it does sound very similar to the studio album versions of the songs even though it's live so keep that in mind but it is a great buy,s +Love the music and voices on this CD.,y +"This CD is getting me walking again. Years ago I had Praise Walk cassette versions and loved them. Obviously, they're outdated now... but I can put this on my iPod and cross train by power walking.",b +"Oddly, Maria’s First Rose is listed as Maria Elena. Still, a good CD.",O +"This is music for dimmed lights, a fire in the fireplace, a cigar, and a very old port. Poured from a Waterford crystal decanter.",L +"This is a nicely put together CD with recordings spanning 1943 to 1949. A discography is included. Kudos to Sabam Crescendo for keeping this music alive.I saw Benny Carter in concert around 1980 and it's amazing that even then he had 6 decades of jazz behind him. Truly an under appreciated giant talent.As well as his prowess on the saxophone, Benny Carter was a master arranger and all the tracks on this CD are rich with polyphony.Having said that, the arrangements are all ""hot"", with trumpets ablare, and generally up tempo. And Benny Carter takes too many solos. Give me Johnny Hodges any day, and the variation and subtlety of Count Basie's band from the same era.The best of both worlds would be Carter's arrangements for the Basie band, that is big band music without peer.",z +"Well, for Justin Timberlake I would say this CD is probably someone else's music that he is singing. There are a few songs that make you recognize the sound but the majority of the CD has a "older," "Jazzy" type quality to the music. I like the songs but it's seems like the majority of the songs were designed for behind closed doors intimacy between two people, not really dancing. #10 (Blue ocean floor) is just a "what??" song. Totally doesn't follow the flow of the CD. I think he could have done better, but I love JT so I can't talk bad.",T +It works,g +"My children, ages 6, 8 & 9 LOVED listening to this. The dramatic presentation is excellent, and the story is one that speaks to character development. It is engaging and keeps their attention well. As soon as they reached the end of the story, they begged to listen to it all again!",] +"I had a John Denver CD that was him, but not all of the songs were actually ""original"" versions. It was recorded for a TV movie about his life. I am a perfectionist about songs, and I recognized a couple of notes that were different, noting that they had obviously (to me) been re-recorded. This 3 CD set includes all 3 of his Greatest Hits CD's, and these beautiful and joyous songs ARE the original recordings. I am always surprised that so many people ignore John Denver, considering him as a lightweight. He was an early supporter of environmental causes, and spoke from the heart. I saw him twice in concert, and he was a delightful entertainer. And, if you listen to the words in his songs, he definitely spoke of non-middle of the road subjects. This set includes humor, love and songs about nature. I love it, and so will you.",Q +"Excellent gift, very well received. Arrived on time and packaged very well",W +I wish I had discovered this when it was first released. But late then never! My absolute favorite song on this album is Afraid.,L +Great item great deal,v +I love !!!!!!!!,_ +love it!!!!!!!!,l +Wife loves Hamilton. bought for her as a gift,p +These people are boring.,Y +"Great CD. Beautiful, insightful lyrics on a wide range of topics. High quality musical production with a backing band and backing vocals. Buddy is in the singer-songwriter genre. I've only seen him once in concert but it was an excellent concert, so I recommend going to see him. Great voice, high quality guitar playing and good stage presence.",s +"Amazing lyrics and her voice is one of the best I've heard anywhere. I think she must be an undiscovered talent because I fully expected to see a lot of reviews on here when I came to buy it/ I borrowed this CD from a family member and loved it so much I'm going to get it and the two other albums of hers (I don't know if they're as great as this, but I'm willing to risk it given the high return I got on this first one). It really is A level music. It's Irish flavored with appeal to a huge variety of people. If Amazon or some other site you find will let you download one song at a time, try ""The Dear Irish Boy"" - gourgeous lyrics and arrangements. Timeless. Most albums you buy have maybe 1 great song, 1 good song and trash after that. This I'd say has 3 great ones, 5-6 good ones, a couple so-so ones and I couple I didn't like. That put it in the top 1-2% of albums I hear and I'm a very picky listener and have listened to loads of music. Her voice and her timing/ability are excellent. Some have a great voice but don't know how to use it. Julee is a rare one with both. Hopefully she gets discovered enough so she can keep putting out great music.",` +In perfect condition and sounds great! Delivery took a little longer than usual but worth it!,T +yes its working i order the 2nd copy,j +"By relating research to the way our body and mind work, the author prompts the reader to look at what we believe, and why we believe, in new and creative ways. I love this book.I've heard him speak on this topic and it's a real treat.",k +"Absolutely wonderful set! 3 year old and 4 year old already singing the verses around the house! This is a must have for children, grandchildren, etc.!",A +"This album consists of nature sounds, from frogs to birds and bees and water. Sometimes there are a feww too many sounds -- more like you're in a weird jungle than a restful glen. But kind of nice for background when you're only half listening. No music",C +"If you are PROUD TO BE A JEW, get this CD! Fantastic and diverse selection of Hebrew and Yiddish music and songs. I cannot play it loud enough!",V +"I picked up a used copy of ATM's ""Unusual Moments"" this week after seven years of trying to find it. Never bought it at the local record store when they were popular locally. Band got good airtime on WCPR (same station that 3 Doors Down broke out on, but slightly later) in Biloxi in 2001. Oddly, the band had to change their name to ATM because of 3 Doors Down. ATM's old name... 2 Doors Down. Maybe and Hold On were the hits at the time. Matchbox 20ish, perhaps Train-like in some respects, with a little Collective Soul rolled in.For me the CD is a five star album, but only because I've been trying to find it forever. For everyone else, it isn't bad nor great, thus a three star rating.Good luck finding this one.Vocals: Jonalan LulayLead Guitar: Bill ""Slide"" ShanleyDrums: Dennis ChippsBass: Ryan StattonRhythm Guitar: Matt LaymanUPC-A code: 7 91022 20092 8",n +"Al Pitrelli and Tony Harnell should be magic and often, it is, but the problem with doing a disc of covers, especially well-known ones, is that it can very rapidly get boring and some of them, like the Queen track, are both not in need of a remake and done poorly here. I'm sure this was huge in the Japan market for which it was intended, but consider this more for completists only.",d +"Both heart warming and tears flowing watching this great talent knowing make his last stand - all of the members shared equally in the revenue, once again reminding us that if you were his friend, you were his friend. The sparkle was there in his eyes and voice, but unfortunately not in his fingers. Although missing his marvelous lead guitar fingers, his heart warms all the music.If you are are a Waylon fan - then get this without question.",m +So very funny!,Z +IF you know what songs are on it.,s +I thought this would be a fun way to help my son learn more sights words. However most of the cards are words that can easily be sounded out. Also there is only one of each letter so exploration outside of the cards is limited.,p +muff potter rocks! I love this album and would encourage anyone else to buy it also. Tip: for those unwilling/able to pay the import price It's available on amazon mp3 downloads.,n +"There are four pieces in this album. The first is the iconic ""Addicted to Love"" featuring the late Robert Palmer. As a tribute, the last is a remix of this by Mel Gaynor. In the middle of the sandwich are two additional selections by Gaynor. In my opinion the remix does suffer with the alteration of Palmer's more exotic beat--a very prominent factor in the original. However, it's still a great interpretation.",H +It was a great buy since I used to own the cassette tape when I was a little girl. Now the case did arrive cracked as well as broken on the bottom corner where the hinge is. But the CD was perfectly fine. But maybe the seller should wrap it in bubble wrap to ensure safety,\ +"I have been listening to New Blood Revival for several years. I only have some of their songs on a CD someone burned for me. If I could, though, I would buy every song they've ever recorded. They're unbelievable!I actually got to see them perform just before the band broke up. They sat down at our table and chatted for a while. Very cool, down-to-earth guys. I wish we could have hung out with them after the show but I was pregnant and in a constant state of exhaustion so I passed up the opportunity and went home instead. If only I had known they were on the verge of a break-up... I would have powered through and stayed!Based on the music I've heard, Matt has a humorously dark and demented mind. Whether he's singing about Jesus stealing his car, using his murdered wife's hair to make fishing flies, killing the guy who taunts him at the laundromat or Satan disguised as the family dog, you are sure to be entertained. Somehow, all those clever lyrics come together in a way that makes the songs so catchy they stick in your head for days.If anyone knows of a way to get more of Matt's/NBR's music, please let me know!!!",N +Great for a Muse fan..but! 2 songs I personally wished were on it..Undisclosed Desires and Feeling Good...,G +One of my all time favorite bands. Best Floyd CD to buy if your a big fan..nice combo of songs..songs you dont get to hear a whole lot and some popular ones.,x +Great band!!!,e +"I love all music ( except country) James , Joni CSN Beautiful soothing voice , and in so many songs can relate",z +I love this cd. I listen to it when I want a lift. The service was so fast. I love 50's music.,] +He has a fun way of talking that is great for the kids to listen and learn thing from him.,W +Nice. Thank you.,h +"Was in great condition, husband is loving it!",H +Missing this cd out of my collection.,K +Great addition to my collection.,K +"yeah, this album is a modern classic! one more year and its vintage! i think i got that right. anyway, its a must have for any music collection!",r +All the singer songwriters CDs are AMAZING !!! Great porch party music. Love them all !!!!,l +All the singer songwriters CDs are AMAZING !!! Great porch party music. Love them all !!!!,a +"At the age of 8, Nirvana changed my mind about music with ""Nevermind"". At 13 the world of alternative-rock possessed me when I 'heard Pixies' ""Doolittle"". When I' was 19, Guided By Voices shook me to the core when I 'finally heard ""Alien Lanes"". And now my twenties have been completely transformed by one of the best CD's I've discovered throughout my entire musical journey, ""Absolute Bliss""' by (the great &' mighty band) 'wedgepiece.Thank the heavens above for Scot Lade, the hard-hitting driving force behind wedgepiece, for crafting lyrics so universally genius that it's impossible not to be moved by one of the 18 songs featured on this hour long debut. My craving for radio-pop is satisfied with hits like ""The Stamp Collector"", ""Adina"" and ""Jesse James Canyon"", and Lade can only be credited as an indie messiah for penning such heartbreaking tunes as ""Lament"", ""A Million' Years"", ""Promised Land"" and the addictively catchy ""On Three"". ""The Ballad of Wedgepiece""' is a witty song addressing the acoustic nature behind the band, but also displays their obvious capability of rocking out. My favorite song on ""Absolute Bliss"", however, is without a doubt ""I'll Come Running"". A song about life, love, and pain, it sums up the message behind the CD:' even when everything around you is crumbling, wedgepiece can always make it better.The world will love wedgepiece, male or female. The raging guitars and powerful vocals drip testosterone, while the sensitive lyrics and buttery voice of Lade make hearts melt. I was lucky enough to catch a live wedgepiece performance in SW' Florida for their CD release earlier this year, and (besides the amazing show) it was clear that Scot Lade and his band are on a mission 'to wow the masses with one thin, shiny CD at a time. And let me be the first to say:' WOW!Be superior to all of your indie-blogging friends. Buy ""Absolute Bliss"" 'now before wedgepiece takes over the world!!!!",X +Loved it!,l +great product! this guy has a very interesting voice and style that isnt really my thing but that my boyfriend loves.,_ +"What can you say,it's the Stones",B +Awesome CD. I bought it for my partner and it ended up in my car playing on repeat. The case came a bit cracked but the disk works just fine.,[ +Took me back to the 60s.,M +"I'm not sure what happened but apparently all record of this album (with the exception of one live performance of Helicopters on youtube) has been wiped from digital existence. Crazy as it seems like that would have been a lot of work!! I was very happy to see 1 copy of this CD on amazon for 11$ and instantly jumped on it.It looks pretty much untouched/perfect condition & looks nearly brand new - I'm happy with the purchase. Strangely, it appears the only album left now is selling for almost 50$ which is nuts. This is a wonderful album, hopefully one day it will be available for all to hear again. I've no idea why it has been wiped from the internet! :(",B +I absolutely loved this cd. He knows how to tell a story!,l +excellent music,k +I haven't played it yet but it appears to be what I expected.,h +Day6 delivers as always,a +Got it for my boyfriend and he was very happy. He didn't open it yet because I think he just wants to collect good records.,b +"I am so disappointed! I finally get the CD, which came fast and I opened it to see the CD has only 2 songs on it! (Tonight I'm F***in' You and Tonight I'm Loving You) I don't remember seeing anything in the description that it was only 2 stinkin' songs on the CD. Very misleading!! I was thinking it was a full CD since it didn't say otherwise! Don't waste your money! Wait for the full CD to come out in the US!!",T +Such a solid uncommon- can really make for some nasty creature lockdown control and open up your opponent's defenses.,^ +Super fresh color.,W +Very rad looking pair of 2.5 cv. Translucent red.Limited Edition.2.5 tracking code is much "better" than original SL control codes- it does track better and bleed less,Q +Listen to it over and over and over again....,` +"I am a childcare provider and the kids loved the music, many of the songs the kdis knew, so they enjoying singing and dancing along.",L +I really enjoy this CD. My kids are loving all the songs. It's a fantastic way to write God's words on your heart:),F +"Very relaxing, soothing music, all instrumental and great for help in falling asleep or as an accompaniment for meditation. Probably will try some of his other CDs",C +"This is instrumental prog in its best. Influenced by Camel, Jethro Tull and even Crimson, and yet very original. Must be a blast to see live!",[ +I had the privelege of running lights and sound for Craig at several stops on his Behind The Veil tour. It was an experience I will never forget. The heart of God is evident in both the lyrics and the music on this CD set. These CDs will offer encouragement as well as an opportunity for self-reflection regarding your personal relationship with God. In short... awesome!,R +I have to rank Manny's rendition of this song the best among champion boxers followed by Smokin Joe Frazier singing 'My Way' and then Larry Holmes singing 'Brick House' both which can be found on youtube.,M +I never got a chance to get this cd when the band was around town. I listened to one of their songs that I forgot about and found this cd here for a great price.,j +"Their best album so far, don't like their new stuff",n +Super!,c +I need this CD for a party,I +Cy Walter is a wonderful pianist - great relaxing music to listen to.,W +Good,c +as advertised,E +nice,m +"Favorite cd ever, I love it so much, I love tasty and this album is my favorite, this album >>> people OMG I LOVE ITTT",\ +Great,L +Great,o +GREAT BUY!,h +Great music,^ +I love Rigos songs he was such a good singer and has a lot of good songs on this CD. I totally recomend this particular product very good quality of sound and selection on tracks,F +I gave it 5stars because it's a very good quality product. Im not too familiar with the songs but I wanted it for my collection since there is always guys complaining that I don't have manly CD with corridos and stuff. I guess they will like it can't wait to take it to the next gathering,o +Amazing selection of tracks a must have in any karaoke lover collection,m +I'm not a big fan of Jenny but I got it because I wanted to have a variety on my karaoke collection. It doesn't have may songs I know,c +It was hard for me to pick a karaoke from Chente because he has so so many good songs. I got this one because it has a lot of tracks tht I'm familiar with. Like "Urge" "Los Mandados" "La Ley del Monte" etc,c +"If you like great balads Karaokanta Vol10 Lupita, Amanda Y Dulce is an amazing 2cds to have in your collection. Great song selection. amazing quality.",E +I love this CD. Playing it often. Going with my daughters to The Kate in Old Saybrook on August 22nd to see a Tribute to Elton John.,[ +Know him and his family. All great people love his music,M +Such a great cd! Has everything I want to hear. Great quality,f +Fun adventure for kids. Classic Star Wars storytelling at it's best!,y +If you love Jimmy Eat World you'll love this band more. Humorous catchy tunes that'll make you sad knowing this band broke up after only one album. Oh what they could have been! At least we were left with this jewel after their demise.,C +"UPDATED Dec 6, 2017 FOR GRAMMATICAL AND STRUCTURAL REASONS. My views have significantly changed since the writing of this review, and I am a lot more Biblically grounded now. If you're not a Christian, do not concern yourself with that, just read my original review.ORIGINAL REVIEW 2001:Willa(Amanda) Ford, had one thing other female pop singers at the time didn't have, swagger and confidence. That is ultimately what sank her, that and mediocre material. She had a unique image, but it is too cartoonish to be taken seriously, and after traditional instrumentation and hard urban music took over after the release of this album, she could never even get close to the success of the single, "I Wanna Be Bad".This isn't the strongest debut, but it works as collection of self-pronounced egotistical fame hungry Amanda Ford's obsession with fame, predating Lady Gaga's debut by 7yrs. Willa was Officially been dubbed the "Bad Girl" in pop music. While not entirely true,but at least she only has one ballad on the this album.For throwaway pop music, this remains an oddity, and one of the more interesting and unlikely outputs of the early 2000's. While nothing original or spectacular, it is brash, rude, cocky, egotistical, and idiosyncratic, with blunt comparisons to artists she never had a hope of competing with.What is interesting is that Amanda co-wrote 9 of the 11 tracks here, including the crude " I Wanna Be Bad", while not crude enough to cause any rela controversy, it was crude enough to keep it from being a bigger hit.. Willa has more of a distinct personality, confidence, and prowress then most of her peers combined.This album is above average for the time it was released, and while it is by-product of it's time, it remains a highlight in a period littered with albums from faceless and personality-less female pop singers. Give Willa a try if you enjoy early 2000's pop. Just because she doesn't have Christina's power vocals doesn't mean she can't sing or that the music is bad for what it is. Willa has a distinctive, but average vocal range, which gives the sometimes mundane material color.God Bless ~Amy",C +Got this as a Christmas gift and it was just perfect. Everyone loved it and what an incredible gift it was!,a +"The music is great. The seller is the problem, selling this at that price is Ludacris, and they fail to even list the songs. Wow.",t +Was glad to get this sheet music. It is hard to find. Love the words and music.,d +"Beautiful, modern religious music.",_ +Our daughter loved it.,S +"Great ""generic"" music, coach provides verbal cues to prepare and push you through the workout to make the extra effort for heart rate, endurance, etc. I won't get bored with this audio for a long time -- but I bought CC 1-6 to be sure I have a repertoire :o)",L +great group,S +Received in great shape. Thanks!,i +Great album. Love Honeytree,l +Great album,t +"This music is as good as I remember it from when I was younger.a really great group. hard to find, but worth it.",S +"Very inspiring and very well recorded. Dr. Martinez speaks in a sure, soft voice.",N +You better get this. You wount be disappointed,r +Not impressed at all with sound quality,] +"wow. more everything than nothing. thoroughly beautiful, addictive. 2cds + bonus cd, inside photos. Also, I had forgotten how much better cds sound than streaming.",t +"Super music, and again as always with me, only for discerning ears....",s +"this is some crazy german dudes covering a bunch of freestyle/electro-funk classics from the 80's. they are well done and pay much respect to this short-lived genre of electronica+rap that people like fergie only butcher. pick this up or go for the original german version titled ""Electric City Of Music Instructor"" (this is the UK version).",K +"Too bad the radio itself crapped out post-warranty though. They'd fix it for $50 with a 90-day warranty, but I opted for a new radio (Lynx).",e +"Under Shade is an amazing band. I can't really place their style. They sound like they have some Phish influence but with much more depth and not so experimental. Under Shade doesn't experiment with their sound, they perfect a beautifully organized tune that you can't help moving to.It's refreshing to hear a band that has no trouble leaving the standard 4 - 4 beat pattern and yet is easy to follow.I was introduced to this band by my wife who went to school with the band members. I've since fell in love with the music and have added it to my short list of timeless albums that never get old.",A +"Love Mozart ... great compilation, great price!",w +Great relationship with her fans. I miss the subtitles for hard of hearing. Fearless had them.,d +"The obscure mixed with the well-known in a package that will please any fan (or make a new one of the novice) of the Big Band sound across the ocean. We Americans tend to think all the great music of that era came from us, but there were many excellent groups and performers in England, producing a similar but less jazzy music. Surprisingly good vocalists (boy do I want a complete set of Diana Miller records!) and toe-tapping arrangements make this music designed to lift the spirits and warm a chilly Depresssion era UK equally toasty today.A fine bargain too...two very full discs. Good performance info as well, though spartan packaging and notes (hey, whaddya expect?) But a one-stop education about between the wars British music.",^ +Fuego,N +I am an old (literally) fan and do not know off hand what cuts are on this album of wonder. I'll find out soon enou by listening but by way of any comment at all I would suggest Always list the individual cuts on the collection. The consecutive timing helps too.Sure I'd settle for a clean (louder) recording of this most excellent album no matter what else. i wonder who has my copy to this day?,x +He used to have it on vinyl and thought it was out of print. He really loves this CD. Great Buy,V +The music was too old for me. I'm a product of the 80's.,x +love it,w +"What can I say about Blackjack that hasn't been covered, possibly more eloquently than I could ever hope to write? I mean yes, we all know that BlackJack launched the career of Michael Bolton, but well, these songs.. they are certainly groovy. As a matter of fact this album has one long groove per side. That's vinyl for you. Yes sir. Blackjack. Even the name itself is two syllables. That is perfect. Why mess with perfect? That's what I say. The Blackjack would have been overkill, and BlackJack would have been a waste of a perfectly good capital J that could be used elsewhere. Buy this album, if it ever comes up for sale. You will then proudly be able to say, yes I made a serious purchase with no goading from a damn critic. Proudly say it loudly. However it's unlikely to happen since this album hasn't been issued in years. Maybe at a yardsale if you are lucky. Really lucky.",K +"Hands down my favorite album of all time. I bought an original copy on here back in 2011 for about $129, and I still have it. The high price tag was because there are supposedly only 1,000 official copies in existence. By "Official Copies", I mean that it's one of the ones printed back in 2003 when it was released. Not a pirated fake copy which any "seller" could download on numerous Torrent websites all around the web, print the labels on it, and sell it to you.The last song, "Not Alone" which is track 10, contains a 'hidden track' called "Just Sleep" which comes on after about 2 minutes of silence when "Not Alone" finishes - thus making track ten precisely 10 minutes (10:00) in duration. There are only 10 tracks on this album, as opposed to a much older review on here from 2008 featuring the track list which is incorrect. "Just Sleep" does not have it's own track number. "Not Alone" and "Just Sleep" are both in the same track on the album.If you ever see this album for sale anywhere and are a fan of Augustana or even newer bands such as "The Plain White T's", "Vega4", "Never Shout Never", or any other acoustic "Emo-rock" bands, I would highly recommend picking this up. It truly is an extremely rare piece of music. Especially now that Augustana broke up back in 2011, and now comprises only of the lead singer - Dan Layus. All the other guys have gone separate ways. That being said, you likely won't see it commercially available anywhere. But it truly is beautiful music. I love it!",j +great combo,s +great,a +great,A +great,x +great,T +great,W +great,H +excellent,l +I enjoyed this audio book. It arrived quickly and my mom liked it too. We finished it in 2 days.,m +What can you say? Best of his best.,B +"Love Dolly Parton, & love her music. My CD arrived in a timely manner. Tnaks so much!",c +"This tour is nothing more than an absolute disaster. I have no doubt that if fans had known that this show would be so poor before they bought tickets, I bet many people would have opted to skip it. Madonna tries way too hard this time to be ""hip"" and ""cool"" with sleazy costumes and dance moves, and it just looks embarassing - nobody wants to see a 50-something doing that stuff. Even worse, the setlist is terrible - filled with songs from her forgettable album ""Hard Candy"" and any of the older songs that she sings are butchered and remixed to the point where they are unrecognizeable. I don't know about you, but I hate remixes and I definitely hate remixes that make the song sound completely different from the original. Lastly, the ""choreography"" is lazy and cheesy and just a lot of bump and grind that anyone could do.I find it funny that Madonna's two best tours - Blonde Ambition and Re-Invention STILL have not been released properly on DVD. If they ever get released, I would recommend getting those instead.",n +60s music at its best !,H +"A recent reply to my initial review led me to revisit this cd. In comparison to the 1990 Capitol two-fer, listening carefully, the Hallmark reissue is way different and from an audiophile standpoint,different.. The Hallmark edition has a strange rechanneling and boost in the midrange which, upon first listening, may appear to have more impact, but I feel that a further remastering from the original multi-tracks would be warranted.",h +Amazing! This vinyl set is a real jewel. i'm so lucky i bought it.,l +Excellent condition...broadway edition. Personal entertainment,P +Duties to get from other vendors. The selection and delivery on cds is unparalled.,F +A superior recording as only Shaw could and the Chorale could do. Thanks.,M +"Great lyrics, it's amazing how this band they docked in a rythm song metal. The voice is amazing in the way she sang every song. Great album who glorified God with the gift of the music.",\ +TAYLOR is HOTTER than Hottest. Thanx for carrying a CD when everyone thinks CD's are done. GREAT JOB AMAZON,A +"A very nice set with very nice packaging (despite what one of the other reviewers said about flimsy packaging). Each cd comes individually packaged in a moderately sturdy, printed cardboard sleeve, color-coded by book (blue, red, green, yellow, purple, etc.) with a nice design on the front and a content summary with chapter titles and lengths on the back. They fit neatly into a very study cardboard case (a lid folds back). The packaging all has simple but nice artwork on it, and between each book set of dvds is a photo card with black and white pictures of the author and vocal artists. There is also a bonus cd included with a conversation between Stephen Fry and J.K. Rowling, which I thought was very nice to include and very interesting as well. In all, there are just over 100 cds.This reading is superbly done, with great expression & emotional inflections and separate voices & accents for each character. I've found that I would much rather listen to the book being read as I work or walk or drive than to ready the book myself or watch the movies. Although I know that some people prefer the more recent reading of HP by Jim Dale, but I honestly love Stephen Fry and his deep understanding of people, his humor, and his reading. He brings all of the accurate detail, humor and magic of the story to life in my head in a way that nothing else could (except his voice for Hermione is a bit high & falsetto, especially in the earlier books, which took some getting used to). I first heard of this set by borrowing a copy from the library, and I'm so happy I did! The cost of the product is of course very high, but it's honestly a steal when you consider the hours and hours of professional skill, read-aloud time, practice and preparation, and sound splicing & editing done to create such a smooth, seamless, animated re-reading of a wonderful book. It is truly remarkable.If you love Harry Potter but don't love reading or want something for long road trips for kids or yourself, this is a high treat and wise investment, and it's becoming increasingly harder to find. I suggest if you're waiting or wavering to secure a copy before it's too late.",U +"Great CD, has some awsome tracks.",U +"I listen to quite a bit of comedy, and this guy is absolutely hilarious. Great delivery and story-telling style that make you laugh so much along the way you forget if there is a punch line or not. Can't recommend this enough!Oh, and I wasn't able to get this through Amazon, but found a copy on the website Rooftop Comedy.",U +"I hadn't heard this album in a long time. It was nice to get to hear it again. After all the years, the album was in better shape than my hi-fi!",_ +"I can't believe that I'm the only person to ever review this cd on this site. How good is Lori Bell on her flute? One day the music God came down to earth and Heard Lori playing as a young girland was so thrilled that he intoned. ""Lori, I want to give a great gift to the world. I want to expand your magnificent musical soul and let all the world hear your gift."" Coming back to earth, I have to say that this woman is the greatest flutist I have ever heard, whether it be as a jazz musician or a classical player she has supreme mastery over this wonderful instrument. She is truly ""one with the instrument"" and there is no-one on this world that can play as well as she can. You will find more of her recorded gems on this site. Look particularly for her playing the music of Djavan. Whatever you hear you will be enriched for all time by this amazing talent. Whew.... what amazing breath control she has. Do not miss the thrill of hearing Lori Bell's musical gems. Thank you Lori, for sharing your gift with this humble student of music! With enormous respect and gratitude, Freddy from Brooklyn",t +"I’ve incorporated Karen Drucker’s music into my daily meditation practice, it is that pure and inspiring.",y +"I lived in Norway for a year, and the Scandinavian sour candy is the best bagged candy I've ever had. The stuff you get in the stores is even better...but living in the US now it's impossible to get, so this is a nice extension of that. This is expensive, but worth it if you want to enjoy sour candy that won't tear up your mouth. The funny part is based on the sugar tax in Norway, this price isn't that far off from what I paid living over there, including shipping over the Atlantic. One downside is the small bags, but that's all they have of this over there, and the quality is such that even for a relatively small bag, I don't have any issues.The candy itself is easily five star fantastic, but I removed a star because I ordered 3 bags and didn't get my % discount from the seller. It's a small amount of money so I'm not going to go through the hassle of wasting my time over a dollar or two, but it ruffles my feathers when sellers promise things like that and don't deliver. If that's a concern to you, I wouldn't trust that you'll get the discount. Just get the amount of bags you need and expect to pay full price.",^ +"After all these years since it first debuted, this has remained an extremely entertaining performance.",C +"Sure, he has a good voice, but I hoped to be hearing a gospel message. Anyone wanting to hear about prayer, Jesus' blood, etc. look elsewhere.",C +"Excellent and in English!I plan to read her book APPOINTMENT IN JERUSALEM again and again. There is so much spiritual nutrition in this book it is worth reading and re-reading. I was SO excited to learn that she also made audio testimony before she passed away! If it is no longer available on Amazon, check YouTube...!",z +great product,Q +"I know these rappers, bago, dboy, ect.. and there music is not that good to be paying what this seller is trying to sell it for, this specific cd is super old from 11 years ago, they dont even rap anymore, so beware",S +"I have had this CD about 1 year and used it twice. The first time to verify all the right sound levels and connections (5 speakers +sub and speaker functionality).The second time I used it to diagnose a subwoofer problem (intermittent) which I could never have found without it.Given the amount of money I (and many others) have in a sound system(s) I want to make sure it is working properly. Normal audio programming will not do this and even my amplifier test mode was not sufficient.It is a little hard to navigate initially but once you get the trick it is not difficult. Recognize all the tracks (82) are listed in the instructions but actually you will probably not use them all. The trick to navigating is to recognize to use the ""next/back"" track functions on your cd/dvd controller to quickly skip over tracks you don't care about. Look at the manual and count how many times you need to press your controller to get there.",T +"Excellent quality audio - good listening with 1950s Cisco Kid radio shows. Good sound effects, decent stories, and humor.",j +I like Greg's music,c +"This is one of the best albums I've ever purchased! I got this as a gift for my 18-year-old daughter on cassette, and I've bought multiple copies for my younger girls. They positively LOVE the music here! Thinking about an audio cassette, each "side" is one long continuous song. I like to put this on continuous play all night for my sleeping girls.",M +This album is fantastic!,v +Fabulous,p +Great,B +great dvd,x +"Drop dead gorgeous master magician, Criss Angel, may very well be Korn's hidden band member. Whaddya think Zakk?",c +"Guitarist Dizzy "Diz" Cortright of Nudeswirl."His guitars scream mood swings on acid! Hypnotic Alice in Wonderland astral tripping interlace with deep, dark, temperamental insanity. With wiry cat screeching intensity, his fusion of electrical overtones and undercurrents cause sparks to fly off the fretboard like a prized mad scientist in an absinthe lab. He has an earthy, gentle spirit that peeps through at times to balance his heavy seasoning of tasty riffs. A bit of a gear freak, this multi-faceted artist is well versed at finding just the right sound for his inner genius. His snake charming presence rings excitement. As he captivates his audience, like a sinister bird of prey on the pounce, his sharp talons pierce deep. There's no escape. His ultimate fantasy of entertaining bewitchment is fulfilled. Hear his roar in the big machine." - Karen St. ClaireFeatured Band: Press Release April 23, 2016"Nudeswirl" is a New Jersey Alternative-Metal Heavy Grunge Rock style band formed in 1989 signed to MegaForce records. Hailed by some of their most noted influences, Perry Farrell, John Frusciante, and The Edge, they toured with White Zombie, Danzig, Flotsam and Jetsam, Mindfunk, and performed on the 1992 Lollapalooza side stage. Videos for "F Sharp" and "Buffalo" appeared on MTV's Headbangers Ball, 120 Minutes, and Beavis and Butt-Head. Their combination of metal and grunge at the emergence of the height of the alternative movement is something that wouldn't be approached again by anyone else for many years. Considered a cult classic, this disbanded group is featured as one of the best obscure bands recently uncovered by Hollywood Undiscovered.",k +great music,F +Nice.,b +Love it.,B +"This band is a bit more straight forward punk mixed in with the pop elements rather than high energy amped up pop music that bands like Blink 182 ad Sum 41 popularized at apprx the same time this band made it big. This is one of their first CDs and the minimalist songwriting is more obvious here than it is in some of the later albums, although this band never did go full on open in their songwriting except during a brief period in the mid-2000's i.e.: COMING HOME. I don't expect Beethoven when hearing rock music, but I do expect the musical arrangements to be a bit more complex than this. Not a lot more. The songs are not outright bad. They just don't ever take off or gain any traction.",R +"Not a *bad* album. It's hard to find a digital/streaming edition that doesn't sound like garbage. The whole package rings hollow to me. It's got catchy rhythms and the echoes of good music running through it, but ultimately it pails very much to their debut. Their upcoming LOVE IS DEAD, so far, is much better (4 singles and only one of them can be criticized as sounding, to me at least, like background music), and has given me a renewed interest in this album. I used my sister's Amazon Prime free trial to pre-order the standard edition of this album. I never even opened it. I sold it to my local CD buyback headquarters (which is now defunct). I think sometime I might re-buy this if I can find a good price on it. I'm in no hurry.",Z +"This is not the same item that Mr. Peabody and his colleague wrote reviews for.Track list isLove is a stranger -- eurythmicsWhose Zoomin Who - Arethra FranklinNever Gona Give U Up - Rick AstlyWax - BridgetoHeartPerfect - Fairground AttractionIt just kinda goes on like that. Basically 16 songs that do not define the 80's at all. I like "Love Is A Stranger" (not a lot), and I love those two big hits that Rick Astley had (one of which is listed above) and I'd say it's a safe bet that the others, which I've never heard of, are good songs, but none of those are even close to being Ultimate. I don't mind buying this when I finally get some money. This item has been on Amazon for over a decade and it seems I'm the only one who bothered to notice it enough to say anything about it in public. Two people are selling it for rather low prices. I got my info from another website. The actual release date is 1995. The cover-art and the "wonder what's behind that door" has me at the throat. Yes, I know I can stream most if not all of these on Spotify for $0.00. I might do that before planking down a few bucks.",L +"I fell in love with ""Complicated"" when I was 18 years old in the spring of 2002. I listened to samples of LET GO, the then-upcoming debut CD which the song was from, via the 'net and for some reason that I can't quite comprehend, felt the other songs weren't worth squat and figured it was one of those CDs where there was one good song with 10 songs that were recorded to give the album the appearance of having a reason for its $15 price tag when a CD single with a $5 price tag would have sufficed for the consumer. Then in the fall of that year I heard ""Sk8er Boi"" and liked it a lot, looked over the album again, online, and was like ""my God, I need to buy this! What's wrong with me???!"" I spent the rest of the fall listening to the CD extensively, and then, after enjoying the CD which I paid $17 for when you factor the taxman, sold it to the very store I bought it from for $1. I'm insane, OK? Let's not make a big deal of it.So then a couple years later she's got UNDER MY SKIN. Good CD, doesn't capture the magic of my 18 year old self listening to LET GO in fall of 2002, but it ain't bad. ""Take Me Away"" and ""Together"" were the big highlights for me, the rest of the album was very listenable tho.Her third CD kinda threw me off in the fact that it lacked all of the introspection of her first two CDs. I learned to appreciate it for what it is, but nowadays, I find the production to be too muffled. If the music itself weren't so shallow, it wouldn't be that big of a deal. The Get Up Kids' SOMETHING TO WRITE HOME ABOUT had bad production values, but was a very memorable experience b/c of the music. But THE BEST DAAA THING didn't have the best songwriting or expressionism, so the songs sounding like they were recorded from the inside of a cardboard box sitting in the midst of a busy street is not something I go back to listen to often at all.Her follow up to that lackluster effort, titled GOODBYE LULLABY, was supposedly a deeper and more introspective Avril Lavigne, and I felt the songs were lacking in originality for one thing, and secondly, the songs were very somber, and, again, the production!: very muddy sounding. I swear I could hear this kind of ""swoosh"" in the background, like the album cover - featuring a piano stationed in a swamp w/ Avril sitting atop it - was not just an emotive image, but a warning: ""this album sounds like swamp water!"".This song reminds me of LET GO, personally. It's got the hooks, it's got the emotion, it's got the live-like-u-were-dying attitude of THE BEST DAMN THING and the freshness that should have been found on GOODBYE LULLABY. It's a great song. I'm not a big fan of her first single from her upcoming full length, but it's not too shabby either. It too is better than the songs on GOODBYE LULLABY and the production is superior to that of THE BEST DAMN THING. I might pre-order her self titled upcoming full length. I'm like dirt poor so chances are I won't, but I'll just have to see where the wind blows my money (haha).",V +They are great,L +My girls love these albums.,v +I Haven't really had time to go over it all. I am still trying to listen to it. not sure yet.,P +Just as described and fast shipping!,m +love it,[ +"This is, in my opinion, the single greatest recording of IL TROVATORE ever made, despite the somewhat muffled Italian TV broadcast sound. Why? For one, because Riccardo Muti follows Verdi’s directions explicitly, not only in tempos but in which high notes to allow and not to allow (there are no high notes at the end of “Di geloso amor” or “Di quella pira” because they are not written). But if that were all, I would equally recommend Muti’s 2000 performance, issued by EMI, which also follows the score exactly. The problem was that in the 2000 performance he had an old, wobby baritone (Leo Nucci) and a mezzo with a fluttery, overripe vibrato (Violeta Urmana), whereas in 1977 he had the services of Matteo Manuguerra and Fiorenza Cossotto.This is, in my view, Cossotto’s finest Azucena. Not only does she sing rather more dramatically here than in the past, but for once she sings the trills (most of them, anyway) in “Stride la vampa.” Carlo Cossutta was a good tenor who sometimes pushed the voice too far, as when he sang Otello, but his Manrico is right for him. Manuguerra, one of the greatest Verdi baritones of his time, rarely had the chance to record and, to the best of my knowledge, Gilda Cruz-Romo never made a commercial opera recording. Both are in super voice here, as is basso Augostino Ferrin who likewise sings his shakes in the opening scene. (Cossutta even takes a stab at the trills in “Ah si, ben mio.”)The downside of this recording is the muffled sound. Alas, nothing much can be done about this, because when you try to brighten it up the singers’ high notes begin to “blast” and break up. But I’ve heard and owned far worse-sounding recordings than this, and considering the extremely high quality of the singing and conducting it’s difficult to think of any other TROVATORE nearly as good. Cruz-Romo is especially brilliant, and not just vocally. In the last act, you can hear her heart breaking during the “Miserere” and “Tu vedrai che amore”: it’s quite an acting job, even finer than anything Callas did in this role. It’s the kind of performance that I find is “accumulative,” that is, as you listen to the whole thing in one long session you begin to appreciate what these superb artists were able to bring to the table and how Muti held them all together. Highly recommended.",[ +"Good old Amazon, listing reviews of three different “Rigolettos,” none of which is the recording featured on this page (Cotrubas-Domingo-Cappuccilli, Sutherland-Pavarotti-Miles, Popp-Aragall-Weikl). Well, let’s review the actual recording at hand, shall we?I was scouring the Internet to re-listen to Leonard Warren’s 1950 studio recording of “Rigoletto” when I accidentally tripped across this one. I admit having my doubts, as much as I loved Warren’s interpretation of the title role, because I know from bitter experience that both Bidu Sayão and Jussi Björling had a bad tendency to distort the music in live performances. I had also never heard any opera conducted by Cesare Sodero, and wondered if he would be better or worse than the miserable Emil Cooper and Fritz Stiedry who succeeded him at the Met. Imagine my surprise to discover an oustanding performance fairly bursting with drama!First and foremost, of course, is Warren in the title role. Here his voice is even fresher than it was in 1950 and he revels in the character. His “Cortigianni” is sung with the imagination and sensitivity of a great lieder singer: listen to the way he colors his tones, pulls back on the volume, then lunges forward. He begins his inventive with both anger and hurt in his voice, and ends with a pleading tone that will break your heart. He doesn’t have the technique to sing “Si, vendetta” with the rapid triplets, but he does something even more interesting: he starts it softly and slowly, gradually building up a head of steam as Gilda begins to realize what he is singing about and does her best to dissuade him. A very dramatically interesting moment.Sayão, for the most part, is surprisingly clean here. Granted, she occasionally indulges in her little sobs and whines which so mar her live Met performances, but in this instance they are at a minimum and, when done, they actually enhance the drama. Gilda is, after all, a 16-year-old, shelted young woman who is abducted and raped by the brutish Duke. The fatal flaw in her character is her inability to let the S.O.B. die when he should have. She sings but one false note, the very last one at the end of “Caro nome” where she goes badly flat; otherwise, her singing is musically accurate and surprisingly touching. One of her few great performances.Björling is Björling. His small voice, well recorded, fairly explodes with energy and brilliant color, and he impresses himself on you with tremendous vitality, but he—more so than Sayão—often indulges in holding notes longer than their written duration. It’s a small price to pay, however, for such an overwhelmingly committed performance.Sodero is a bit relaxed in places but overall conducts a well-paced performance. His reading of the storm scene in the last act is as powerful as Toscanini’s, and he elicits surprisingly fine playing from the generally third-rate Met Opera Orchestra and chorus of that era. A shame he died two years later at the age of 61, leaving the Italian wing of the Met open to the arrival of that stick-in-the-mud Stiedry.The two biggest drawbacks are, of course, the 1945 broadcast sound, which is clear but two-dimensional and somewhat “hissy,” and the fact that the Met not only allowed but encouraged cuts in the music. Not only are “E il sol dell’anima” and “Addio, addio” cut, so too are “Tutte le feste” and other moments. And of course, “Possenti amor mi chiamo” doesn’t even exist in this era.That being said, this is still the historic “Rigoletto” of choice—much better than any of the mono studio recordings made between 1928 and 1956 (yes, even better than the overrated Callas-Gobbi recording, mostly because Giuseppe di Stefano sings the rhythms wrong in the Act 1 Scene 1 finale and in “Parmi veder le lagrime”). My favorite recording of all time, however, is the DVD version with Paolo Gavanelli, Christine Schäfer, Marcelo Alvarez and conductor Edward Downes, but for those who know of it there is also another superb version made back in the 1970s with the vastly underrated soprano Margherita Rinaldi, Rolando Panerai and Franco Bonisolli.",y +"Since Amazon seems to have placed reviews here of performances by Bruno Walter and Karl Böhm, I thought it might be nice to have a review of this actual recording.This live performance of October 12, 1953 is in most ways not only superior to Furtwängler’s studio recording, begun the next day and made over a week, but also shows what is wrong with most other recordings and performances (including the Toscanini, Klemperer and Abbado versions). Not only does the conductor pace the opera effectively (though, being Furtwängler, not perfectly), but he knows how to BUILD DRAMA, not only with the orchestra but also with a superb cast of singers who not only have fine voices but also create CHARACTERS. Granted, Windgassen isn’t quite as fine a Florestan as Jon Vickers, but then, no one was…certainly not Julius Patzak, who bleats his way through the role on the highly-overrated 1950 performance. Martha Mödl is possibly the finest Leonore on records, to my mind even better than Flagstad in either her 1941 or 1950 performances, although Flagstad sings nobly and with commitment.In five places I found Furtwängler’s studio performance a bit better: the second half of the opening Jacquino-Marzelline duet, the Rocco-Pizarro duo, the “Abscheulischer” and prisoner’s chorus (Mödl is too far off-mike in the live performance) and the vocal portion (but not the orchestral introduction) of “Gott! Welch dunkel hier.” Otherwise, this live performance has it all over the studio version. Not only are Furtwängler’s tempos more consistent and less erratic (his fans well know that he had a bad habit of speeding up and slowing down the music at whim, for no particularly good reason), but in most cases there is more energy, feeling, and forward momentum in this live performance has an undercurrent of menace missing from most of the studio version. The sound quality is certainly not optimal but better overall than the RAI “Ring.” But by and large, this is a performance to obtain for the three most important principals, Leonore, Pizarro and Florestan, along with some wonderful conducting. I only gave it four stars because of the limited mono sound—good for its day, but not optimal. As an overall performance, I still prefer the Flagstad-Maison-Bruno Walter performance, but this one has Martha Mödl (and better sound) which gives it an edge to those who won’t be able to stand 1941 radio transcription disc sonics",y +"The year was 1956. The world was arguing the merits of Maria Callas vs. Renata Tebaldi - the power of dramatic interpretation vs. a beautiful voice - when an entirely new soprano burst on the scene. She was still very young, only 26 years old, and had a huge lirico spinto voice of enormous power and focus. She was also highly musical, meaning that she did not take liberties with tempo or stretching the line; she had an excellent technique for a voice so large, lacking only a cleanly defined trill. And she could drive audiences to the point of madness over the sound and fire in her voice. Her name was Anita Cerquetti.Like the young Callas, she was very heavy, topping the scales at about 250 lbs, and she wasn't very pretty. Yet all that was forgiven when she opened her mouth and sang. Within two years, Cerquetti became the darling of both Italian audiences and musicians, each of which admired her for different reasons. She lacked the almost demonic madness of Callas, yet still came within hailing range of the great Greek-American diva. And she sang an incredible range of roles, from Cherubini to verismo, all with impeccable taste and flawless execution. In 1958, after Callas' infamous ""Rome walkout"" in a production of ""Norma"" for which the management had hired no understudy, Cerquetti was subsequently rushed in to take over the role. She did so with aplomb, marking herself as one of the great singing-actresses of the age.By 1960, it was all over. At age 30, Cerquetti suffered a strange malady, paralysis of her vocal cords. Doctors were baffled in one so young, but ascribed it to over-singing. Cerquetti gracefully retired from the stage, leaving behind only two commercial recordings - an opera recital that showed her gifts to full advantage, and a complete recording of Ponchielli's ""La Gioconda"" that didn't - and several ""live"" recordings in which her powers were in full bloom. This recital draws from those sources, a 1958 recital in which she sang the aria from ""La Wally"" and ""O Re de Cieli"" from Spontini's ""Agnese di Hohenstaufen,"" one of her signature encore pieces, as well as several famous complete performances, among them Cherubini's ""Gli Abencerragi"" conducted by Giulini, Weber's ""Oberon"" (in Italian) conducted by Gui, Verdi's ""Ernani"" conducted by Mitropoulos, ""Forza del Destino"" conducted by Sanzogno, ""I Vespri Siciliani"" conducted by Mario Rossi. These were legendary performances, then; they remain vital and tremendously exciting today. No soprano since, not even Elena Suliotis or Carol Vaness or Karita Mattila, has quite captured the qualities that Cerquetti brought to the world. She was like Deborah Voigt in overdrive, just as big and beautiful a voice with all the passion and dramatic fire that Voigt so often lacks.Anita Cerquetti settled down, got married and had children after her early retirement. Five years later, her voice came back, but she had no proclivity to revive her career. How could she possibly have topped, or even equaled, what she had done? Better to let sleeping dogs lie and let her reputation rest on her youthful accomplishments. And what accomplishments they were. I defy you to listen to this recital and not be moved, thrilled, and appreciative of her great gifts. By the time you are in the cabaletta of ""Mare, superbo mare"" (""Ozean zu ungeheuer"" in Italian), your heart will be in your throat. Cerquetti had that kind of effect on her listeners. She grabbed them in her thrall and did not let go until she stopped singing. The spell still works after a half-century; Cerquetti is still great.This is a desert-island recital. Put it alongside your Callas, Chaliapin, Vickers, Gobbi and Terfel records. They are the only singers who were in her class; they represent an operatic world gone and vanished, like a morning mist burned off by the sun, but in this case the mist has been bottled and preserved for future enjoyment - if, indeed, anyone in the future will care much. Anita Cerquetti was one of a kind.",e +"I initially heard a Tuff song from an internet radio station and thought I would check them out. This album is great with some really awesome songs. However the recording sounds like it may have been from a tape or maybe just lower quality. There's some high pitched feedback on some of the songs that make it a little difficult to listen to. The CD showed up autographed with a couple of buttons, which was super badass.",p +To describe this album in one word: Dull. Monotonous. Irritating. Take your pick.,[ +"This CD has been one of my favorites and it is an old CD. I was so glad to find it on Amazon as I wanted to order several copies for next year's Christmas presents! It arrived well within the alloted time frame and was very well packaged. Amazon continues to be one of my favorite shopping sources.... Thanks much, EP",m +Love it!,B +If you are a fan of the Chronicles of Narnia you will enjoy this book. It offers songs that span all of the books and if you have read them all you will readily identify what each song refers to.,T +As described...thanks!,B +"reminds me of my mother and grandparents, the songs they sang and records they played. a couple brought me to tears because i remember my mother singing with ""Tennessee"" Ernie Ford - How Great Thou Art, Precious Lord. those 2 were songs she sang to my children when they were infants until she died. they still remember her singing to them.",c +"Panda moved away from the heavy synthesizer sound they used in Poetics. Thanks goodness. I like this album much better.Panda uso demasiados teclados en Poetics. En este album, regresan mas al sonido de guitarras que los habia identificado en el pasado.",r +"I waited for this CD for years, but due to the price and the limited quantity in production, was surprised by the low price.The music is very moving, relaxing and brings to mind a distant memory.",r +Such a beautiful voice. Sad to learn it is now silenced yet thrilled to admire the songs which Yao Beina has blessed us with.,C +A great compilation. A must for any Beatles fan.,] +"Love this CD. Only problem is the format is UK and wont play through my computer. Music is beautiful, lovely voice,",i +"The mail got mixed up and this had to be forwarded (we moved). With COVID, shipping overseas was delayed. But, it got to us eventually and the seller even gave us a replacement, saying we could keep it. We ended up with 2 copies. Thank you for your generosity!This CD is the best. Relaxing, beautiful, classic Kokia.",V +"Loved the original! Was going to update with a CD, but, at $90, forget it.Edit, January 2023: Now only $234.00 shipped...hahahahaha, no way man...cannot recommend.",q +Great listen.,W +"I love all of Andi's stuff. Her sense of humor just cracks me up, plus her accent is pretty awesome too.",S +Did not work in my CD boombox or car CD player... :(,C +Great! thank you!,T +"Wow... just wow.. this band swings like h....!!!For starters, let's talk about the choice of tunes... ""Woodchopper's Ball,"" ""One O'Clock Jump,"" ""Two O'Clock Jump,"" ""Intermission Riff,"" etc., all borrowed from the masters of the Swing Era. There's no doubt that Kaempfert puts the usual touch with the strings and chorus that came to be his hallmarks but don't let that deter you from the fact that this rhythm section drives these hard swinging tunes like a well-oiled machine. The brass cuts through like a razor sharp knife and the reeds are every bit as impressive.It's a great album, well-recorded and executed the way a big band recording ought to be. The only weak link in the chain are the original tunes towards the end, which, in my opinion, lack the spark that Kaempfert's tip-o-the-hat to his American big band idols possess. Even so, it's an enjoyable trip back in time and you'll be smiling ear to ear.... or rolling up the rug for an extended jitterbug session.In short, love it!",S +"For Boswell Sisters and Connee Boswell fans, this album is a ""must have,"" for this is Miss Boswell's final album. After this LP was cut, she only made one more commercially released record, a 45RPM single on the independent Charles label from 1963.For those new to the Boswell legacy, you will be interested to know that Connee had been making records since 1925, so when this Design album hit the market in 1958, she was already a well-established and highly influential jazz and popular singer. Her career had spanned several recording companies, with her first record (along with her sisters Martha and Vet, the Boswell Sisters) was made in New Orleans in 1925 for Victor. There were a few sides released on the Okeh and Victor labels in 1930, then a series of records by Connee and the Boswell Sisters on Brunswick and Decca. After the Boswell Sisters broke up in 1936, she continued recording for Decca until 1946, briefly recorded a few sides for Apollo, returned to Decca in 1951-1956, back to RCA Victor and finally, two LP's made for Design in 1958.When this record was released, jazz and vocal big band standards still somewhat shared the charts with Rock and Roll, though the era of the jazz pop vocalist was passing. Also, Connee was past her prime and as one fan has said, the fabric of her voice had begun to unravel by the time this record was made. That's not to say her singing isn't top knotch-- she does a fine job here but this isn't her best work.The tracks:Everything I've Got Belongs to YouYou're NearerI Could Write a BookMy Funny ValentineI Didn't Know What Time It WasMy RomanceThe Lady Is a TrampBewitchedWhere or WhenThere's a Small MotelMy personal favorite is ""Where or When.""As to the recording quality itself, the records are available in high fidelity mono or hi-fi stereophonic; however, I will warn any purchaser of the LP's (or the less common reel-to-reel) that the stereo versions suffer from exceptionally poor engineering, what might be best described as an experimental stereo, resulting in odd effects and at times, Connee being drowned out by the orchestra. This makes the listening experience less than stellar.So all of that being said, you decide you want to give a listen but you don't have a record player or reel-to-reel player? Since this album has never been released on a properly remastered CD with this cover, you'll have to shop around for a long out-of-print CD called ""An Evening with Connee Boswell,"" on the Pickwick label.Enjoy!",T +"If you like America, Firefall, Nitty Gritty Dirt Band, etc., you will love this band. It's really too bad they didn't record more!",i +"Great product it’s great they have this on here. Mac demarco does it again, even tho at this point it’s time to leave the cassette band wagon. It’s got to mainstream",q +love it,G +I have some other Phil Hendrie CD's. This isn't quite as good but it is still vintage Phil.,V +"I wish I could give this CD tn stars. :D Kim Sung Kyu (whom I dub the Snuggly One) and his solo album are fantastic. I CANNOT get enough of this CD; it's hard to choose a favorite song, though I think "41 Days" wins the prize. All I can say is get this CD! You will not be disappointed!Oh, and it's packaged beautifully, and the book has pretty pictures. :)",e +A must have for every TPAHBS fan,G +This is such a good CD. I'm so glad I got it. I used to see it in my mothers collection but never paid attention to it. Now that mom is no longer with me I. Saw the cd and took a chance in buying it. It was the best thing I've done The music is so beautiful and brought many memories.,_ +I used it to play all day on my sisters death anniversary .,O +I love that this music connects me to my grand parents and my mother. May they rest in peace. When I was little I'd hear them playing this music and singing so happy. I have looked for so long for these cds. I know now that I should of gone to Amazon first!!! 😍♥️😁,n +What I liked the least is I didn't get this one. I even forgot all about it.,R +Ok,j +In great shape and received it super fast!!,Z +Christmas gift.,m +Gets repetitive and whiney,] +Terry MacAlmon instrumental-what can I say? So beautiful!!! Relaxing to play for meditating on the Lord or as background music that fills your space with beautiful praise. I highly recommend this CD!,s +great,d +The best CD I've purchased since The Joshua Tree.,t +The description clearly states that it’s a factory sealed cd. It’s just some nicely packaged pictures and junk. I was expecting that cd. Disappointing.,I +Great CD,k +"Kind of out of date , like myself.",U +I love it wow just wow,Z +What a great way to celebrate the season. My husband loved this too. We took it on our trip to a friends for Christmas! Perfect for the road trip! Very talented young people. A great addition to my Christmas music library.,e +"I give this five stars because it is exactly what it appears to be. This is not a slick studio production for the mass market. Its a master finger-style arranger/composer ""sittin' back pickin' some great stuff. While the rest of us were listening to and admiring Chet, John Knowles is one of the pickers Chet was listening to and admiring.",r +Wonderful find for my Jukebox and reminds me of high school!,H +Loved this song when I heard it from the movie. Just wish more US stores would stock stuff like this locally.,I +Low sound quality.,M +GREAT WORSHIP MUSIC ❤️GATEWAY CHURCH 🙌🏽✝️🎉,Y +Great product great price,x +Great singer,j +"My mom bought this box set when I was just a kid. As I got older, I would play these on my little record player, over and over. Sometimes, I would cry as I sang along. I loved them so much. All of the old favorites are in this box, and some I had never heard before. I have the entire 5 record set memorized now, which has come in so handy as the mom of a colicky baby... I have hours and hours of lullaby material. Thanks, Simon & Garfunkel!",[ +"Based on good reviews, I bought all 6 in the series. The first one I listened to was so relaxing and brought me such peace. And another I use at night to help me fall asleep and as I wake up every 2 hours due to insomnia from menopause. I highly recommend the series.",J +Nice CD,P +Great Music to cook to.,i +Not so great.,B +"I have not seen the film but I am so happy to have the CD. It's very short, only five songs, but it's extremely beautiful. The genre is a very good mix of classical & pop.",X +Awesome song. Great quality,b +"A beautiful, easy listening classic. So enjoy Gleason's arrangements and interpretations.",X +Great Easy Listening!!!! Love performers..both Beegie and Jack J.!!!,F +I am singing solo for Christmas Cantata...this is just what I needed. Wonderful example why I love Amazon.com. You need something - you will probably find it.,v +I am singing solo for Christmas Cantata...this is just what I needed. Wonderful example why I love Amazon.com. You need something - you will probably find it.,i +"It was unexpected and it’s live! Other reviews mentioned the songs go quite at the end duh- I think The Kids Are Alright does too, whatever it’s meaningless. This is The Who, so the price here is ridiculous, find it for under $25. The sound is awesome. So Daltrey , Pete and Zach f. Starkey continue in 2022, Tommy recently put out having Daltrey sing lead solo is fabulously different, and his voice was fixed with an operation in Boston. But this Blues… Who disc is live songs we’ve all heard before… play Bargain and it’s worth it.",r +"Nice and all classic songs, but limited. Sounds of Summer is cheaper but has 30 songs! This collection is more fun and everyone knows the additional 10 songs, most everyone. Some “musician” gave one star, but I’ve seen The Beach Boys 5 times...and with this cd’s excellent sound quality—only 5 stars works!",N +"Way back in the eighties, before cds, I bought the vinyl Beatles live at the Hollywood bowl. They clean it up and almost stereo, screams separated from the boys pretty excellent performance when they can’t hear themselves or one another, or barely. This 4 cd package is that repeated on 3 cds supposedly at venues around the world- because it’s never announced where they are!? I am not interested in the 4th cd which is radio stuff.",E +"Soothing, relaxing, and my new favorite cd.",[ +Great memories.,j +"Sarah Tiana is a very funny comedian. If you want to hear more from her, check out her eps on the Ari Shaffir Skeptic Tank Podcast. A very moving and interesting look into her love life. And go see her live too and i can't wait to get another album from her.",R +I remember listening this album back in 1979 and was addicted the all the songs. Such great classics - every song on this album. Listened to the album again last night. It was like being back in 70s/80s again. If you from that era you will enjoy it.,f +"I bought t this CD in the blind. I heard music from Klingande this listed... this was definitiely not a CD I wanted, had I'd know about the return policy it would have been gone! (horrible music, well... not my taste rather)...",Q +Nice to find a very well coordinated collection of first class music by a very talented artist like Anna Hawkins.Rare in this day to have a great team arrange a work such as this devoted to God.The bonus tracks alone are worth the price of the CD in my humble opinion.Very pleased with the purchase.,Z +I like this album and hearing it in 5.1 is worth the price (to me).,I +Enjoy this type of music. Good replacement for FM broadcast.,Z +Was a gift and the person loved it!,y +Great,p +"I owned this cd a while ago, but misplaced it!!! Was so glad I could find it again, it's an oldie but goodie, so I bought two!",w +"WRR is a public classical radio station in the Dallas area. This CD is a collection of carols from their annual (not sure if they still do it) caroling at the Adolphus hotel.I enjoy listening to this collection. It's live, as the applause clearly shows.This is a choral collection featuring the groups listed in the track listing below (there are 14 tracks):1 - Holiday Moods (Medley) [Dallas Symphony Chorus] a. Deck the Halls b. O Christmas Tree c. Here We Come a Wassailing d. Carol of the Bells e. Jingle Bells2 - Christmas Portrait (Medley) [Highland Park Presbyterian Church Chamber Chorale] a. It's Beginning to Look a Lot Like Christmas b. The Christmas Song c. We Need a Little Christmas3 - Sleigh Ride [Dallas Symphony Chorus]4 - Spanish Allelu [Women's Choir of Dallas]5 - Do You Hear What I Hear? [Turtle Creek Chorale]6 - White Christmas (Dallas Variation) [Dallas Symphony Chorus]7 - Song of the Faithful (Medley) [Women's Choir of Dallas] a. O Come All Ye Faithful b. Little Star c. The First Noel d. The King of Us All e. On a Starlit Night f. Silent Night8 - Ave Maria [Women's Chorus of Dallas]9 - Masters in This Hall [Turtle Creek Chorale]10 - Angels We Have Heard on High [Turtle Creek Chorale]11 - The Twelve Days of Christmas [Highland Park Presbyterian Church Chamber Chorale]12 - Bethlehemu [Texas Boys Choir]13 - We Wish You a Merry Christmas [Dallas Symphony Chorus]14 - Peace Peace [Turtle Creek Chorale]",[ +"Episode listing:Series 9, #12 "The Call of the West"Series 10, #6 "The Last Smoking Seagoon" (the final show)Series 5, #15 "1985"*Series 7, #17 "Shifting Sands"* According to purists, the notes incorrectly state that the original broadcast was 08 Feb 1955, but the correct original broadcast date for this show is 04 Jan 1955. Apparently, this show was rebroadcast as #20 on 08 Feb 1955",] +"Episode listing:Series 4, #23 "The Greatest Mountain in the World"Series 5, #8 "The Mystery of the Marie Celeste (solved)"Series 5, #9 "The Last Tram (from Clapham)"Series 5, #11 "The Spanish Suitcase"I have all of these; the last three are all about equal; I'd give a slight edge to "The Mystery of the Marie Celeste (solved)"",Z +"Great for a BB collection, never saw this CD before-",v +"had this on cassette tape..LOVED IT GROWING UP, GREAT WAY TO LEARN BIBLE VERSES AND CATCHY TUNES",` +This is an AWESOME cd,] +love it,R +not able to listen the cd is BLANK,R +Ok not what I was looking for,L +just a nice quiet cd to listen to at meal time: kind of space age music: will use again: real good flute!!!,R +good seminar,h +nice music.,r +nice. would have liked a few more songs on it,Z +peaceful praise.,b +Love having the beauty yet simplicity she brings to old hymns. Wonderful.,U +"BTS ARMY all the way. You go boys, you rock!",n +"Wasn't any song info on this cd so I figured it was the same as the other & get a better deal but not. 5 songs on this, none from the other, not even sure it's the same band & nothing like the the other or to my liking.",d +"Good music, fast shipping",V +First heard In the Game song and love it. Rest of the album is equally as good.,i +"Great music. Great price. When I ordered, I thought I was getting Karaoke, but happy with the CD anyway..",f +I love everything Marty does musically.,u +"I had no idea who the young man singing with Cynthia Erivo was, and didn't care. After seeing her in the play, The Color Purple, I just wanted to listen to her voice again. Since the cast album is a few weeks away from release, I tried this cd. I hate people who always say, "Oh, my God!", but, honestly, "Oh, my God!!!!" These are two of the most beautiful, complimentary voices you will ever hear! They each sing by themselves, but there are a couple duets. The music reminds me almost of show tunes, though mostly slower, sometimes haunting tunes. It's just such a beautiful album!",t +absolutely love this CD.works well and my vehicle and in home devices. well packaged and shipped very quickly. Very pleased,a +"Zelda is AWESOME, and my 12 year old son loves to listen to all the music and effects on this CD.",] +Bought it for a friend who was thrilled.,N +perfect,z +Soothing for all ages. Beautiful,] +"Larry Norman is cited as having the first major label Jesus Music release on Capitol Records in 1968. Shortly after that, a small church in Costa Mesa, California would suddenly find itself the epi-center of the Jesus Movement in the United States, with tremors that are still being felt today!Under the watchful eye of pastor Chuck Smith, a new record label was born to be used to spread the music that was being birthed at Calvary Chapel. Those first two albums were ""The Everlasting Living Jesus Music Concert"" and ""Maranatha! 2"". Those two classic albums, which launched the careers and ministries of artists like Love Song, The Way, Children Of The Day, Gentle Faith and Debby Kerner plus many others, are included on this first of four ""Collections"", which include all but one song that were included on the first seven Maranatha! albums.Over all, the transfer to digital and remastering is quite nice, with only a few ""defects"" and wobbles from the original two track master tapes apparent. These re-issues were only sold in limited numbers and will not be re-issued again, so they are becoming quite collectible AND expensive so if you don't have a copy yet, the longer you wait, the more it may cost you. A word to the wise, make sure you exhaust all other sites online before buying as you may be able to find a cheaper source.Maranatha! Collection Volume 1 Performers and Songs:1) Love Song - Little Country Church2) Blessed Hope - Something More3) Country Faith - Two Roads4) All Groups - Holy, Holy, Holy5) Gentle Faith - The Shepard6) Debby Kerner - Behold I Stand At The Door7) The Way - If You Will Believe8) Love Song - Maranatha9) Children Of The Day - For Those Tears10) Country Faith - Come Quickly Jesus11) The Way - Jesus Is All That We Need12) Denny Stahl - The Son In My Life13) Blessed Hope - Never The Day14) Selah - He Lives15) Debby Kerner - The Peace That Passes Understanding16) The Way - Jesus Is The One17) Country Faith - Ballad Of The Lukewarm18) Ken Gulliksen - CharityIf you were a part of the Jesus Movement during 1970 - 1972, you probably are very familiar with the music on this CD. If you don't have this re-issue of the original vinyl albums, you should get a copy while they are still easy to get!",L +Mine got lost in a move. But I'm going to replace it soooon. Great acoustics; you really HAVE TO hear it to believe it."Speak Low" piano / drums wow goosbumps. (NO I am not gay.)GREAT album.thanks for reading,R +"I absolutely love "The Song of Crazy Horse." I'll fully admit that this song (parts 1 and 2, or one live long version of both at the end of the CD) is the main reason I bought the album, but it's so good that's worth 5 stars to me by itself.",S +"Wow, wow wow!! A product that actually delievers on it's claim. From the moment this CD started I could feel the absolutely pristine sound cleansing the atmosphere. So let me tell you, I'd had challenges in my kitchen. A year and a half ago on the coldest night of the entire winter the neighbor upstairs pipe froze and burst and completely flooded my kitchen. My kitchen had to be gutted from ceiling to 2 walls to floor. Some new wiring all new cabinets it was a total remodel. I had to move out of the condo it was so extensive. Then, just a little over a year later, there was a slow leak in the wall that caused problems in my newly remodeled and beautiful kitchen. Not normal to have two such events and so close together. It has taken a while to deal with this and it's still not totally done. I thought I'd like to play this CD in the kitchen and yes! It has cleansed the energy and it feels so nice in there once again. This CD is available from Billy Topa Tate's center in Evanston, IL. They sent it quickly. One more thing, this may be just me, but I find I like to play this Enhancement at a much lower and softer level than I use for music. It seems more mysterious and effective at a soft volume. Very effective and cleansing.",N +This CD is defective. It plays the first two songs okay. The 3rd one you wait about 1 minute (60 seconds) and it starts to play. The rest of the CD doesn't play at all - just a series of clicks. I'd like to return it for a full refund including shipping. Hopefully this particular CD is a dud.,B +"Very nice collection for evening listening. Kirshbaum should be more famous then he is. Indeed, he has played in groups with some of the superstars. Who knows why these things happen that way.",L +"This is a good introduction to the world of classical piano. You get to hear at least two very fine pianists -- Earl Wild and Ralph Votapek. The sound is average. The vinyl of the album was my introduction to the world of classical piano. There are some interesting pieces here. The truncated version of the Rach #2 works for me. I was surprised when I heard the real, longer version. The Slaughter on 10th Avenue piece is rarely performed and this is a good version. All in all, if you want to get someone started on a musical journey, this is a good gift.",i +"This is a nice collection of obscure 20th century works by American composers, for small orchestra. The sound is not sumptuous, but it is clear, and the interpretations are convincing. Standouts are Nicholas Flagello's Adoration, Barlow's Winter's Passed, and Wilson's Dedication. It is a somber, evocative album, good for a winter night by the fire. I would call the bulk of the music neo-romantic, mostly tonal with some dissonance. If you are looking for something different, this is pretty good.",h +"If you love Grieg's orchestral versions of these pieces, you'll equally love the piano versions played here by Katsaris. His playing is crisp, but never loses the romantic side of these compositions. The warm piano sound is very good. I have owned this disc since it came out 30 years ago and it has never lost its charm.",I +"This has become a favorite in our house. We listened to it twice in a row the first two times we put it on. Summer has a clear, lyrical voice. Purists will say it is not an A+ voice it sure is pleasant to listen to. An obvious comparison is with Katherine Jenkins, and Ms. Jenkins has a somewhat stronger voice, but I slightly prefer the softer edges of Summer's voice. I think this will appeal to both fans of classical and pop music.",O +"Sherri Huffman has a very soothing (and good) voice. The musical accompaniment is soothing. As they get older, the kids will enjoy the lyrics to the songs. Highly recommended.",P +"Barbara Rosene has a voice of remarkable clarity and purity. She sings in the tradition of the great women singers or the 1920s and 1930s,such as Ruth Etting and Annette Hanshaw. This exceptional collection combines a beautiful voice, talented jazz musicians with some of the the most memorable songs of the Jazz Era. The moon has held a unique place in the hearts of composers, musicians and listeners and this CD shows why.--------------------------------------------------------------------------------Record Label: Stomp Off Records 1405 Total Time: 78:14--------------------------------------------------------------------------------Tracks on 'Moon Song'1. Get Out and Get Under the Moon [3:43]2. Moonglow [4:51]3. Me and the Man in the Moon [3:02]4. Moon Song [3:48]5. I Never Knew What the Moonlight Could Do [2:18]6. Under the Moon (you-oo-oo-oo) [4:09]7. Moonlight Becomes You [4:00]8. Livin' in the Sunlight, Lovin' in the Moonlight [3:16]9. It's Only a Paper Moon [4:30]10. Me and the Moon [2:20]11. Moonlight and Roses [4:59]12. (there Ought to Be A) Moonlight Saving Time [3:24]13. The Moon Got in My Eyes [4:18]14. On a Chinese Honeymoon [2:51]15. Moonburn [3:32]16. When the Moon Comes Over the Mountain [3:43]17. I Thank You, Mr. Moon [3:15]18. Sheltered By the Stars, Cradled By the Moon [3:45]19. Moonstruck [3:49]20. Shine On Harvest Moon [3:21]21. I Wished On the Moon [5:20]",K +"Barbara Rosene obviously enjoys these old songs. Her voice is unique in its clarity and beauty. And the recorded sound by Jim Czak is so vivid that you'll feel like Barbara and her splendid musicians are in your living room with you. You can almost imagine that you are back listening to Ruth Etting or other remarkable women who sang with the great music groups of the 1920s and 1930s. This is just one of several outstanding CDs she has done, including: Deep Night and Moon Song.--------------------------------------------------------------------------------Record Label: Stomp Off Records 1422 Total Time: 75:48--------------------------------------------------------------------------------Tracks on 'It Was Only a Sun Shower'1. Medley:oo-oo-ooh! Honey (what You Do to Me), Ooh! that Kiss [2:33]2. There Must Be Somebody Else [2:51]3. My Baby Knows How [2:35]4. Close Your Eyes [3:18]5. Tip Toe Through the Tulips [4:02]6. Love Me Tonight [3:30]7. Mine All Mine [3:12]8. Song of the Wanderer (where Shall I Go?) [3:19]9. Just Like a Melody Out of the Sky [3:19]10. It Was Only a Sun Shower [3:18]11. Jeannine, I Dream of Lilac Time [3:29]12. Love Me or Leave Me [2:14]13. Red Lips, Kiss My Blues Away [2:33]14. Say It Isn't So [3:30]15. Nobody's Using It Now [3:28]16. We're Back Together Again (my Baby and Me) [3:04]17. He's My Secret Passion [4:51]18. I Think You'll Like It [4:21]19. Was It a Dream? [2:47]20. The Right Key But the Wrong Key Hole [3:59]21. I Don't Mind Walking in the Rain [2:51]22. Funny Dear, What Love Can Do [3:27]23. Got No Time [3:17]",W +"The finest authentic re-creation of the Big Band Hits of the 1930s and 40s. Done in clear, crisp audio. This is the first collection of Big Band standards from Vince and the Nighthawks. They capture the essence and swing of this era perfectly.Here are the playing numbers and playing times on this remarkable CD:Tracks on 'Moonlight Serenade'1. In the Mood [3:33]2. Pennsylvania 6-5000 [3:00]3. One O'clock Jump [3:46]4. I Can't Get Started [4:51]5. Don't Be that Way [3:39]6. Tuxedo Junction [4:53]7. Jumpin' at the Woodside [2:51]8. Song of India [2:57]9. Moonlight Serenade [4:51]10. String of Pearls [3:22]11. You Made Me Love You [3:32]12. Little Brown Jug [2:45]13. Begin the Beguine [3:23]14. Take the ""a"" Train [2:40]Buyers of this CD should also consider his other outstanding performancesmost of which are out of print, bu all available on AmazonCHEEK TO CHEEKCOTTON CLUB REVISITEDDEEP NIGHTQUALITY SHOUT",A +The previous review says exactly what I would say. So here's some added information: the precise musicial numbers on this magnificent CD.1. Cotton Club Stomp [2:58]2. Stormy Weather [4:28]3. Get Yourself a New Broom (and Sweep the Blues Away) [2:45]4. Trickeration [2:50]5. I've Got the World On a String [4:04]6. Harlem Holiday [2:33]7. Happy As the Day Is Long [2:33]8. Minnie the Moocher [3:02]9. Raisin' the Rent [3:33]10. Between the Devil and the Deep Blue Sea [3:22]11. As Long As I Live [3:35]12. Minnie the Moocher's Wedding Day [4:43]13. Truckin' [2:46],F +"This is Tim before he got a bit commercial. Each of his three early albums are an eclectic mix of blues, rock, and unique meanderings.",Q +My folks loved it. They were dancing to Li'L Wally back in their younger days. Thank you.,H +Great! My wife and I saw them in person multiple times.,c +"This collection of guitar music from William Elwood and another artist are so soothing! I purchased it because I've had other CDs by William Ellwood and enjoyed them very much. While each piece is different, they've been chosen to blend very well, one after the other. (There's nothing more jarring when listening to a CD than to have some crashing music after a gentle piece!) I play this music when I'm drawing or painting and it is great to help me let go of stress and get into the beauty of color and design. I think it'd also be good during meditation (which art is for me.)",D +Heard her in the city and bought the album on the spot from her.,z +Loved the music. Great CD. The sleeve it came in was cheap and too small. I had to cut down the sides to get it out. Not a good idea.,I +Outstanding performances. Arrived timely and was as described. Thank you very much.,f +Item arrived timely and was as described. Thank you very much.,t +"I love this cd! It was very hard to find, so glad Amazon had it. The music and nature sounds are beautiful.",\ +great product. great price,D +great product. great price,A +Great music at an affordable price. Ward was one of the best and..you get Shaw and Goodman to boot!,` +Thank you,w +Thank you,M +"See it to believe it, I wish they would come to America and perform they are so cool and talented",s +"This is an excellent album of rare gems.I feel your sure to be surprised,when you here this album.It contains 32 tracks,of great goodtime oldies.A must have for those looking for rare oldies music.i own this disc 5 star rated,and a good deal...I thank you Stewart L.",` +"The ronettes ultimate collection,is a super collection of their greatest hits,and,some of their lessor known hits...All tracks are in true stereo,unlike,the horrible back to mono us releases..The sound is fantastic...This album is a must have for your cd collection... I would like to see an ultimate phil collection re-released in real stereo,like the original LP'S.They sound wonderful...Now i highly recommend this cd 5 star rated.Thank You..Stewart L.",c +"Great song that he did with the doobie brothers, and Carly Simon di also a great version.",c +"I received it fast and in great shape. The LPs were like new. The cover had been taped on the fold. Other than that, It was great.",^ +"If you saw this stage production in person. You will love the album. I used to have it when it was first out, but over the years it had gotten lost. The used vinyl is near perfect. Only a couple of small clicks on a track. I had forgotten I owned the album when it was new. But, I knew the words of every song as it played. Wonderful memories for me. The only Broadway show I have ever seen in person.",n +"LOVE ME OR LEAVE ME was made famous by Sammy Davis, Jr. in the fifties. Nina Simone put out her version and it topped the lists. I was glad to get this, even on vinyl. It isn't on any of her CDs. I wore it out in the early 60's. The other side is I LOVES YOU PORGY.",O +I really liked hearing this classic cd plus it is rare and out of print you'll love to have this for your collection this is a much have for any ace juice or mc hammer fan.,v +good,m +"Every track on this album is quite simply, beautiful. Sarah has an immediate presence that captivates movingly. ""Tori-like"" but still Sarah uniquely. Take the best of Tori Amos and this is where Sarah begins and then evolves. Shine on starlight however long the muse beckons.",A +"I admit I never listened to Justin's music until now. Especially love the song, Ghost.",[ +"My wife and I hope to go to Mexico someday, but this is the next best thing. This CD really gives you a feel for what it might be to be in a cozy bar in Mazatlan having a couple of beers, just taking in the music and the fun. This would be a good CD to have a couple weeks before a Mexico trip, to fire up for the week of fun. :)",d +"This is an excellent collection of songs from a doctor/lawyer combo called ""Depression Glass."" Jeff Mead's heartfelt vocals and Scott McAndrew's technical wizardry make this one a winner!The original and remakes are both outstanding on this CD. I am glad this CD is available on amazon. That will allow more people to discover the great melodies and words that I have discovered! :)",h +Shipped fast and Cd worked. My brother introduced me to this Cd. He been in the military his entire life. The songs are pretty cool and perfect for a 4th July barbecue or any day honoring the America troops. I did have 1 problem is that my Cd case broke right away. It's cheaply made I hate to say but I didn't knock any points off because of it,U +Great album. Bought it for a gift,O +Excellent CD!!! Great condition! Bought several others for friends who also went head over heels in love with this CD!,a +"We had this album on cassette as kids and always loved it. It was the best ""road trip music."" Now, I have the CD for my son and he loves it, too. He is only 2 and knows several of the songs by heart and often asks to hear the ""Amen song."" I'm happy to be able to share the music I loved as a kid with my kids.",A +"One of my absolute favorites for relaxing and thinking through life. I have heard my share of cheesy relaxing music. This cd is not the same. It is filled with soft melancholy piano music with thunder,rain and birdsong in the background. I have had it for over 6 years and still love listening to it.",^ +"An absolute masterpiece. I love Eric Carmen’s music. I would go back and forth to Penn State listening to this cassette on my old Walkman to the point that I recalled every song on this record by heart. One time I was in a bus going to college and the driver yelled at me to turn my headset down, “personalize it,” I recall him saying and I laughed. This was that kind of record. I got totally lost in it and every song is a good one. One reviewer did not like “I Want To Hear It From Your Lips” which was the single that got a lot of air play on VH1 which was in the mid 80’s sort of an MTV for a more grown up sound. I loved the video and the single. I agree there are good ballads on this record too like “She Remembers” which is an ode to seeing a high school love at a reunion and having that moment at a reunion where you feel that feeling of why she was important to you even though the relationship has ended. Bottom line: this is a great Eric Carmen recording from the mid 1980’s. Buy it!",v +Came to house very fast,s +"Secondhand News is such an astonishing song, doctors should prescribe it to depressed patients. It is not possible to feel down listening to it.The opening guitar notes announce that the singer of Monday Morning and Blue Letter (perfectly great songs) has found a way to re-sort all of that ...into a giant, propulsive, pounding masterpiece.",q +"KREEPKREEPSHOW-THE LOST ALBUM1.INTRO-excellent instrumental intro with some familiar horror movie music, you will know it when you hear it. Produced by Boondok.2.RACKS ON RACKS-the first track comes out knocking like crazy. Kreep tears up every verse and also produced the instrumental. This one would have been a great single. I could see this one getting a lot of views with the right video.3.STACK THE PAPER-this one was produced by Hostyle ad has that classic West Coast feel to it. Kreep spits about getting that money with Doc Holiday. This is a nice smooth track to roll to, another great song.4.ON ONE-Kreep is back on production and this is a thumper right here. This one has that Hyphy feel to it. Kreep comes correct with King Duce Boyz. This is another one like track two that could have been a cool single. I could have seen this one banging in the clubs.5.THE SOLUTION-Kreep drops a darker sounding beat on this one finally matching the cover art of this album and that intro. Kreep and Skillz deliver hard on this one with that darker street spit. Skillz in particular sounds great over Kreep’s production here.6.OFFICIAL-the song that I bought the album for back in the day as Brotha Lynch Hung comes right out the gate spitting that real over Kreep’s excellent beat. This one right here should have had a video for sure. The beat is great and the hook is catchy with Lynch, Kreep, Skee 64 and The Illiacs killing this track. This is probably my favorite song on the album.7.SUPER FLY LIKE-Kreep sounding great again over his banging instrumental. This one is another banger with Kreep and Young Game dropping some great verses. You will recognize a sample on this track, maybe from a few 90’s era Hip Hop tracks that were popular.8.GANGSTAS-this one to me fits the style of the cover as it makes me think of a messed up carnival for some reason. Kreep goes solo on this one over his kinda eerie production. I really like this one for some reason although it is not my favorite.9.HOES-Kreep and Young Game team up once again over another self produced track from Kreep. This one is kind of a dance style beat but a little darker. I am sure you can guess what this track is about based on the title. Young Game sounds really comfortable over this beat, his flow here makes me think this could have been another single.10.CLAP, CLAP-we get another club type beat and song with that darker tone to it. I really like Kreep as a producer I have come to learn. Kreep does his thing of course as does Yaberation who kinda sounds like Yukmouth to me, especially his backup vocals and adlibs.11.CAN I SHINE-once again Kreep goes solo over another self produced track. This one also features the style of the album based on the cover and name of the album. This is a cool piano driven track,12.CITY OF RIVERS-Kreep gives another club type beat for himself and Nino Tha Block Kontrolla to rip up and bounce to. This is a nice bouncy type track that I am sure most would enjoy.13.HANDLE DIS-straight West Coast right here with Kreep dropping that beat and first verse here. This is also the track were we get Legend Keak Da Sneak on the album. Keak takes that second verse followed by Lameez on that final verse. This is another good song on a strong album.14.HAVIN BABIES-Kreep telling some real stories right here, kinda his “Brenda’s got a Baby” type track. This one was produced by Night Train and while different it fits the album well.15.PAY ME-Kreep and Young Game are at it again this time spitting it for the ladies. A more fun instrumental this time from Keep but they still sound great together. Another solid track for the album.16.GET PECOS-this is another banger that could have been a single with Kreep going dolo again both on production and rhyming. Obviously you know what the song is about. Another great song on a strong album.17.NEXT 2 U-Kreep producing another club track for the ladies with a nice catchy hook by J Malik who also gets a verse here on a potential single. I love the shout out to the late Mac Dre here, bouncy track.18.JUAN GUATEMALA-Jay G drops a Latin flavored instrumental here and Kreep sounds excellent over this. This for sure should have been a single, I could see a video for this that would have been great.19.DOUBLE DIP-Kreep himself produces the final song on the album and the beat reminds me of something but I can’t think of it. Kreep and Mr. Imakulent do their thing on here and end the album kinda on a humorous note I think.20.OUTRO-excellent instrumental outro that I assume was produced by Boondok but I am not sure.This was a stronger album than I remembered with some good production and verses. The album has a mix between styles of songs from club stuff to serious stuff. The album cover and the knowledge of Lynch being here leads one to believe it would be a darker album but one would be wrong.",t +Nice curly wig if looking for lots of hair,t +"Ah Not what I anticipated, but ok",d +"Jasmines voice is amazing, love every song she does.",_ +price,X +Steely Dan is awesome...RIP Walter :(,^ +Love this! Where have they been and how did I miss them! Can hardly wait to get more of their music!,k +Great to hear the early sparks sound live even though live on the radio. Must have for your collection,T +If you are a fan of the Y100 Sonic Sessions- you cant miss getting these. You will not be sorry that you bought them.,Y +Love her music. What's left to say? :-),K +"I used to go to a hypnotherapist. I've tried various self hypnosis cds since he retired and Blenn Harold's are THE BEST. I go under after about two minutes and wake up the second it's over. I feel a lot better, refreshed and yes the message does up lift me. I have two of his. I wore wone out and ordered it again. I've tried various CDS like this but his are by far the very best, truly matching my session with my old hynpotherapist.I don't normally lie on a bed during the day and just drop off; I'm going under as I do his breathing and my subconscious is picking up his message. On a few occassions I wasn't totally under and I heard the message in full and its as if he tailor made this for me.",v +I love Belleruth Naparstek's guided CD's. They really can help many people,y +Great Cd!!!,h +I bought this just for the first song on the CD.Nothing Can Hurt you There is an awesome song.It reminds me a little bit of Steve Winwood.(While You See A Chance),W +Was supposed to have 2 CDs. But there was only one.,D +great album,G +"This book is a life changer. And in my personal opinion, a life saver. It changes everything about what we've been told about hereditary diseases and frees you from them.",L +"I didn't get to go to this concert, so it was nice to find it on cd. This is the complete concert, including between song banter with audience.",K +I ordered this because I heard her cover of Shocking Blue's "Send Me A Postcard" and loved it. Amy is from the Detroit area and I like to support local talent. I was hoping that at least half the CD would be good and was surprised to find that I like all the tracks. This is fresh music with a little 80s punk meets 60s girl group and some Shal Talmy British invasion mixed in.,P +"This recording was made after the only surviving Messerschmitt Bf 109 G-2 Trop was lovingly restored. Includes start-up, take off, in flight, and landing, along with pilot chatter. Booklet is loaded with the history of this aircraft.All the recordings in this series make you feel as if you are there on the tarmac. Be careful, my husband blew out one of my speakers with this one.",H +"This is one of a 5 CD set of classic radio broadcasts from 1938-58 when comedians used wit, not vulgarity, to elicit laughs. Includes a 56-pg booklet chock full of facts & trivia about Vaudeville, the performers and the shows on which they appeared.Disc 1:5/12/48 Abbott & Costello's only known surviving radio program to feature ""Who's on First"" --also includes ""Bob Feller,"" & ""Itch.""9/30/47 Milton Berle Show's ""Salute to Brooklyn"" starring Uncle Miltie & his zany sidekicks Pert Kelton, Jack Albertson, Billy Sands, Roger DeKoven, Jackie Grimes, vocalist Dick Carney, Ray Bloch & Orchestra and announcer, Frank Gallop.Disc 2:4/3/43 Amos & Andy w/announcer Bill Hay.10/22/38 Your Hit Parade ""Promotions Unlimited,"" starring W C Fields.1/29/47 Philco Radio Show ""The Road to Hollywood,"" starring Bob Hope, Bing Crosby, Dorothy Lamour, Skitch Henderson, The Charioteers, John Scott Trotter Orch & announcer Ken Carpenter.Disc 3:10/21/45 Fred Allen Show w/cast of characters including Titus Moody, Mrs Pansy Nussbaum, Ajax Kennedy, Sen Claghorn (""Allen's Alley""), & The Voice of the Century, Frank Sinatra.12/22/48 Martin & Lewis Show w/Shirley Booth as the hotel maid, Frank Nelson as the usher/producer, Louise Erickson as Henrietta & Lucille Ball (the queen among kings).Disc 43/31/58 You Bet Your Life starring Groucho Marx w/Ernie Kovacs & announcer George Fenneman.1/7/45 Charlie McCarthy Show w/Bergen starring as both Charlie & alter ego, Mortimer Snerd along w/host Don Ameche, vocalist Joan Merrill, Ray Noble Orch & guest Carmen Miranda.Disc 59/17/44 Double Feature starring Jackie Gleason (The Great One) & Les Tremayne w/vocalists Andy Russell & Patsy Garrett backed by the Toots Cararata Orch.4/3/49 Command Performance w/Red Skelton & Zero Mostel, host Linda Darnell, Harriett Hilliard as Mummie, Truman Bradley as the passerby, vocalist Virgina O'Brien and announcer Don Wilson.",U +"was a gift , sister loves it",^ +"A lovely collection of tunes, thank you!",w +"This is probably my favorite Cranberries album. I am a huge fan, so I am biased, but this would be a good one to start with if you are new to the band.",m +I truly love the music of Nicholas Gunn! So exquisitely beautiful! I highly recommend!,L +"This is Advaita blended with the investigation into the sensations that lay latent in the body long after clarity has come. Would not be good for anyone who isn't already interested in Rupert Spira's talks and work. He's one of the most clear and lucid teachers and speakers in this arena, so to speak, but unless you have a natural interest in this field, it's best to just stay with whatever you're into.",r +"Voces8 fills my soul. This album is like a musical collection prayers that range in era from the renaissance to contemporary, and in mood from sorrowful outpourings to sweet peaceful gratitude. Like all great music, this album makes me feel things deeply. I feel closer to God and assured of all the beauty He wants me to experience in the universe.",q +Wonderfully relaxing music. I have a few Brian Crain albums and really enjoy them all. Great for decompressing or just ending a long day.,h +Very great musical fun from John Tesh and the Olympics. Brings back fond memories of attending the John Tesh concert promoting this album. Moving music well arranged and performed. Do yourself a favor and have a listen.,j +The one and only...,o +I bought this because Danny Brown said it was his favorite album of all time and I wanted to see what it was like. Flower child music but it's cool,L +"Loud hard tight fast thick as punk music should be 13 tracks at 40 + minutes.""on a note the movie ""Black Sheep"" with the late ""Chris Farley "" the song ""Get a Job"" is the closing song for that movie and that is how I heard about ""HOG"" too bad these guys did not get a contractGet this CD!",G +Excellent.,F +Excellent.,I +Two tracks catch and won't move forward ...and this was purchased NEW!,v +"Mamie Van Doren is one hot broad. You watch any of her movies and she could hands down beauty wise beat Manesfield and Monroe. I got this CD because in some of her movies I remember hearing the music and some of it was good. I said some. This CD has a high camp factor which in my book places it above all others yet the music brought my rating down a bit. Out of the 16 songs on the CD 10 were good and I make a judgement usually if at least half of the songs on a album are good I haven't wasted my money. In this case I haven't. The CD starts off with ""Nobody but You"" a steamy swingin' love ballad that got my beatnik bop sunglasses steamed up. The way she sings on this track got my blood boiling. Then there is a bit of a Rockabilly twang on ""Rolling Stone"" and then it really starts to rock on Do Ba La Baby written by Eddie Cochran. The background singing is stupid but the rest of the song is great. Beat Generation is a song in the jazz vain that rips into the beatniks a put down obviously written by some square musician. ""Something to Dream about"" is slightly erotic and the song is OK it is one of the better ones. ""Salamander"" is a pretty rocking one and a good rip on us men. Then theres the hilarious song ""The Bikini with No Top on the Top"" a pop song sung with June Wilkinson made me laugh but at the same time made me hot under the collar. The best song though is the awesome storming instrumental ""So What Else is New?"" Where some guys are cat calling at Mamie and June and in the off hand hot chick way they reply ""What Else is New?"" Then the instrumental just rips. All in all not bad. The rest of the songs aren't noteworthy. Their gauzy banal pop ballads from the 50's the worst kind of music the beats and Rockabillys rebelled against. However the above mentioned songs are awesome. A Fly by Night Dutch company re-issued this from vinyl and it is hard to find. It really pisses me off that good music goes out of print so some dumbass can charge 100 dollars for 15 dollar item. If you are fan of camp music like I am get this if you see it. It is well worth it.",X +"If you are a fan of the Wave music from the 1980s this Men At Work collection would be one to add to your library.While I’m not really a fan of Men At Work compared to so many of bands from the 80s, it’s a nice offset to listen to material every so often.",x +"Just about anything Adrian Belew was involved with in the 1980s is well worth your while. The first Bears album is no exception. In fact it’s pretty damn good and the better of the two Bears albums.Sadly, this album has been out of print for many years. If you are a fan of Belew and you run across this somewhere, be sure to snatch it up.",j +"If there would be only one Alice Cooper album to get, Billion Dollar Babies is this the one.The amount of good music that was produced in the early 1970s has a certain unsurmountable edge on the period of classic rock that would decline 5 years later. Billion Dollar Babies is no exception and fits right in with the Apex of Classic Rock music. It's a great album.",m +"Third in succession of the early Fixx albums, Phantoms is a fairly comparable and complementary disc to Reach the Beach. That is to say if you like the first two Fixx albums, you’ll like this one.The guitar work of Jamie West Oram once again stands out. While I am not a big Strat guy, in my mind Oram’s manipulation of the Stratocaster sticks out as some of the best application of the instrument from the early 80s. He evokes a huge tone.Presently it would appear the only early Fixx album available in CD format in its entirety is Reach the Beach. I suppose the compilation Fixx CD collections are just as wonderful. But Phantoms and a number of other Fixx albums could stand to be reissued. They are that good.Meanwhile, Phantoms and other Fixx albums represent a refreshing genre of music that was coming out in the early 80s. And for those that love Pop and Progressive Rock, this album should fit right in. I highly recommend this CD.",A +"Finding all of Johnny Winter’s albums is not the easiest undertaking. While some albums are more attainable, others are not. John Dawson Winter III is one of the hardest to find and at good price.The album itself is a missing link of sorts for Winter with it being the last studio album he did with the McCoys. Winter was to do a couple of live albums with this clan after, one being with his brother Edgar. But this album marks the end of Winter’s Rock era when he transition to something more of a full on Blues player. But this album has become something of an obscurity to the Johnny Winter catalog. Why? Who knows. There are some inconsistencies with the production as there are some changing lineups in personnel throughout. But none less it is a great album.",c +"The title collection might be deceiving. There is plenty of material from Ram and McCartney’s first album. So it’s not just a Wings compilation.I bought this collection because the single disc of Band On the Run issue does not contain Helen On Wheels, but I digress…If you are looking for a greatest hits sort collection of McCartney’s material, this would be as fair as any of good place to start.",v +"ZZ Top’s first album is well worth you time.There are aspects to this album that are better than its follow up album. And if you like the early ZZ Top, you’ll like this.",[ +"I loved this album back around the time it came out. I still appreciate it now in conjunction with the rest of Elton John’s catalog from the mid 70s. However, I wouldn’t say this was one of John’s better moments. The two albums that sandwich Caribou, Goodbye Yellow Brick Road and Captain Fantastic are better acclaimed, musical accompaniments, IMO.There are a lot good tunes on caribou. Don’t Let the Sun Go Down on me is a masterpiece while The Bitch is Back is dumb Pop stuff. Another interesting song on this album I always like is the ballad Ticking, a song about a mentally disturbed young man. Well done.Anyway, Caribou is a good record in compliment to the other Elton John albums from the time period.",V +"If you are a fan of the records Joe Meek produced, you'll love this collection.",D +"If this record didn’t spawn a major impact on the history of Rock music...You already know about this breakthrough album by Van Halen after their raise to the top of the heap of the LA Rock scene in the later 1970s. But the tone of this band and album is phenomenal for how simple the recording process was.Perhaps from a critic’s point of VH is a bit more on the Pop bent, but their sound is nothing short of huge.",O +"The information is good, but the yellow CD pictured are the latest version and the one I received was an older set. I was very disappointed I didn't know about the older set being recorded around 2008. I would have not have purchased the set I received as it isn't complete and I know there is more info, plus a manual with the newer version.Buyer beware.",Z +"Any of Sevin's CDs are great, but this one is much better in my opinion than the more mainstream Father Forgive Me.",l +Love listening to the angel codes,J +"Decent Dr Who audio adventure of the Fifth Doctor, but a little overly slow and draggy in spots.",K +"Glenn Beck is been accused of being a lowbrow opportunist who pandersto the worst elements of American culture. The ironic thing is that this isoccasionally true, but can also be said about his biggest media foe....STEPHENKING. Plus, despite the uneven output of the Beckster, at least he hasn't beeninspired by his Dubya-hatred to write a massive turkey like UNDER THE DOME...",c +"Although not quite BOGGS VOL 2, this is a good release from the Dillmeister.Also it's 35 minutes instead of an hour, so it wouldn't be quite so overwhelmingas BOGGS.",g +"very immersive, just wish the music was mixed a little lower than the height and surround effects. very happy with my purchase",s +Nice record of an early artsy band. Lyrics are a little weird at times but great instrumental playing throughout.,C +Dickie is the real deal. I appreciate his lyrics.,o +"If your looking at KTel vinyl, your not in it for the pristine sound lol!It's the pure fun of looking back,KTel will always hold a place in my heart from the years in my childhood when I was experiencing new music,Don't be offended by the substandard sound, often asking yourself ""how did that song get on this album?""It's a great trip down memory lane!Any one of them!I'm a collector of them all... Love them",k +"Along with the great Sabicas on the record alblum: ""Day of the Bullfight,"" is the late great Pepe Segundo, who was tragically killed in an automobile accident in his early thirties...Never have I heard a greater flamenco singer with the vocal range that Pepe had...He had no peer, not even Cameron de la Isla could match Pepe's voice...If you want to hear him at his best, just listen to the bulerias (coffee shop) on this album to prove my point...Not only is Pepe fantastic to hear, but Sabicas is playing one of his fastest bulerias' I have ever heard and I have heard all of Sabicas' record albums...There is also another three pieces on the record where Pepe sings...Being a flamenco guitarist myself, I wish I had heard Pepe sing in person while he was alive...It is a shame that this ABC Paramount recording has not been reissued on a CD...This record is very rare...I already have it so I won't deprive some lucky person from buying it...I can't begin to tell you how great this recording is...There is also at the beginning of each piece an orchestral introduction of bullfight music which adds to the overall atmosphere of the recording....",N +"Got this when it came out, and am still enjoying it 20 years later. Less timbral and more in the classical western composer tradition, it has stood the test of time for me.",w +"My CD of A.R. Rahman's RAANJHANAA arrived today in New Zealand where winter snow has begun. Thanks to Rahul & I - Indian Music House. I am a huge fan of Rahman and have listened to practically all his music. This new release is wonderful, a return to more folk India - and some of the melodies feel familiar. I was delighted to hear great Sitar. Only Rahman can weave the classical Indian sound with his original and fresh creativity to perfection. Rahman always invites many listenings. I would imagine the best talent in India, like Shreya Ghosal, line up to perform with Rahman. Indeed it must be a pleasure to work with a genius composer who is also a kind generous humanitarian. What would our crazy world be without his music?I am a sap for his romantic melodies, so NAZAR LAAYE appeals to me right away. But as always Rahman's inventive layered orchestrations lie waiting to be discovered. Many of these songs are a bit reminiscent of earlier favorites, with fragrances of remembered phrases that haunt the memory. All in all a happy and enjoyable collection of songs. In praise of Rahman!",t +"This is probably my most favorite album of all time. I had a copy, but 'someone' managed to lose the sleeve. I'm so happy to have this, and it was lovingly packaged. Thank you!",S +As noted by others this is the real deal here and the sound quality is amazing over the standard Legend of 1900 CD missing the extra songs. Well worth the price.,o +"Scuttle me bones, me buckoes, 'ere be more salty, seafarin' shenanigans from the gun deck of the Pieces o' Eight... a little on the Bilgy side...As the proud owner of the both of the band's DVDs, I can tell ye that the BP's are as good visually as they are on album; so I can only urge you to go onto Youtube and play the videos to The Mermaid, Bonny Ship the Diamond and the outrageous Ranzo Ray (watch for the golf ball and the garden hose...)Step lively there ye swabs, and get yourselves this fine album. One of their best, by a long sea mile...",A +"Our Song... as another reviewer rightly said, is a high point on the album, and this is quite possibly the only place to hear it on CD at this time.Jack always delivers, and this is a very strong collection of songs. Opening with one of my personal favorites, A Day In The Life Of A Fool, Jack treats us to some classic covers, including the Bacharach-David masterpiece Wives And Lovers, Lennon and McCartney's Michelle and McKuen/Brel's If You Go Away.Jack has never been afraid of taking chances, and his jazzy treatment of Michel Le Grand's I Will Wait For You, is just one of the delights from this Classic Jack era.Sadly, this does not contain my fave Jack cover And I Love Her, but I could forgive anything of a man who flew out with the USO to share his genius with the troops in Nam.But then, that's Jack.",j +"Scuttle me bones, me buckoes, 'ere be more salty, seafarin' shenanigans from the gun deck of the Pieces o' Eight... a little on the Bilgy side...As the proud owner of the both of the band's DVDs, I can tell ye that the BP's are as good visually as they are on album; so I can only urge you to go onto Youtube and play the videos to The Mermaid, Bonny Ship the Diamond and the outrageous Ranzo Ray (watch for the golf ball and the garden hose...)Step lively there ye swabs, and get yourselves this fine album. One of their best, by a long sea mile...",B +"Our Song... as another reviewer rightly said, is a high point on the album, and this is quite possibly the only place to hear it on CD at this time.Jack always delivers, and this is a very strong collection of songs. Opening with one of my personal favorites, A Day In The Life Of A Fool, Jack treats us to some classic covers, including the Bacharach-David masterpiece Wives And Lovers, Lennon and McCartney's Michelle and McKuen/Brel's If You Go Away.Jack has never been afraid of taking chances, and his jazzy treatment of Michel Le Grand's I Will Wait For You, is just one of the delights from this Classic Jack era.Sadly, this does not contain my fave Jack cover And I Love Her, but I could forgive anything of a man who flew out with the USO to share his genius with the troops in Nam.But then, that's Jack.",S +"Very good, very pleased.",x +Great album that arrived in great condition and sealed. I love History and am glad I can still collect some of their albums even though they’ve disbanded.,q +Gave it as a gift to my mother and she loved it! Included all her favorite songs! Great seller too!,u +Came with everything that was listed! Packaged safely in a box with plenty of bubble wrap and arrived quickly from South Korea. I pulled Yonghoon and Dongmyeong for the photocards ^^,k +"Solid material from one of the realest and most talented polish rappers Lukasyno. Exceptional bars delivered over hypnotic trap beats, experimental mixed with classic hiphop sound. As always strong lyrical content any adult hiphop head can relate to. Great album. 5 stars. I recommend.",m +Love "This is Soul Kirtan" and CC White!,s +Love ot,[ +Happy with purchase!,O +Great collection of original recordings.,M +"Has ""Bad Girl"" on it that was featured in the movie ""The high note""You won't hear this kind of deep soul anywhere else.",Y +"Expensive, but it was out of print. It still was worth it!",c +It was every bit of what I expected.,K +Beautiful DVD.,c +Purely subjective as music is. Two tracks. Slim case.,Y +The Soundtrack is really a Score:1. Intro / What’s Up (:37)2. Headless Granny (1:18)3. Subway Kills* (1:03)4. Transformation (1:44)5. Possessed Room (1:20)6. Hospital (1:27)7. Rainy Window (1:25)8. The Chase (3:10)9. Charlie Shows Castle (1:12)10. The Mangles (:45)11. Kaz And Denny (:59)12. Fly To The Castle (:58)13. Sonja’s Call* (1:03)14. Flashback Sequence (1:00)15. Fuse (:48)16. Doggie Talk (:54)17. Finale (5:56)18. Chip Returns / Fixers’ Tricks (1:12)19. Sonja’s Kiss (:42)20. Voodoo Shop† (:27)21. Sonja’s Plan (:43)22. Dynamite / Kaz Saves Denny (1:12)Missing the following songs:"Let Go" (Theme from My Demon Lover) by Intimate Strangers"No Comment" by The Shangs"Snake Charms" by The Shangs"Chemical Reaction" by John Nuelin,J +"Bought this for 'Andromeda', which is also known as Odyssey Part 1 by Johnny Harris (which you can look up an listen to on 'u-toob'). This is the song played by the band in the 'Space Rockers' episode and is great, but it's electronic disco and not rock. Aside from this song and the opening/closing music, the rest of the 3 cd's contains the fluff music from a handfull of episodes. A few of those fluff songs are o.k., but I'd never have bought this if the 'Andromeda' song wasn't on it.Liner notes are great. Cover artwork is great. Inner holder for the middle disc is too tight, which wouldn't be a problem if I hadn't wanted to actually play the cd.The music, itself, is very clean and has a great dynamic range for its age. No hisses, snaps or pops. Original reels were either fantastic or someone spent some time restoring the sound.Technically great. Music... depends on how much you enjoy the show or if you're looking for a particular song like I was.",y +"I wanted the 'I Hate You' song, that wasn't on the original soundtrack. Only a couple other decent tracks on the cd, but nothing that was worth listening to (just my opinion as I would have given it a 3.5 out of 5). The first three movies had far better music scored for them. Liner notes were great. Cover art is great.Better soundtracks (orchestral pieces):2046 (Mixture of old and new / European and Asian)The Cook, The Thief, His Wife & Her Lover (depends on your ears)BladerunnerBrainstormDonnie DarkoAkira (Japanese)AliensBatman (Original score)Battlestar Galactica (the new series)Bleu / Rouge / Blanc (French trilogy... Bleu is the best of the three)Braveheart / Gladiator / Lord Of The Rings (Epic type music)Dune (Toto)E.T.Ghost In The Shell (Movie... although the t.v. series is good too)Halloween (Who knew J. Carpenter could create a catchy tune)HardwareKoyaanisqatsi (P. Glass)The Good, The Bad And The UglyLabyrinthMishima (P. Glass)Resident Evil (Marylin Manson stuff)Risky Business (Tangerine Dream stuff)Star Wars (Original and Empire)Lola Rennt (German)The Abyss (My college room mate really hated this cd)The Fog Of War (Documentary P. Glass)Terminator (Original)Tron (Original)",j +In good shape,R +Sounds good!,l +It’s an awesome CD of prince. Prince had a awesome band. These are awesome songs to listen to all day. I love Diamonds and Pearls. As well as 1999 and When Doves Cry. Would not mind listening to this all day. If not other CDs Prince and the Revolution did.,q +"It's good music. It is also good ""meditation music."" Or music to use if someone is not feeling good, if not even listen to while laying down relaxing. I was hoping to have the two songs when Daniel and his girl friend there were in it. And when the song when they went to the dance. I was expecting for those to be in it.",\ +"Poetry Of The Negro is one of the best Poetry on Vinyl Records around. Sidney Poitier's rendition of The Creation by James Weldon Johnson was absolutely Superb!! Together Sidney Poitier and Doris Belack mastered the poetry in a way that made me, one who has not always understood it fall in love with it. This is a rare gem that I would definitely recommend.",B +This is a delightful story about Snoppy Cat owned by Marian Anderson. The tale is a classic and little heard about. Adults as well as children will be in awe of Ms. Anderson taking the time to capture her audience through spoken word reading. She has left a timeless legacy for generations to come about the mischievous exploits and adventures of Snoopy Cat. It is a real treasure.,A +"I discovered this artist on Youtube, so I bought a CD that every song made me teary.Truly love his voice, its so calming and tender.",W +"This came bent and covered in dirt, very upset as it was ordered as a gift.",D +Absolutely Awesome live album from Faith No More. Highly recommended listening.,U +Can't believe nobody reviewed this gem by Joan Baez. It was a favorite from early childhood. I scored a new RSD 2012 copy from an seller for about $12.00 + shipping on Amazon a few months ago. The remaster sounds really good but the vinyl is the thin 110 gram variety that warp easily so be warned. The songs are wonderful as is her young voice on this classic record that sounds much more natural than the bright CD.,\ +I am so happy with my purchase. I got everything that was promised. They packaged it really well. Not a single thing was damaged. Definitely recommend.,C +I like all of them the photo book look so amazing and the case look so fabulous too and the pcs were awesome i totally luv it so much i enjoy Weny album alot,v +I like the album I got everything I got and totally luv the stickers it's so amazing Dang this is my first time seeing this album and I was so amazed by it,Y +Her music commentary is sharp and to the point from a British angle. Picture Julia Child or Margaret Thatcher as a music history professor.,U +"I saw Herbie Mann perform live in the 1970's. It was one of my first exposures to live jazz and I loved it. This music reminds me of movie soundtracks from the early 60's. The stronger, firmer Mann jazz didn't come until Memphis Underground and later explorations into Afro-Cuban sounds. The sound is OK, but I feel like I'm waiting my turn in the dentist's office.",J +"whoah ... what a treat! I just got back my Speedy J CDs -- on loan to DJ Dragonfly -- an he plops this Saafi Bros CD on my player. An now it is stuck! In repeat. The CD has informed my hand that it isn't allowed to touch the eject button for a long time to come. Which is odd, given that there are a dozen new tasty treats awaiting their turn for a spin. What can I say?Well first, take equal parts Kruder, Dorfmeister, Maus on Mars & add a bit of Trent Reznor low rez bass-o-matic for spice (but not too much) you'll end up with much of the same sort of groove.The perfect companion to writing code.",h +Maybe the Best Compilation.,X +love David's work,H +Quit your ridiculousness.,e +Sister was very happy with her Christmas gift. Came well packaged and played well. It was in perfect condition.,I +"Another great set of 66 songs from unknown bands, bringing you power pop. Good price for being exposed to all these bands. I would not say every one of the 66 songs is a keeper, but there are no stinkers in my opinion. Where are you going to find a 66 song compilation to give you 66 new bands to listen to?",Y +"Funny and brilliant! Sue Fink writes highly sophisticated, catchy songs with hilarious feminist lyrics. Check out her "Boys Are Thugs", for example. ("They're out to steal your candy, so they can buy more drugs…") Polished with quite a range of moods.",N +"This album including ""Oh-oh God, hear the cry of your people"" ""I'll praise you Your Name"" ""Songs of deliverance"" , ""Lord of the heavens"" etc are one of favorite all time songs since I first heard them in 1993",N +I purchased this for my parents as a gift and they loved it so much that they requested I buy more copies for them to give as gifts to friends and relatives. The vendor handled the extra order expertly and quickly and nice packaging.,h +Bough this as a gift for my parents who loved it so much they asked me to order more copies for them to give to friends and relatives as gifts. So happy that they are enjoying the music and the seller was very nice - handled the extra order beautifully and packaging was very nicely done.,H +Bought this for my parents at their request as a gift - they absolutely love it! They had me order them several more copies to give to friends and relatives as gifts. I was very pleased that they are so happy with the music. Would definitely order again from this vendor.,c +"Loved it. Big fan of her and her husband's podcast anyway. If these lips could talk is pure comedic brilliance. The cover doesn't hurt either. She is hot and funny, her husband must have been a saint in a previous life.",z +"I've been a fan of Incubus Succubus for some time.The music is spiritual and entertaining. It speaks to the soul, and the heart. Church of Madness was the song that made me a fan. Call out my Name and Pagan born are two of the best. The rest is a strong variety.",T +great,W +THOMAS S. MCDONALD,r +THANKS FOR MANY YEARS OF MUSIC,j +"THANKS PINK FLOYD FOR YOUR'E MUSICIANSHIP, THOMAS . MCDONALD",J +"THANKS, Thomas S. McDonald",d +"Thanks for the music and to David Gilmour cheers, Thomas S. McDonald",A +"Thanks, Thomas S. McDonald",D +"Thanks Later, Scrat",J +"This is very special to me, Thomas S. McDonald",K +"This album is full of good classics, Black Sabbath is one of the earliest bands I ever listen to and still do, Thanks, Thomas S. McDonald",G +"The only song missing was midnight train to Georgia,by Glady's Night and the Pips. I decided to buy also. the 70's 20th century masters collection: the milellium collection. Thanks for the product, later,update even though I had them both i decided to give the 70's 20th century masters collection to, Target Stores, for reissue, and 70's Gold first disc to 100.3 fm and the second disc to 95.5 fm, Thanks for this, Thomas Stephen McDonald, previous name, current name, Thomas S. McDonald, later, Take it Easy",l +"What did I drum on for led zeppelin to sample, Rock and Roll, part of Celebration Day, and the title track. Just maybe if NASA took my drumming along with the music from my old address under water, in PG County in Maryland in 1995 in my peak of drumming skills, at the address of 87 Watkins Park Drive Upper Marlboro, Maryland,20774, The United States of America, Thanks, Thomas S. McDonald of Led Zeppelin The Song Remains the Same",_ +I am sure it is just probably as good on other format.later,W +Best 2pac Cd hands down,x +"I have purchase a CD from this doctor before and it is amazing how wonderful his CDs are! If you are looking for music that will truly relax you, purchase this set. The first one I bought over 10 years ago and played it 8 to 12 hours a day at my Day Spa. I never grew tired of it and clients were always so relaxed. It's not just another run-of-the-mill relaxation CD, this is the one you are looking for...",F +"This worked exactly as they said it would. It calmed my dog during the windstorm and the thunder. I also use it at night as I find it restful too. My only complaint is that it flashes but it’s playing so even when you use it at night you have to cover it with something, or it’s annoying.",\ +Very talented duo.,A +my massage clients love it,q +"Came as listed in ad, and came fast!",X +Bob Fitts live worship material got me through some very dark times in the 80's. Being able to find his work again was a wonderful surprise. Hiswonderful songs still bring joy and hope.,T +"What can I say here that others haven't? Awesome band, awesome music, awesome price. Only complaints I can think of are I wish it came with a booklet and lyrics, and scratch-resistant sleeves.",o +Great CD!,V +Absolutely love her ! And this Album's fantastic!,` +Returned...Don't waste your money.,R +"I've known Wendy for over 20 years after first seeing her singing in NYC's Grand Central Station. She has the most beautiful, most crystal clear pristine voice I've ever heard. I own most of her CDs and this one (""Silky"") is one of my favorites. The track ""Go and Catch a Falling Star"" is especially haunting & beautiful. Every person for whom I've played her music has reacted the same way: ""Wow!!!""",u +Easily one of the greatest space rock albums I have ever heard.,v +Great CD. Love it,[ +awesome!,w +We were first introduced to this CD in a dance class and are happy that we now own it. The music is varied in style and tempo so that you don't get bored. It lets your creativity flow.,K +"it's wonderful and loads of great songs on this cd, but the sound quality is not the best. The sound is low so you'll need to raise up the volume. Still I am glad I purchased it because there are some songs on here I could not find on other discs, such as "I don't wanna hang up my rock n' roll shoes", "Caldonia" and "Willie and the Hand Jive."",a +"This album is excellent from the first song to the last. If you enjoy the black crowes, then you will enjoy this album. The sound is awesome, and the lead singers voice is one-of-a-kind. Some songs tend to be raunchy in their subject matter, but the way the music is put together is awesome. I truely believe that if the Black Crowes had not exploded onto the music world in the way that they did, then this band very easily could have exploded in the same way. The sound is excellent.",v +The BEST Queen CD! Really encompasses their true musical and vocal talent!,B +My husband loved this !!!,P +"Oh my good, can never say enough good comments about elvis gospel",M +"I remember seeing Deep Purple at Fillmore West in the '60s and went to Tower Records the next day to buy this album, glad to have it back in my collection!",A +"Some with backup band and some solo, this reminds me of the Moody Blues when they were at their best. After Pinder left the band, it was Hayward that focused the band's direction and wrote a string of hits.",a +a definite must!!! for all!!! it will leave you in absolute adoration to our Lord Jesus and humble you to worship Him.,i +Another wonderful CD from BTC!,V +"Fantastic, incredible voice, all over great CD.",` +I grew up with this music- now my kids can hear it!,f +"I personally got to see Smokey Pleacher in 1970 when I was 7 years old.Wow!!, what an impression he left on me.Smokey Pleacher's vocal range is unbelievable.To this day, I have not been able to find any singer that can match the capability of his vocal accuracy.",F +"I personally got to see Smokey Pleacher in 1970 when I was 7 years old.Wow!!, what an impression he left on me.Smokey Pleacher's vocal range is unbelievable.To this day, I have not been able to find any singer that can match the capability of his vocal accuracy.",k +"A friend gave me a cassette tape of this album back in the day when it was new. I lost it and have been searching for it ever since. Full of music that gives glory and honor to God, it is outstanding! The fact that the songs are sung by various artists that made the songs great is even better. I found in vinyl form, and that's alright be me. Just to have it again in my praise and worship library is worth every penny!",I +Very nice CD...talented Artist. Very soothing and beautiful old style Mariachi music. I like a lot !,i +Sounds tinny. Really disappointed in the album..,D +"My fiancé told me about her grandfather who performed as Senor Cortez and His Italianos. She had no idea how to get his albums he had recorded back in the 50s and 60s. I searched the internet and found two on sale through Amazon. I have always had success shopping on Amazon, so I made the order for "Italian Holiday: Senor Cortez and His Italianos." It was the first of two I would order. The album condition was just as described and it arrived on time. My fiancé was delighted. Her grandfather's music was amazing. Thank you for another family treasure!",^ +"My fiancé told me about her grandfather who performed as Senor Cortez and His Italianos. She had no idea how to get his albums he had recorded back in the 50s and 60s. I searched the internet and found two on sale through Amazon. I have always had success shopping on Amazon, so I made the order for "Italian Mandolin Madness." The album condition was just as described and it arrived on time. My fiancé was delighted. Her grandfather's music was amazing. Thank you for the family treasure!",P +Matt Redmon. Inspired. Beautiful.,^ +"Unfortunately this is another movie that won't play in my region, sure wish I had known about this region thing as I am quite disappointed about getting this movie and not being able to watch it, if a return label could be sent via u. s. mail that would let me exchange for one that does work here would be great. if an e-mail was sent I could provide u with my mailing address. Your prompt attention would be greatly appreciated!",P +Gotta love it! Sent it to my sister since we already listen to it most mornings. (3 of the songs on it at least!) Great orchestration as well as the vocals done by men who really know the ONE they are singing about!,] +This is one of Zakk Wilde's better projects. I like this earlier stuff more than his efforts with Black Label Society. More raw and emotional,p +Nice CD but I still think their first CD is better. They seem to reuse the same formula. I do like it but I was hoping for some real "hit" material,a +I thought it would be the songs of Maria Grever; instead it was a pianist playing the music with the lyrics!!! It's not what I thought it would be and I returned it for a full refund. That's what I like about Amazon...they are very fair with their customers.,P +Most beautiful music,V +DON'T LIKE THESE ALBUMS INSPIRED BY A MOTION PICTURE. GIVE US THE ORIGINAL MOTION PICTURE SCORE,p +I enjoy watching them and so does my 99 1/2 year old client. He loves watching the band preform,d +love it,\ +The song is great but I don't know what else is on this cd. Can't be paying 25. dollars for one song. how about listing if anything else is on this cd.,J +Got what I wanted. Wish there were more of this groups music to purchase.,B +"This is a departure from traditional Christmas Music, but that is what I like about it. Great sound from a Great Group!!",X +"This is one of my favorite Jars of Clay recordings, along with ""Good Monsters"".I know their debut CD was really good, too.But when I bought ""Much Afraid"", I was hooked.The first seven or eight songs are great, but I like it all!!",F +Great cd,g +Great cd,Y +"This is an amazing praise work! Little children singing in English and Afrikaans and praising the Lord in many many songs. As always, Michael Tate is amazing, but he does not lend any help to the children until the fourth or fifth track. And that’s OK.",v +Arrived in a timely manner and in new condition,` +Songs from the time of Jesus!,i +"I have been a Beatles fan for a long, long, long time...and these alternate versions of songs is just fine with me!",Z +MARTY GOETZ IS GREAT!,K +I ABSOLUTELY LOVE TO LISTEN TO MARTY GOETZ'S MUSIC!!!!!,h +I really miss these type of songs and music in The Church.,x +LOVE THIS ARTIST AND INCITES INTO THE STRAUSS REPETOIRE. DIFFINATIVE!!!!!!! A VOICE OF COLOR IND INTERPRATIVE GENIOUS!!!!!!!!!!,S +great cd,] +"What a voice, great cd",\ +sounds great,g +Arrived as described. Well packed and promptly shipped. Thank you!,F +"This is an AMAZING praise cd. I listen to it whenever I'm in the car. Tracks 5,6,7 are my favorites",h +"...The arrangements are almost other worldly, almost like those 60's Italian Sci-Fi movies. To me the music is so cool ( different ).",y +UNITED KINGDOM'S FINEST FEMALE SUPERSTAR VOCALIST OF THE 60'S.,i +Boring. Guitar sounded repetitious in all songs. Wishes there was more creative picking rather than just harsh strumming. Great potential though.,n +He escuchado la música de Roger Williams por más de 25 años. Es uno de los más destacados y excelentes pianistas modernos que ha tenido nuestra cultura. Su maravillosa creatividad y ejecución musical le garantizan un sitial especial para siempre. La producción musical de Roger Williams fue numerosa y la muestra compilada en estos CD nos permite disfrutar y atesorar el maravilloso talento de este singular pianista. La calidad de las grabaciones son excelentes. Estoy contento con esta compra.I have listened to the music of Roger Williams for over 25 years. He is one of the most outstanding and excellent modern pianists that our culture has had. His marvelous creativity and musical performance guarantee him a special place forever. The musical production of Roger Williams was numerous and the sample compiled on these CDs allows us to enjoy and treasure the wonderful talent of this singular pianist. The quality of the recordings are excellent.,Z +"You might imagine my relief, punk rock not being my favorite genré, that the songs on Ugly Cherries are clever, funny, gay, and the music & musicianship is very listenable and competent. (Probably in a few more listens i'll think they're excellent.) I had already liked their moxie. I also like that they do raunchy drag: don't shave their body hair, don't have gym bunny bodies, don't give a fig what people think (oh yeah, that's moxie).I ordered this CD because of a scandal threatening to deep six the band. Recently i read the accused categorically denies the charges (which i'd figured were possibly bogus) and the duo are plowing ahead. Maybe they'll break through into the mainstream, or maybe they'll remain obscure with a loyal core of fans. I'd categorize their style on UC as punk folk rock, which is why i like it. Who knows what they could evolve into. To reiterate, these 2 guys do have considerable talent.",w +"I greatly admire Neil Young and have followed his career since almost its beginning. This CD contains little known songs of his, in concert. It seems he can do no wrong (artistically speaking). He's a good person, as well, and I have read his well written autobiography. (I had never heard the soundtrack, "Where the Buffalo Roam," included on the disc, so experiencing that was also a great treat.)",\ +Bought for a stroke victim...she loves them!!!,E +This isn't the soundtrack. The soundtrack is what I was looking for. I purchased this CD by mistake.,c +really old but OK,S +I have had this CD for several years.After chatting with a friend decided to purchase one as a birthday gift.,a +"Back in the late 50's and early 60's where the music on the 4 CD's hails from music-companies withthe emergence of 'vinyl 45's' realised that a whole new generation of potential customers were outthere including a large following from the fairer-sex who in the post-war era were becoming far morerelevant in the workplace therefore having spending powerThey realised that good-looks would sell records, it was a bonus if the emerging idol could also sing,this was to be the age of 'The Teen Idols'This release of course does have artists that had both appeal and sound vocal ability....I myself bought 45's released by all four featured artists though since the introduction of CD's have neverowned a 'Frankie Avalon' album...Till now that is.Disc One - Elvis...there is very little that hasn't already been said regarding 'The King'In little over 20 years he left a legacy that has long outlived his far too short life and will i'm sure prevailfor many many years to come.The tracks featured on disc-one include many of his late 50's and early 60's releases including several chart-toppers, along with one or two fondly remembered B-Sides, album tracks and indeed one or two songs thatmany remember from his movies -Disc 1 Elvis PresleyJailhouse Rock(Let Me Be) Your Teddy BearAll Shook UpBaby Let's Play HouseA Big Hunk O' LoveToo MuchLawdy Miss ClawdyBlue Suede ShoesDon't Be CruelHound DogThat's All RightMystery TrainRip It UpHard Headed WomanI Got StungHeartbreak HotelIt's Now Or NeverMy Wish Came TruePartyReturn To SenderStuck on YouSurrenderKing CreoleDon'tAre You Lonesome Tonight?She's Not YouWooden HeartFollow That DreamGood Luck CharmLove Me TenderFrankie Avalon was born in Philadelphia in 1940He started his show-biz career at an early age, but in terms of chart success broke onto the scene in the late50's first featuring in the U.K Chart in 1958 with Ginger Bread (reached '30') his next U.K success came thefollowing year with 'Venus' (Chart '16') though it actually topped in the States.Most of his recording success was indeed in America, however his number 'Why' did peak at number '20' inthe U.K and became his second U.S Chart-Topper.He was also known to turn his hand to acting playing a role in 'John Wayne's' - 'The Alamo' (which i own) aswell as several TV roles in America.The album enclosed includes all of his U.K hits along with many of his U.S Chart Entries -Disc 2 - Frankie AvalonVenusWhyDede DinahJust Ask Your HeartA Boy Without A GirlGinger BreadAll of EverythingYou Are MineDon't Throw Away Those TeardropsI'll Wait For YouTogethernessSwinging On a RainbowWhere Are YouA Perfect LoveBobby Sox To StockingsYou Excite MeThe Puppet SongDon't Let Love Pass Me ByTwo FoolsWhat Little GirlDianaHoneyJust Say I Love HerSecret LoveShe's Funny That WayTrouble With Me Is YouA MiracleTrue True LoveTuxedo JunktionWho Else But YouRicky Nelson was born Eric Hilliard Nelson in Teanek, New Jersey in 1940.Ricky also started his show-biz career at an early age featuring on both the radio and later T.V with his parents.As a solo-artist he broke into the charts here in the U.K in 1957 with 'Stood Up' (Chart '27') the following yearthe song 'Poor Little Fool' (Chart '4')(U.S. '1') was to signal quite a run of hits for him this side of the Atlanticmany of which are indeed featured on the disc on-board including 'It's Late' (Chart '3' 1959) - 'Hello Mary Lou'(Chart '2' 1961) and the same year 'Travelin' Man' (Chart UK '2')Ricky also had a turn on the acting front (which many a young pop-idol of the day seemed to do) he had a role(which i remember well as i have the film) in 'Rio Bravo' with 'John Wayne' and 'Dean Martin'Sadly 'Ricky' died quite young in 1985 when a victim of a plane-crash.Disc 3 - Ricky NelsonPoor Little FoolTravelin' ManYoung WorldTeenage IdolBe-Bop BabyStood UpNever Be Anyone Else But YouLonesome TownBelieve What You SayA Teenager's RomanceIt's LateJust A Little Too MuchHello Mary LouI Got A FeelingA Wonder Like YouYoung EmotionsEverlovin'You Are The Only OneI'm Walkin'Waitin' In SchoolMy Bucket's Got A Hole In ItI Wanna Be LovedYou're My One And Only LoveI'm Not AfraidHave I Told You Lately That I Love You?Yes Sir, That My BabySweeter Than YouMighty GoodRight By My SideSummertimePaul Anka was born in 'Ottawa' in 1941Singer/Songwriter 'Paul Anka' started recording at just 14 years of age, his first U.K success at the age of 17 wasin fact his only Chart-Topper this side of the Atlantic 'Diana' (1957)The album on-board includes all of his best known and fondly remembered U.K hits during 1957-62 including (inaddition to 'Diana') - 'I Love You Baby' (Chart U.K '3' 1957) - 'You Are My Destiny' (Chart U.K '6' 1958) - 'LonelyBoy' (Chart U.K '3' 1959) and 'Put Your Head On My Shoulder' (Chart U.K '9' also 59') among the many.He recorded the first version of 'Puppy Love' back in 1960 which he also wrote, it peaked at '33' here in the U.K, itmade the number '2' slot in the States, many i'm sure will remember 'Donny Osmond's' 1972 version.His chart-success in America spanned a much longer period of time in America, here he had little success after 1962until his release of 'Your Having My Baby' in 1974 which reached number '6' in the U.K (Chart-Topper in America)The album on-board a great reflection of his Golden-Era...Disc 4 - Paul AnkaLonely BoyPuppy LoveDianaPut Your Head On My Shoulder(All Of A Sudden) My Heart SingsLove Me Warm And TenderCrazy LoveI Love You, BabyI Miss You SoIt's Time To CryJust YoungLet The Bells Keep RingingMidnightThe Teen CommandmentsYou Are My DestinyAdam And EveCinderellaDance On Little GirlEso Beso [That Kiss!]Every Night (Without You)Hello, Young LoversI'm Coming HomeI Love You In The Same Old WayKissin' On The PhoneMy Home TownSomething HappenedA Steel Guitar And A Glass Of WineThe Story Of My LoveSummer's GoneTonight My Love, Tonight",q +"For me, all these multi-artist Compilations are worthy of a glance, though unlikely if a collector you'll find any difficult-to-find recordings,what you will find is a collection of in the main Top-Ten-Hits from the early sixties in unfamiliar play-order which when listening to so manyfavourites from those times is always a pleasure.Back in the late fifties and early sixties competition for chart-supremacy was tough for' the U.K artists as there were indeed so many American-idols hogging the Top-Spots.Many U.K artists released cover versions of many of the Smash-Hits material the other side of the Atlantic (several examples of which can befound upon this 4-CD release.The collection offers a sample of hit-material on alternate CD's from both sides of the pond such as......'Brian Hyland' - 'Sealed With A Kiss'.......'Dion' - 'Runaround Sue'......'Brenda Lee' - 'Here Comes That Feeling' (saw little-miss Dynamite live back in the early 60's, quite a performer).....'Tommy Roe' - 'Sheila' (always felt he should have had more hits than he did)......and......'Chubby Checker' - 'The Twist' (always fell-over trying)......just some of the great sounds from the U.S to be found upon disc one......and on disc two......'Ricky Valance' - 'Tell Laura I Love Her' (thiswas one of the cover-versions of a U.S hit, 'Ricky' topped the U.K Chart with it)......'Eden Kane' - 'Well I Ask You'......'The Allisons' - Are You Sure'(came 2nd in the Eurovision-Song-Contest 1961)......'Brook Brothers' - 'Warpaint'......and......'Tommy Steele' - 'What A Mouth'......again just a sampleof the Brit-Artist Hits that can be found on disc two's track-listings......on disc three......'Ray Charles' - 'Hit The Road Jack'......'Duane Eddy' - 'DanceWith The Guitar Man' (always a favourite of mine)......'The Drifters' - 'Save The Last Dance For Me'......'Ketty Lester' - 'Love Letters'..... .and......'TheHighwaymen' - 'Michael' (many will remember Lonnie Donegan covering that one)......some of the many great sounds from the U.S that graced theU,K Charts to be found upon disc three.....disc four, back to U.K-Artists......'Adam Faith' - 'Poor Me'......'Marty Wilde' - 'Rubber Ball' (a cover of BobbyVee's smash-hit - both made the U.K Top Ten)......'Anthony Newley' - 'Do You Mind'......'Frank Ifield' - 'Lovesick Blues'......and......'Kenny Ball' -'Midnight In Moskow'......again just an example of the many great sounds that can be found on disc four........Glancing at the track listings i reckon i bought the vast majority on vinyl way back when....i have listed just a few of the many great memories thatCcan be found on each of the four discs to offer a feel of what's on-offer on this Collection.88 favourites from the 60's-era - gotta be worth checking out surely.TRACK-LISTINGS -Disc: 1 1. Only The Lonely (Roy Orbison) 2. Sealed With A Kiss (Brian Hyland) 3. Cathy's Clown (The Everly Brothers) 4. Take Good Care Of My Baby (Bobby Vee) 5. Three Steps To Heaven (Eddie Cochran) 6. On The Rebound (Floyd Cramer) 7. Runaround Sue (Dion) 8. Poetry In Motion (Johnny Tillotson) 9. Nut Rocker (B. Bumble & The Stingers) 10. Here Comes That Feeling (Brenda Lee) 11. Last Night (The Mar - Keys) 12. Happy Birthday Sweet Sixteen (Neil Sedaka) 13. (I Wanna) Love My Life Away (Gene Pitney) 14. Love Me Warm And Tender (Paul Anka) 15. Travelin' Man (icky Nelson) 16. Sheila (Tommy Roe ) 17. Venus In Blue Jeans (Jimmy Clanton) 18. Blue Moon (The Marcels) 19. Jump In The Line (Shake Shake Senora) (Harry Belafonte) 20. Shout (Parts 1 & 2) (The Isley Brothers) 21. Johnny Angel (Shelley Fabares) 22. The Twist (Chubby Checker)Disc: 2 1. Tell Laura I Love Her (Ricky Valance) 2. The Young Ones (Cliff Richard & The Shadows) 3. Bobby's Girl (Susan Maughan ) 4. What Do You Want To Make Those Eyes At Me For (Emile Ford & The Checkmates) 5. Well I Ask You (Eden Kane) 6. Up On The Roof (Kenny Lynch) 7. You Don't Know (Helen Shapiro) 8. Image of a Girl (Mark Wynter) 9. Wild Wind (John Leyton) 10. Starry Eyed (Michael Holliday) 11. The Man With The Golden Arm (Jet Harris) 12. Are You Sure? (The Allisons) 13. I Remember You (Frank Ifield) 14. Jealousy (Billy Fury) 15. Norman (Carol Deene) 16. Ain't That Funny (Jimmy Justice) 17. Warpaint (The Brook Brothers) 18. Be Mine (Lance Fortune) 19. Marry Me (Mike Preston) 20. Shakin' All Over (Johnny Kidd & The Pirates) 21. What a Mouth (What a North and South) (Tommy Steele) 22. Telstar (The Tornados)Disc: 3 1. Hit The Road Jack (Ray Charles) 2. Green Onions (Booker T. & The MG's) 3. Duke Of Earl (Gene Chandler) 4. Running Bear (Johnny Preston) 5. Dance With The Guitar Man (Duane Eddy) 6. Hey Baby (Bruce Channel) 7. Speedy Gonzales (Pat Boone) 8. Runaway (Del Shannon) 9. Save The Last Dance For Me (The Drifters) 10. Roses Are Red (My Love) (Bobby Vinton) 11. Let There Be Drums (Sandy Nelson) 12. A Hundred Pounds Of Clay (Gene McDaniels) 13. Devil Woman (Marty Robbins) 14. Stand By Me (Ben E. King) 15. V - A 16. Love Letters (Ketty Lester) 17. Never On Sunday (The Chordettes) 18. Please Mr. Postman (The Marvelettes) 19. Two Lovers (Mary Wells) 20. Will You Love Me Tomorrow (The Shirelles) 21. Shop Around (Smokey Robinson & The Miracles) 22. Michael (The Highwaymen)Disc: 4 1. Wonderful Land (The Shadows) 2. Poor Me (Adam Faith) 3. A Picture Of You (Joe Brown) 4. Johnny Remember Me (John Leyton) 5. Walkin' Back To Happiness (Helen Shapiro) 6. Ya Ya Twist (Petula Clark) 7. Tribute To Buddy Holly (Mike Berry) 8. Ain't Misbehavin' (Tommy Bruce & The Bruisers) 9. Rubber Ball (Marty Wilde) 10. Cindy's Birthday (Shane Fenton) 11. Tower Of Strength (Frankie Vaughan) 12. Do You Mind (Anthony Newley) 13. Time (Craig Douglas) 14. I Wanna Go Home (Lonnie Donegan) 15. Angela Jones (Michael Cox) 16. Lovesick Blues (Frank Ifield) 17. Wimoweh (Karl Denver) 18. Moon River (Danny Williams) 19. Portrait Of My Love (Matt Monro) 20. Reach For The Stars (Shirley Bassey) 21. Midnight In Moscow (Kenny Ball) 22. Stranger On The Shore (Acker Bilk)(Hope this helps)",X +"As is often the case the 'Now' music series continues to deliver collectable 'Theme' releases in addition to theirever popular year-on-year numbered offerings.There are many albums of this type out there to choose from, and as is often the case many tracks are repeatedtime and time again, for me, it's all about hearing so many favourites in a unfamiliar play-order.I did buy and review a 'Power-Ballad' album quite recently, and yes many of the tracks found on that one can befound on-board this release, however the selection overall does come up with many different numbers in additionto the 'Usual-Suspects' which, if you love these foot-stomping albums makes this 3-CD Compilation surely worth checking-out.I will simply highlight a few from each of the discs to offer a flavour of what can be found among the tracks on offer.On disc one........'Phil Collins' --'Against All Odds'........'Heart' --'Alone'........'Take That' --'Rule The World'........'Bonnie Tyler' --'Total Eclipse Of The Heart'.......,'Cutting Crew' --'I Just Died In Your Arms'........and........'Texas' --'I Don't Want A Lover'.......just some of the great sounds to be found among the 19 track play-list on disc one.......On Disc Two........'Coldplay' --'Yellow'........'Travis' --'Why Does It Always Rain On Me'........'Shania Twain' --'You'reStill The One'........'Miley Cyrus' --'Wrecking Ball'........'Starship' --'Nothing's Gonna Stop Us Now'........and........'Michael Bolton' --'How Am I Supposed To Live Without You'........again a random selection of the memorable tracksfrom down the years that can be found among the 19 on disc two........On disc Three........'Survivor' --'Eye Of The Tiger'....'Alice Cooper' --'Poison'........'Tina Turner' --'The Best'........'Crowded House' --'Don't Dream It's Over'........'JeffBuckley' --'Hallelujah'........and........'Genesis' --'Mama'.............some of the 18 numbers to be enjoyed on disc three....I have simply highlighted around half a dozen tracks from each of the three discs to give an indication of the qualitythat can be found on-board, my selection was purely random there are obviously many more equally worthy of amention, for me, having so many great sounds from down the decades on one release is always worth checking out....",q +"When I started collecting CD's back in the 90's duplicating my Vinyl collection bit by bit was the target, obviously it,had to be done gradually, sometimes making choices carried one or two regrets, such as, why didn't I pick that upwhen I had the chance.One of these instances occurred with a Dave Clark Greatest Hits Album, had it in my hand, put it back because I'dalready picked up several to buy, thinking of course that I could pick it up another day, well, when I felt the urge todo so, couldn't find it, in the event it seemed to disappear of the scene, despite several searches couldn't find a 'DaveClarke' greatest hits CD , until this releases in 2008, obviously I didn't think twice.......again.The Group formed back in 1957 first calling themselves 'The Dave Clark Quintet' however after one or two line-upchanges the group changed their name to 'The Dave Clark Five' in 1962.The groups first hit was 'Do You Love Me' a number also covered by 'Brian Poole and the Tremeloes' which restrictedtheir Chart-Entry peaking at number '30' in late 63' however their next release 'Glad All Over' actually knocked the 'Beatles''I Want To Hold Your Hand' off the Top slot a little later the same year.Among some of the other U.K chart successes 'Bits And Pieces' (Chart '2' 1964) 'Can't You See That She's Mine' (Chart'10' 1964) and 'Catch Us If You Can' (Chart '5' 1965) they did achieve several lower-order Chart-Entries and had a resurgenceof their fortunes a few years after their last Top 10 success with songs like 'Everybody Knows' (Chart '2' 1967) 'Red Balloon'(Chart '7' 1968) 'Red Balloon' not on-board this release......'Good Old Rock'n'Roll' (Chart '7' 1969) and 'Everybody Get Together'(Chart '8' 1970)The Group proved very popular in the U.S in truth having greater success than here in the U.K...........the group disbanded 1970.This album contains most of the groups best known tracks along with one or two I remember as 'B-Sides'Inducted into the Hall of Fame in 2008",L +great CD..awesome,R +I bought this CD single for the bonus tracks - Animal Logic was much better live.,f +This is just a regular CD. I waited 2 weeks for a huge disappointment !,k +Pleased with the item,\ +Simply wonderful! It is always pleasant to hear the instrumental...I liked it and enjoying in my free time while I am travelling!,C +"In February 1968, the Metropolitan Opera presented Verdi's LUISA MILLER for the first time since 1930, with a principal cast that included Met veteran Richard Tucker and relative Met newcomers Montserrat Caballe and Sherrill Milnes. One performance of the run (the matinee broadcast?) was preserved on CD. While Tucker was in good voice for his age at the time (50), the youthful, musical, and affecting singing of Caballe and (especially) Milnes make this recording a keeper; Milnes absolutely shines here. Giorgio Tozzi and Ezio Flagello as the villains are as good as they were on the complete, studio LUISA of 1964, with Anna Moffo in the title role. The remastered, "live" sound is astonishingly vivid. If not a first-choice LUISA (it has cuts), this set is a must-have for fans of Caballe, Tucker, and/or Milnes.",w +"[[ASIN:B01K8ML9G8 Sherrill Milnes in Recital, Vol.1 [IMPORT] by Sherrill Milnes (1997-01-29)]] In the best tradition of American operatic baritones, Sherrill Milnes had a voice that was rich, resonant, round, and robust, with a smooth legato, ringing high notes, and a lovely mezza voce. In the 1980's that voice -- in its prime during the 1960's and 1970's -- experienced gradual decline due to a vocal-chord injury Milnes sustained in 1981. KINGDOM BY THE SEA presents the best of Milnes' recital singing recorded between 1980 and 1990. Few of the tracks on the disc show evidence of serious vocal decline, and even those that do also suggest that Milnes' artistry never diminished during his later years.Posa's death scene from DON CARLO and the Champagne Aria from DON GIOVANNI are the only selections from Milnes' operatic repertoire on the CD; the remainder is comprised of rarer opera arias (e.g. Gretry's "O Richard" from RICHARD, COEUR DE LION) and the German, English, and American songs that Milnes so loved. The Gretry aria sounds almost as though it could have been written for Milnes, so well does it suit his voice; the same might be said of the Gerald Finzi songs and Loehr's charmingly funny "The Little Irish Girl," which draws laughter from the (Carnegie Hall?) audience. The most magical track of all, however, is #19: a song called "Duna" by Josephine McGill. Recorded in 1980, it shows Milnes' voice -- and particularly his mezza voce -- at its most hauntingly beautiful, his talent for communication at its very best. Milnes' friend and accompanist, the late Jon Spong, provides adept piano support in every track. If you are a fan of Sherrill Milnes, then you need to have both KINGDOM BY THE SEA and its "companion" disc, THERE BUT FOR YOU GO I, in your collection.",V +"[[ASIN:B01K8MWNP4 Sherrill Milnes in Recital, Vol.2 [IMPORT] (1997-10-02)]] This CD is a compilation of some of Sherrill Milnes' best recital work from the years 1980 to 1990. The American baritone, in his prime in the 1960's and 1970's, had a vocal-chord issue in 1981 (when he was forty-six) that led to a relatively early vocal decline. However, only tracks #3 and #18 really reflect this decline; and those tracks both show, if not the most beautiful tone, then the superb legato that always was a hallmark of Milnes' style."Nemico della patria" from ANDREA CHENIER is the only operatic aria on the disc, the remainder being comprised mostly of French art songs and Milnes' beloved oratorio repertoire; the comically Handelian song "Old Mother Hubbard," written by Victor Hely Hutchinson in the twentieth century, is also present. Though the two Baroque oratorio selections that open the disc find Milnes in wonderful voice (these were recorded in 1980), the clear highlight of the CD is the set of four Serious Songs by Johannes Brahms. Captured in 1984 or 1985, they show Milnes at his peak: serious vocal decline had not yet set in, and his artistry was fully mature. Most impressive is the fourth of the songs, "Wenn ich mitt Menschen," which sounds almost as though it could have been written expressly for Milnes' voice. But to hear a side of Milnes that was not frequently apparent in opera, go to track #19, "Old Mother Hubbard," and delight in the droll (and agile!) performance that draws laughter from the enthusiastic, Carnegie Hall audience.Milnes' friend and regular accompanist, Jon Spong, provides adept piano support in all of the tracks. THERE BUT FOR YOU GO I is a valuable document of the non-operatic, later-career Sherrill Milnes. If you are a fan of his, then you need this CD.",m +"I originally got this CD free from Tilex. I used it in my dance classes for cool down exercises. Some students said it was their favorite part of the class. Then I became careless, carrying it loose in my bags, and it broke. Because the package gives no names of artists or composers, I did not know how/where to find a replacement. I was delighted to locate it through Google to Amazon, and once again my dance students are enjoying moving to this exceptionally beautiful, restful music. I recommend it to anyone seeking a serene musical interlude.",W +What beautiful music coming from the heart of each one.,X +Did not come with CD case!!!! Cheap cardboard wrapped in celophane!,e +Excellent.,c +It is very pleasant for listening and relaxing.,O +I like Enya and This music was no disappointment.,R +Some of the finest Icelandic folksongs. Brought back great memories of my time there,h +Love it@!,n +You can trust this Goodwill with their descriptions.,` +"As the title suggests, we keep having to buy this DVD for others the minute they see it! We've gone through several copies now and just keep passin' the faith right along to others (complete strangers who love the Gaithers or just love the CD or DVD. Myself, I prefer the DVDs. This is a favorite out of the 50 or so DVDs we have of them!",b +"There are NO CD recordings of the Largest Theater organ in the world! I tried to buy ANY recordings at Radio City Music Hall. They had NONE! Maybe someone should publish some, or better MAKE some new ones!",C +This recording is taken from the original 10 inch LPs which where issued in the early 1950s. The sound quality isn't up to todays standards but the performances are first rate and they are the ones I grew up listening to. It's also great to have the original vocal performances for Quo Vadis. A great selection for Rozsa fans.,i +This is well worth the money. Lots of great Christmas songs. Songs sung by Crosby that I grew up listening to. I enjoy the fireplace CD as well. It will bring warm memories into any living room.,h +"I was a little disappointed that this CD arrived in a thin cardboard jacket rather than a jewel case, but the CD is superb and I definitely enjoy the music🎶",P +"La fea más bella (The Most Beautiful Ugly Girl – 590 minutes)We are talking about an uncommonly unattractive young woman who is uncommonly smart. Lety (Angélica Vale) keeps trying to get a job but is consistently bumped by buxom blondes. Our gal has braces on her teeth, bad hair, a unibrow, a terrible fashion sense, ugly glasses and poor posture.Through a fluke that discloses how bright she is, she gets a job at an ad agency but is stuck in a back room so clients won't see her. With her post-graduate degrees and excellent education, our homely heroine quickly becomes indispensable. The two most handsome men (Juan Soler and Jaime Camil) fall for her because she is so different. She ends up owning the company and loving both men; she just can't make up her mind.The most consistent comic relief is an odd couple that consists of a brainy but homely man (Luis Manuel Ávila) and a very, very buxom blonde (Patricia Navidad). We also enjoy the world's worst spy. His cell phone goes off regularly.... This is laugh-out-loud funny at times but is a bit too cartoony. Which I can forgive because it really IS fun.",n +great source,s +great,B +great,p +without recording defects,y +without recording defects,i +Manufacturing defects present on CD; multiple breaks in music recording. I desire a refund. I will forward CD if refund is granted.,D +The blending of their voices and the way they respond to each other in a performance adds to the listening/ viewing enjoyment,l +GREAT,W +"Super Mexican Music, One of the Best Mexican Singer Ever Natural VoiceEnjoy! special Esclavo y Amo ,Que Bonito Amor From one of the Best Mexican Opra",s +"not as good as #2...but still excellent work by these talented artists ...apparently they are not making this CD anymore.If you want uplifting Christian hymns, this is the best I've found....some of the sound isn't as good in the later editions (e.g. 3,4,5)",S +"the record was scratchy, not in a very good condition",h +My Favorite unknown band. This album is excellent & is my second favorite of their releases.,l +"This is by far the best of their albums, not to slight the first two. They were great also but this one drives. I'm almost exhausted when I finish listening to it. I saw them live in the 60's at an anusment park in Oklahoma City. They were opening for Mitch Ryder They got so many encores that Mitch finally came out himself and said no more, It was time for him to play. Needless to say Mitch was second fiddle that night. They were doing a lot of soul and Motown style music, I suppose because they were touring with Mitch. One of the songs that blew everyone away was "Reach Out (I'll Be There)" Which they performed better than the Four Tops could ever hope to do. The absolute best unknown band I've ever seen. They were actually better that most known bands. Wish there was more nusic available from them.",[ +What's not to like?! It's Johnny Mathis!,] +These are such wonderful songs for preschool age children to learn about the lord. I cd was so happy to find a copy and the cds are in great condition,u +A blast from the past. Great CD,l +Very satisfied!,q +The lettering on the case and CD was english but the music was greek. I sent it back. Sneeky!!!!,q +Tennessee Ernie Ford is great.,F +I have read the book at least six times in the 25 plus years I have owned it. I checked out the audio book version several years ago from my public library and really liked the experience of hearing my favorite book read by a very good English narrator. I finally purchased it recently and am presently listening to it in my car on the way to and from work (two hours total). What more can I say? This novel is a true masterpiece!,x +"I was a tad apprehensive paying $34 for a used CD but I took a chance & like 99.99% of my orders, rather from Amazon or used from Amazon Sellers it paid off! I have this CD on cassette bought back in 1986 but I no longer have a player. It arrived sooner than estimated & im thrilled about this also!! Thank you to the Seller for a used CD that is as perfect as new!! Your description was 100% better than I thought it'd be! I highly recommend & will buy from you in the future! ☺",H +Love the cd it is worth the price,T +"Wow! Finally a Western made for the adult mind! Thank you Hollywood for a job well-done!What a delightfully complex film about motivation, relationships and values. It's about a father and a son on the verge of manhood, and another son with an illness; a bad guy who has a strong sense of self and truly knows how to lead; about a society that condones the mass murder of women and children as long as they are Native Americans but not of armed robbers stealing from what was then one of the most powerful corporations on earth - the railroad industry.It's also about a society which tolerates racism towards hardworking Chinese laborers building the railroads; and at the same time allows working-class, Caucasian farmers to be brutally pushed off their land in order to make the powerful even richer. But the only real ""bad"" guy society is organized against is the one who steals from the rich and powerful! How incredibly convenient and ironic!This story is also about pride, self-respect, intelligence, courage and even self-confidence. We are also confronted with senseless violence committed by evil men on one hand and the immediate need for self-defense on the other. What's worth risking your life over? Is the central question 3:10 to Yuma asks.There's so much going on this amazing film, there's no way to absorb it all the first time around! What a breath of fresh air to watch such a clever story!",a +Good songs but a little shrill towards the end of songs ...and would be better with more outstanding songs,W +Wiser to buy it from one of the stateside sellers,y +Play it constantly,l +Have a collection of Terry Macalmon's CD's. I am in the process of acquiring all of his that I can.,F +I loved this album and some of tunes I'm finding ever so dear to my heart causing me to cry,] +"Fernando Ortega is one of my favorite hymn singers. His gentle and warm voice is so soothing, it fits the subject of the Lord's love and faithfulness perfectly.",I +Beautiful and peaceful,Z +Great training by Jim Rohn,S +God blessed Donna. And He blessed us with her talent.,q +I have always liked the music by this man and I am very glad I have this CD.,M +like this CD a lot it has all the music that I want on it. Thanks,E +"I really like this CD very much. it has some great songs on it. I think anyone can enjoy this, I believe any Elvis fan will like this CD.",v +"This CD has a lot of the TV Themes I remember and some I don't remember listening to growing up, but thanks anyway",R +"I had this CD from several years ago. I really wanted to get another, because I really enjoyed this CD so very much with all his big hits. I think anyone can enjoy this CD. Its the greatest",I +This is a great CD with aa lot of great songs.Thanks,i +This is a great CD by the King. I love all the songs on this CD. Thanks,D +As I have said before anything by the King has to be great and this one is thanks,H +This is a great CD. Everything I expected from the King of Rock and roll. Thanks so much,Y +This CD is just great like I thought It would be. as far as I concerned Elvis is the greatest Singer there is. I am so sorry that he is gone,N +This CD has some great songs on it. I enjoy listening to it very mmuch. Thanks,G +I really like this Cd. It has a great range of gospel music that Elvis recorded. I think its great,^ +This CD has some of the best music on it from the 40' and 50' and it great. Thanks so much,K +I may be purchasing another copy as I am wearing it out!,\ +great cd,f +great,s +Love love,H +Always outstanding. Shipped quickly,V +"I love John Denver, what a loss we suffered when he was murdered!!!! I love his songs, and they brighten my day. His music is timeless, and I hope my grandkids will keep this after I'm dead and gone, and realize what good music was!! Here's hoping!!!",y +"Some of the songs are good for quiet reflection, the rest are good for congregational settings",j +Not the whole cd,W +Love that this is on audio. It helps to focus.,L +DJ Kaoris is awesome. English music. Buy it,y +"I bought all 10 CDs (1970-1979) for use at my high school's multi-class reunion. They worked out great, music played for hours, without the expense of a band or DJ. :D",j +"I bought all 10 CDs (1970-1979) for use at my high school's multi-class reunion. They worked out great, music played for hours, without the expense of a band or DJ. :D",M +Muy buena presentacion le.encanto a mi hija,J +"This package included a number of classic albums that I didn't already have. For me, it was a good deal even though I owned a number of the albums included. Please note that many of Emmylou's CD's are re-mastered with additional tracks. The same album if included in this package will feature the original song list, without the additional tracks, and (I'm not sure) probably not re-mastered. I'm still pleased to have bought this compilation, but wish that this would have been made clear in the Amazon product description. As to Emmy Lou and the content here, there aren't enough stars to give her.",g +The CD is good for your sold.,W +"My old friend returned home, very nice.",q +"I had a vinyl copy of this for many years, and loved listening to it. It you have never seen or heard the Smothers Brothers, you are in for a treat.",B +"I LOVE Michael W. Smith. I LOVE old hymns. I enjoyed this album. However, I prefer many of his others. That said, this contains some great old songs.",z +I don’t know I just like you album,I +"This music is so beautiful, it takes your breath away.",l +ok,D +"Beautiful music, Elvis was a wonderful gospel singer and I never tire of listening to this collection of his gospel favorites.",i +"This is beautiful "REAL" Christian music, so unlike modern "Christian" music.",z +A very hard to find Pedro Infante music ! Was very good condition and happy to have investedIn this legendary music album! Very lucky to have been able to have shared it with my grandchildren!,A +"This CD has many great and inspiring songs. I especially like "Blessed Are You, O Lord Our God."",c +Had no idea it was an original pressing. Fantastic service and totally reasonable pricing.,l +"Amazing and very underrated band. If you like original post-punk, buy this.",f +Just love this album. So many great songs. He's got a nice sound that soothes the soul.,e +"If you are interested in music of the Codex Calixtinus or of the Camino de Santiago more broadly, you should consider this album. The content is one of the masses contained in Book I of the Codex and it is superbly produced. Among other elements, the production places the singers in positions (relative to the microphones) that would have been used in the actual mass, resulting in the sensation of being there.",Z +"CD was delivered Unscratched. I've been looking for the CD version of LIA for years and finally found it. The music is just like I remembered it, southing.",l +"The fact is, too many people can't accept when a band grows in another direction so this album and the great songs on it is not the issue, the real issue is the fans who live in a little box from the 80's. Did they really expect Cherry Pie in 1996?Jani Lane grew as a songwriter and this album shows that and it's excellent.And if you are stuck on the 80's, listen to ""Letter to a Friend"" on repeat, it's classic (and awesome) Jani Lane songwriting. AYM is slamming as are many of the tracks after a few listens.And Jani still has one of the best voices on earth so there's that too.",r +Expanding Nat's collection,n +Great album,C +good music,p +Best pre mix cleaner you can buy. Highly recommend for a reason. Make small batches and store in brown containers out of light. The growler bottles I never refill are perfect for storage. I use an okki Nokki. I don’t notice any sonic difference if I rinse after suction.,g +Awesome,X +Another first for Lynda,f +"I'll update this review after I have time to listen to this album, which I assume I'll at very least like, so 4 stars for now. I want to note immediately that the tracks on the album I received (Arista -- catalog #ABM 2008) are not the same as those listed in the review by Joe. Here is the track listing for mine (updated 12/11/14 because the tracks actually on the album are ordered differently than as listed on album jacket):Side 11. Drivin' Wheel2. Lord I Felt3. It Don't Matter Now4. When I'm Home5. I Think I Love You Again6. MelodicSide 21. Midnight Rider2. Billy3. Dear Me4. Where Men Don't Care5. A Good Old Time Love Song",c +"This is my favorite of her collections. I have been looking for it for years as once I had the cassette form of this treasure. I kept the cover of it that came inside the cassette till I could find it again in CD form and discovered Amazon. Expensive "FOR SURE" but well worth it if you love her stuff. The main reason is that it contains her classics and other songs you may have not heard form her but will fall in love with once you hear them, because they are preformed live and in ENGLISH as well as French. Again a TREASURE!!!",N +Nice to have the true A's and B's in one place!,g +"While there are former Beat Farmers in this band, this is NOT a Beat Farmer album.",A +"Many of you that found your way to this page know that April Wine was all over the radio for a few years. This live set ""Future Tense...Live"" is a great snapshot of this time which was likely the pinnacle of April Wine's success in North America. Taken from a concert in 1985, you get the following tracks:Anything You WantFuture TenseCrash & BurnBefore The DawnWaiting On A MiracleEnough Is EnoughIf You See KayJust Between You and MeSign of the Gypsy Queen21st Century Schizoid ManI Like To RockRollerOowataniteAll Over TownYou Could've Been A Lady (bonus track)",a +"***** BEWARE as this Thin Lizzy CD may be subject to DISC ROT ( aka CD rot) ******As a long time Thin Lizzy fan, I have had this CD in my collection for years. I pulled it out to listen to it recently and found it to sound rather odd. The sound would go in and out and sound muffled. I thought it was my CD player but then found this to be the case on any CD player I used. On CDs, the rot becomes visually noticeable in two ways:(a) When the CD is held up to a strong light, light shines through several pin-prick-sized holes.(b) Discoloration of the disc, which looks like a coffee stain on the disc. See also CD bronzing.Mine has the pin holes - see attached pictures. One is a picture of the disc looking fine but when one holds a mag light up to the Cd, you see the pin holes!In audio CDs, the rot leads to scrambled or skipped audio or even the inability to play a track. This is the case for my CD making it a waste of money. I have come to the conclusion that this is due to the manufacturer, Windsong. I saw a similar thing with some Crowded House CD singles in which the digipack caused the CD to rot and become discolored and unreadable.I am NOT saying those for sale here are bad CDs but I would be fearful to spend good money with no guarantees. All of my other CDs are fine so this is the one case where it can only be attributed to the manufacturer.",B +"RARE promo CD (less than 500 made) that was given to radio station DJs for the holidays in 1995. Rare versions of songs by the Goo Goo Dolls (acoustic "Name") and Better Than Ezra ("Merry Christmas Eve") and many others.Track listing 01 The Goops - Vulgar Appetite 2:20 02 Better Than Ezra - Merry Christmas Eve 4:07 03 Dog's Eye View - Everything Falls Apart 3:58 04 Edwyn Collins - Makes Me Feel Again 4:26 05 The Chemical Brothers - Life Is Sweet 3:58 06 Ministry - Lay Lady Lay 5:50 07 16 Horsepower - Black Soul Choir 3:34 08 Throneberry - On the Strobe Flume 3:01 09 3 Lb. Thrill - Diana 3:58 10 Goo Goo Dolls - Name (Acoustic) 3:55 11 Goo Goo Dolls - Naked 4:10 12 The Screaming Santas - Jesus Christ 2:28 13 Superchunk - Yeah, It's Beautiful Here Too 3:50 14 All About Chad - Kristin 3:40 15 Blur - Charmless Man 3:37 16 Jars of Clay - Flood 3:36 17 Limblifter - Screwed It Up 3:47 18 Chris Knox - Half Man / Half Mole 2:39 19 22 Brides - A Coventry Christmas 2:23 20 Dwight Twilley - That Thing You Do 4:21",v +"Be sure to see if you are getting the import Definitive Hooters Collection 2CD version or the 1CD version. The track listing below is for the 2 CD version. A great collection of Hooters tunes and the bonus disc in the import versions is worth the price alone.Disc: 11. Twenty-Five Hours a Day2. Satellite3. And We Danced4. Day by Day5. All You Zombies6. Where Do the Children Go7. Nervous Night8. Johnny B.9. Karla With a K10. Fightin' on the Same Side11. Brother, Don't You Walk Away12. 500 Miles13. Don't Knock It 'Til You Try It14. Give the Music Back15. Mr. Big Baboon16. Shadow of JesusDisc: 21. The House of Wolfgang2. Deliver Me3. Hanging on a Heartbeat4. Graveyard Waltz5. Silent Night",q +"Be sure to see if you are getting the import Definitive Hooters Collection 2CD version or the 1CD version. The track listing below is for the 2 CD version. A great collection of Hooters tunes and the bonus disc in the import versions is worth the price alone.Disc: 1 1. Twenty-Five Hours a Day 2. Satellite 3. And We Danced 4. Day by Day 5. All You Zombies 6. Where Do the Children Go 7. Nervous Night 8. Johnny B. 9. Karla With a K 10. Fightin' on the Same Side 11. Brother, Don't You Walk Away 12. 500 Miles 13. Don't Knock It 'Til You Try It 14. Give the Music Back 15. Mr. Big Baboon 16. Shadow of JesusDisc: 2 1. The House of Wolfgang 2. Deliver Me 3. Hanging on a Heartbeat 4. Graveyard Waltz 5. Silent Night",j +"This hour-long, 16-track compilation selects the highlights from Wire Train's three Columbia albums (but not its two MCA albums) along with a few rarities.COLLECTORS TAKE NOTICE: Amazingly, early copies mistakenly substituted the B-side "Half A Lifetime" for "The Last Perfect Thing" (track #6). Ultimately, the mistake was corrected but meanwhile an interesting collector's item was created and surprisingly hard to find. Obviously, if you were lucky enough to get one of the first 500-1000 copies, you have this "mistake". The bonus is that "Half A Lifetime" is a great song and unavailable for digital download anywhere!!!ALL Music BIO:Wire Train was formed as the Renegades in April 1983 in San Francisco by San Francisco State University students and guitarists Kevin Hunter and Kurt Herr with the rhythm section of Anders Rundblad (bass) and Frederico Gil-Sola (drums). The group signed to the local 415 label, also home to acts like Romeo Void and Translator, all of which found themselves with national distribution when 415 entered into a deal with Columbia Records. Wire Train's first album, ...In a Chamber, made the national charts in 1984, but the group began to suffer personnel changes. Gil-Sola was replaced by Brian Macleod for the second album, Between Two Words, after which Herr left, to be replaced by Jeffrey Trott. A third album, Ten Women, charted in 1987. The group's last two albums, Wire Train (1990) and No Soul No Strain (1992), appeared on MCA",H +"This CD was released after the success of the Basia album ""London Warsaw New York"" in 1990. It features the following:Brave New HopeUntil You Come Back To Me (That's What I'm Gonna Do)(Phil Harding 12"" Remix)*Brave New Hope (Brave New Mix)*Cruising For A Bruising (Phil Harding 12"" Remix)*From Now On (Band Version)*Masquerade **Come To Heaven *** special unreleased versions** unreleasedFor the Basia completist, this is a must have CD.",L +"This release was a limited edition release of the two albums "Stand Tall" (1989) and "Dig Down Deep" (1988). Re-released in 2003, it features liner notes from Steve "Eddie" Rice. These albums were made after they left the major label and returned to their roots. These albums captured a more live sound of the band than the previous releases.Tracks: Stand Tall, She Lives For The Summer , Nothing Like Love, If You Were Mine , Pray For A Miracle, In Her Dream , It's About Time, Driven By Love, Days Go By, Just Gotta Rock, Never Stop Believin', Loving You, From A Distance, Don't Let It Take You Over, Light Up The Night, Nothing But A Dream, Color Me Gone, Romeo And Juliet, It's My Job",\ +"Where does one start? The latest release from the Willie Hines Band covers a tremendous amount of territory. If you want pop sensibility, you get it in songs like ""Say Hello"", ""Hippie Chick"", and ""Downtown"". If you want a straight ahead rock tune, then ""Boyd's Party"" is definitely in the sweet spot along with ""Solitare"" not far behind. If you want circa Led Zep sound, then ""The Drift"" is right there with big time nods to the mighty Plant/Page machine. If you want power-pop-rock, then ""Smile"" is the song that will get you singing the chorus immediately and wanting more! There is not a dull moment of sound on this album.Much has changed musically since Willie Hines was singing for The Lights and Jet Red but one thing remains constant- Willie Hines continues to bring finely crafted music to those who are willing to find it. With the input of Jeffery Babineau (guitar, backing vocals, and much more!), John Wise (bass, keys, backing vocals), and Michael ""Fro"" Frowein (drums, backing vocals, the album sparkles with talented musicianship. Will this album make the charts? Not likely due to the state of the fractured music industry today where great talent is overlooked while posers somehow get a seat in the spotlight. Nonetheless, if you are a fan of real music being made by real people, then this album is for you. It has so much to offer to so many people that it just might actually gain some new followers for the WHB. Don't miss out and get it today and continue to support your local music scene! Rock on!!Track listing:Say HelloSolitareDowntownThe DriftHoneymoon JunkieGreta GarboEverybody ElseSmileGypsy's GiftHippie ChickBoyd's PartyGoodbyeMy picks are ""Smile"", ""Boyd's Party"", and ""Solitare"" but then again, I just like good music from good people.............",u +"This is a great sounding live recording from the Misplaced Childhood era with Fish on lead vocals. It was recorded in Germany on June 21, 1986 in Mannheim at the Maimarktgelände.The tracks are as follows:. 01. Garden Party ........................... [04:49] 02. Freaks ................................. [04:21] 03. Assassin ............................... [06:23] 04. Misplaced Childhood (I) ................ [18:53] a) Pseudo Silk Kimono b) Kayleigh c) Lavender d) Bitter Suite e) Heart of Lothian 05. Misplaced Childhood (II) ............... [17:34] a) Waterhole b) Lords of the Backstage c) Blind curve d) Childhood's End? 06. White Feather .......................... [06:51] 07. Forgotten Sons ......................... [08:28] 08. Market Square Heroes ................... [08:28]",h +"Issued prior to the ""Heaven and Back"" release, this CD-EP contains 6 songs. They are as follows:Love Is Everywhere (single mix) 4:14Love Is Everywhere (remix) 4:13Soaring Bird (2.0 version) 4:02World On Fire (sound track mix)5:44Love Is Everywhere (love mix) 5:05To Feel This Way (sound track mix) 4:02This was released primarily through the band's record company- Fingerprint Records in very limited supply. The versions of the songs are very cool. Check it out. Michael Been, Tom Ferrier, Scott Musick, and Jim Goodwin together as the CALL is a great thing.",] +"The track listing for RRAF- Roots Rock Action Figures ""Calling Dr. Strong"": The Ionizer's solo project is as follows:Check-up Form The Neck Up, How Long Can She Last, One Of Those Nights, Twangeaux, No Great Shakes, The Rest Will Take Care Of Itself, Calling Dr. Strong, Friendly Little Game, Powerful Stuff, Heart She Can't Beat, Long Short Story, Secret Heart (1982)THE PLAYERS:Bobby Field- vocals, guitar; Les James/Jimmy Lester plays drums; Kenny Vaughan plays guitar (If you haven't seen or heard Kenny play with Kim Richey, you're in for a treat.); Scott Baggett on bassAs you can see there are several Webb Wilder tunes (as expected since R.S. Bobby Field and webb are longtime collaborators). The versions are a tad different than the one's we know and love from Webb Wilder. Are they better or worse? Hard to say. I love ""Twangeaux"" as it is an instrumental ala WW fine works like ""Horror Hayride"" and the ""Goldfinger"" cover. ""Powerful Stuff"" is the old Fab T-Birds tune that made it into the movie Cocktail. ""Calling Dr. Strong"" is an interesting song in which the Ionizer uses an electronic sort of voice. This song really rocks. Many of the songs are very mellow yet show off The Ionizer's strong songwriting skills. This one-off record is a must for any self-respecting Webb Wilder fan!",a +Track listing only:Del Amitri- Always The Last To KnowBabylon AD- Bad BloodMelissa Etheridge- 2001Phish- Chalk Dust TortureJaime Kyle- Ragged HeartBlack Sabbath- TV CrimesMedicine Men- ShineWildside- Hang On LucyIndigo Girls- GalileoThe Soup Dragons- I'm FreePeter Murphy- You're So CloseSuzanna Rhatigan- To Hell With LoveThe Believers- Extraordinary LifeThe Badlees - Like a RembrandtRAM- Has Anybody Seen The WindEnjoy!,^ +This special release contains the following songs:artist - songMatthew Sweet - Sick Of MyselfDaDa - All I AmThe Smithereens - Miles From NowhereFreedy Johnson - Bad ReputationStabbing Westward - LiesThe Connells - Living In The PastThe Auteurs - Lenny ValantinoDandelion - Weird OutMother May I - Meet You ThereThe London Suede - Stay TogetherA not too bad collection of tunes.,k +"This promotional CD has the only known released live tracks by the band. Fortunately, I have been able to uncover their live recordings and am able to hear a show from circa 1989/1990 containing Blown Away, Big Elgar, Glow, Is This Love, Man Behind The Gun, Motherlode, One Step Over The Line, Original Man, Widder's Dump, Year Zero.....but I digress.......The interview section of this CD gives some insight into how the band formed and what they intended to do. The live tracks are very cool to have as well. King Swamp lives on through the release of Walter Wray and his solo album and there is hope of more Swamp-like music to come forth as Walter is playing in the London area again as of 2007.This is a must for any Swamp fan......",v +"UPDATE 2012- The long lost second album was re-released and can be found here ([[ASIN:B006L8BSFQ Flight Plan]]). Stay tuned to jetred.com for more info as the Willie Hines Band plans for a new release as well as Johnny Feikert's solo release due in 2012.JET RED was not your typical rock band. With powerful guitars and vocal harmonies, they were poised to make a splash on the rock scene but thanks to grunge and a poor record company, they did not get the press and airplay they so richly deserved. The best songs on this album are ""Bad As Heaven"", ""Let It Roll"", ""Outside"", and ""From The Skies"" although song for song, this is a solid release that was overlooked by the radio folks. Always a great live band, JET RED lives on. Bass player extraordinaire Brad Lang is now a full time member of Y&T. Willie Hines is doing solo material and has an album that can be found on cdbaby. The website for the band is out there so do a google search for ""Jet Red Willie Hines"" and you will find it or go to jetred.comRock on-Ken",z +"GREAT!! Walter Wray was the lead singer for King Swamp. When the Swamp fell apart due to lack of label support, he moved to a solo career. This album was made with former band mate Dominic Miller. Sound familiar? Yes, the same Dominic Miller who plays guitar for Sting! The music is stunning and is a natural evolution from the Swamp.Nice melodies and wonderful lyrics make this a nice find for the Swamp fan looking for the never made third album. Of course, a quick search on line will find KING SWAMP and then a further follow up via that site will connect you to more rare Walter and King Swamp music but you need to do that.....If you can get this album, you should. Walter has re-started his musical career in the UK and is active on his my space page. In fact, he has this album available via Amazon UK autographed!!!",U +"This album released in 1987 features the great song ""Why Won't You Stay (Come In, Come Out Of The Rain)"" which charted on the radio airwaves back in the day. Great song indeed perhaps a lost classic. The video has been located and posted in the usual places. While the rest of the album is OK, it is worth purchasing for that one song!For those looking for a TRACK LISTING:1. Why Won't You Stay (Come In, Come Out Of The Rain)2. We Hear The Thunder3. Telephone4. Passenger5. Loving You Is Strange6. Twisted7. She's So Sorry8. Sugar Candy9. Devil With A Halo",N +"This was originally released in 1988 on a single CD. There is a re-release that was a limited edition release of the two albums "Stand Tall" (1989) and "Dig Down Deep" (1988) on one CD. The "2 on 1" limited edition CD was released in 2003 and it features liner notes from Steve "Eddie" Rice. These albums were made after they left the major label and returned to their roots. These albums captured a more live sound of the band than the previous releases.Tracks: Stand Tall, She Lives For The Summer , Nothing Like Love, If You Were Mine , Pray For A Miracle, In Her Dream , It's About Time, Driven By Love, Days Go By, Just Gotta Rock, Never Stop Believin', Loving You, From A Distance, Don't Let It Take You Over, Light Up The Night, Nothing But A Dream, Color Me Gone, Romeo And Juliet, It's My Job",_ +"This album, STAND TALL, was originally released by the band after they left the ATCO record label. This album has been since officially re-released by the band in a limited edition of 1000 copies pressed up with the addition of the DIG DOWN DEEP album thus making a 2 albums on 1 CD now available for the first time ever. Please note that these are in short supply and you should act now.The official web site at [...] has more information regarding this release as well as the I DO IT FOR YOU re-release as well as song samples.As for the music on this CD, it is very simple yet complex, straight ahead rock and roll. Some would call it ""power pop"". I'd call it real music made by real people. From Stand Tall to Nothing Like Love to the rest of the album. You can't go wrong.The track listing for the 2 on 1 CD is below: Stand Tall, She Lives For The Summer , Nothing Like Lov, If You Were Mine , Pray For A Miracle, In Her Dream , It's About Time, Driven By Love, Days Go By, Just Gotta Rock, Never Stop Believin', Loving You, From A Distance, Don't Let It Take You Over, Days Go By, Light Up The Night, Nothing But A Dream, Color Me Gone, Romeo And Juliet, It's My JobEnjoy!!!",_ +"UPDATE 2012- The long lost second album is now available [[ASIN:B006L8BSFQ Flight Plan]]. For those that enjoyed the first album, this will be a welcome addition. Go to the band web site for more info on ""FLIGHT PLAN"" which is an awesome slice of melodic rock and roll.JET RED was not your typical rock band. With powerful guitars and vocal harmonies, they were poised to make a splash on the rock scene but thanks to grunge and a poor record company, they did not get the press and airplay they so richly deserved. This album still holds it own years later. It is full of the sound that you expect to hear. The guitar solos of Johnny Feikert as well as Willie Hines are sweet indeed. The driving bass of Brad Lang as well as the drumming of Billy Carmassi make for a solid rhythm section.Regardless, if you are a fan of good music, please do yourself a favor and get this CD. Willie Hines has a solo album out via the web site for the band is out there so do a google search for ""Jet Red Willie Hines"" and you will find it.The best songs on this album are ""Bad As Heaven"", ""Let It Roll"", ""Outside"", and ""From The Skies"" although song for song, this is a solid release.Rock on-Ken",L +"Light weight, but the spring clamp isn't as solid as the metal screw clamps. The nut holder attachment might have worked if the nut was twice as thick.",g +"Liked the songs, sound and quick shipping. I only wish it came in one of those plastic jewel cases. It didn’t. It came in some plastic wrap thing that will last for about 6 plays before it rips and then the cardboard “case” will begin to disintegrate.",O +Nice sounding CD. Great songs. I think I’m going to get the LP version just because I love Steely Dan.,h +"The credits to who all played on the recordings are included. The sound quality is best ! I have a very high end stereo, ( Top line Coda, Legacy system). It plays these with masterful clarity and dynamic range. No mush, no compression. Even the art work is clearer than other releases.However Hey Jude is not in the collection as it was not USA.Greg Chick, Audiophile since the Beatles came here..",_ +"Terrific recording of Doreen and her fellow musicians. She is a New Orleans fixture, and listening to this music brings back happy memories of time spent in NOLA.",\ +"I failed to find the revolutionary aspect of this, but I guess it's alright. Not rocket science by any means. As a psych major, it all seems to feel the same when put into play.",_ +"I am disappointed with this album. I was expecting a variety but got easy listening. These artists have remarkable talents but they did not display too many of those talents in this album. Unfortunately, this is the first Vocal Majority album I have purchased so I cannot recommend a better one. Try to find one with classical music not popular.",h +"In 1997, Tony Blair wins the peoples' vote to become Prime Minister. The Queen doesn't like this; apparently he is a modernist. However, she must follow proper protocol and ask him if he'll be prime minister. He says yes, he and his wife back out of the room (never show your back to the queen), and then Princess Diana dies.She and Prince Philip don't think it's any big deal, but Prince Charles actually has a tear in his eye, and raves about what a great mother she was. This makes the Queen feel as though he's putting down her mothering skills. Anyway, as we all remember, the public went crazy with grief, and expected the royals to share in it. But they didn't, and that's what this whole film is about, which makes it super boring.Advisors are afoot everywhere, and Tony Blair finally talks the queen into coming back to London from her 40,000 estate in Balmoral. She happens to see a 14 point stag that her husband and grandkids have been stalking to get their minds off the death of their mother. Seeing the free and beautiful stag means something to her and she begins to cry (with her back to the camera). What does the stag mean? Does it represent Diana stalked by photographers, which some believe lead to her death? Or does the stag represent Diana as she was when married to Charles - stalked by the royal family trying to get her to change, never show her feelings, and put up with Charles having a mistress? OR, does the stag represent the Queen herself, roped into the monarchy due to her father's death, and feeling it's ""God's will"" that she is royalty, and must represent her people? Which she does by being filthy rich, being waited on hand and foot, and having 40,000 acre estates. The queen finds out later, that the stag was killed, having ""wandered"" off the 40,000 acres to another estate, where a paying guest wounded him, then a servant had to finish him off. She makes a visit the stag's decapitated body, however, it doesn't show her going to pay her respects to Di. Is the stag a substitute?What upsets the queen so much, is she can't believe that her people are making SUCH a fuss about Di's death, and that now 1 out of 4 have voted to abolish the monarchy. She feels as though she has given her life to these peasants, going around shaking hands with the poor, and they don't even appreciate it. The queen believes that Great Britain was widely respected for its decorum in the face of tragedy, and that showing your emotions is simply not done.However, to prevent an uprising and perhaps losing her castle, she bitterly and against her better judgment gives in to Tony Blair's (who has gone to the other side and become her champion)advice, goes back to London, looks at some flowers that mourners have placed there, and takes very personally the hand-lettered notes (""your blood is on their hands""). Bummer.Well, it was all a very unpleasant affair, but now the Queen can go on about her business, without having to worry about Di causing any more headlines.",x +"This is an album that's usually part of an audiophile's collection because it was recorded well. I like Dire Straits but not a diehard fan so I like about half of the songs here, but I end up playing the entire album just because it sounds so good! If you're not a big fan you might lose interest after the hits play - ""So Far Away"", ""Money for Nothing"" and ""Walk of Life"".",T +Rockin' Out !!!,o +Completes Collection.,t +AWESOME !!!,h +Excellent book on how the body heals itself. Easy to read on modern day discoveries of the cell,z +"The Cover Art of the Real Fanderson Version of the Space: 1999 Year Two CD is NOT pictured here. The genuine Fanderson - Space: 1999 Year Two cover art is completely different than the crap shown here. The real Fanderson version of Space: 1999 Year Two CD has the following on its back. The CD information states... This album has been digitally remastered from the best possible analog source material. Every effort has been made to minimize noise and distortion inherent in the original recordings.The Fanderson Space: 1999 Year Two collection that contain more original tracks contains TWO CD's. The cover art for the real Fanderson Space: 1999 2 CD set has the fanderson logo on the CD's back cover. The front cover art shows a rather expansive view of Moonbase Alpha looking from the outermost base and buildings toward the Main Mission \ Command Center hub. Slightly to the right of the Command Center hub in the sky above Moonbase Alpha is a green nebula. Closer in are two reddish orange planets one larger - closer the other smaller further away. the Space: 1999 logo is emblassoned in yellow between Moonbase Alpha and the green nebula. In a purple border above the cover art picture is printed in white, ORIGINAL TELEVISION SOUNDTRACK. On a lower purple border is printed in white YEAR TWO.In the lid cover assembly can be seen the words... THE WORLDS OF GERRY ANDERSON. There are numerous elements and artifacts that mark the legitimate version of the Space: 1999 Year 2 CD set from imposters. Most obvious the real Fanderson Space: 1999 Year 2 collection includes 2 CD's not one. The real Space: 1999 Year 2 collection includes a full color booklet showing scenes and background from select Space: 1999 episodes that relate to the music content offered on each of two CD's. The booklet is the highest possible image quality using glossy paper. The images are crisp and clear. Each of the two Space: 1999 Year 2 CD's is covered on one side by a full picture of the moon. On top of the lunar picture is the Space: 1999 logo in bright yellow. Underneath the CD's Bright yellow Space: 1999 logo is a reflective void in the CD spelling out YEAR TWO with a number 1 inside a circle near the bottom edge of CD1. Likewise for CD 2 of the set except the Circle on the lower most edge of CD 2 in this set has a reflective number 2 in a circle. These pictures of the Moon are NOT labels that can be pealed off. The Whitish gray moon pictures are heat bonded to the surface of the CD in a process that makes them effectively impossible to remove without damaging the respective CD's. Finally the best identifying quality of the 2 CD Space: 1999 Year 2 version is the art of the back cover. On the best quality 2 CD Space: 1999 Year Two CD set there is a picture of an Eagle Transporter hovering just above the lunar surface near one of the Nuclear Waste Domes with tanks in the foreground. The Disk One and Two music list and times are also viable. The Fanderson logo is in the lower right hand corner. If you see all these things you likely have a good 2 disk Space: 1999 Year 2 TV Series Soundtrack if not blast outta there at full thrusters and use booster units if available.In the 2 CD Set Space: 1999 Year 2 collection ""The Taybor"" is well represented and very nicely done comprising a full 19.15 minutes of CD number 2. This is an awesome CD set if you want the entire suite of soundtrack music from Space: 1999 Year 2. I consider this 2 disk Space: 1999 CD set one of the crown jewels of my Space: 1999 collection and were what was on offer here the real thing it would be a steal at $399.00. I paid almost $200.00 for my 2 CD Fanderson authentic Space: 1999 Year Two set and wondered at the time was I foolish to pay so much. I must say any doubts I used to have about having paid too much for my rare 2 CD, Space: 1999 Year 2 set are long gone. It is rare if ever that I see the genuine Two disk Fanderson Space: 1999 Year Two set on offer anywhere. I hunt auctions for others who would love their own authorized two CD disk Fanderson Space: 1999 Year 2 set and I found another set once 7 years ago.I write this review as a warning to good people. There are LOTS of fake offerings out there that suggest they contain the entire expanded genuine Year 2 TV soundtracks. The real Fanderson expanded editions contain tracks never heard on the actual Space: 1999 Year 2 series. I just don't want good Amazon.com customers getting burned by people saying they have the real two disk Space: 1999 Year 2 set when at best all they offer is a cheap substandard often incomplete imitation. The over $300 dollars being asked is a lot of money to part with to end up disappointed and maybe even cheated. The quality of the genuine Fanderson Remastered Space: 1999 Year 2 CD's is nothing short of excellent. Caveat Emptor (Buyer Beware!) Good people do not let yourselves be ripped off.If this CD was the real thing it would be a steal but given it is not its Dreck. (Garbage!) I write this review because; shopping on Amazon.com is made better for everyone when we all share insights and information that helps fellow shoppers make more informed buying decisions while avoiding pitfalls. I love Space: 1999 and I don't want it getting a tarnished reputation due to lame or bootleg products.",O +This sounds like it was recorded on a cheap cassette Walkman in someone's pocket. It is not a legit live recording. I loved this concert when I saw it but I cannot listen to this. Complete waste!,u +My grandparents had been looking for a certain song from Hank Williams and it was on this set. They absolutely love it!,d +Forever Changes is of the best albums from the 1960's. This is one of the few Sgt. Pepper concept type albums that works. A truly overlooked masterpiece.,E +I still like to hear Hank enjoy this very much,e +"I can't believe they only have two tracks on this CD!!! And the second one is loaded with cursing and can't be played in mature adults presence, or very young.",` +"I have really enjoyed this album of great, relaxing music. Can't recommend it highly enough. But then, I'm just a guy.",S +"Excellent sound quality. What distinguishes this show from Live Dates I and II, and the Live In Tokyo show, is that Powell and Turner were here experimenting with pedal effects (talkbox and wah) in a prominent way, rather than playing their normal clean Allman Bros.-like lines. It's a good show but the effect songs (3) just don't come off that well, particularly the talkbox in Jailbait, which at times when used as rhythm sounds like static or feedback distortion. Still, there some really enjoyable performances here.",a +Why on earth would this Cd Single be so hard to find. Some parts have been added to compilations adding only 1 minute or 2 minutes of the song. Not enought to enjoy the pleasure of the song. A song that is so full of inspiration. It makes you wanto getup and dance...it shines a light where there is darkness. Do we need to pray to find this CD ? Maybe. This tune diserves more than just a moment in time.,j +Excellent CD of religious songs.,q +This is an excellent CD of religious tunes. It is not an album of traditional gospel music but rather more modern songs of faith. David Patillo has a great voice and is a very good singer. Most of the songs on the CD were written by him. The selection and balance of songs makes it a very pleasing CD to listen to. I highly recommend it.,t +Great seller!!! Great band!!! The amazing Blue Öyster Cult!! In the day! Thank you!!,t +"Graham Bonnet, need I say more, what a line up of songs!",l +Everything was great. Thanks again.,h +Great music. Fast delivery. Thanks.,^ +Thanks for the fast delivery. Nice record.,j +Love the music. Thanks for the fast delivery.,h +Great record. Thanks for the fast delivery.,B +Beautiful music. Quick delivery. Thanks.,L +Another great record. Thank you.,^ +Love the music. Thanks. Fast delivery.,r +"A lot of historical facts, that make for an interesting background to Christ's historical AND religious significance.A perfect Lenten listening for two old snowbirds, on their way, driving back north.",q +there are exactly what I expected,R +"Love Kevin Hoo - always beautiful, soothing music that I listen to daily. The Christmas music is outstanding.",^ +"Thanks!!, but not what I expected.",M +I feel like I know these songs from a past life. Wonderful music that really hits a spot no other band has ever hit.,c +I was excited when the band reformed and started recording again. Smashing pumpkins is known for there unique style of rock. It’s what made them what they are. This album throws out everything they’ve built and flushed it down the toilet. Creating a experience that is bland and down right boring. No rock what’s so ever. Nothing here but a Billy corgan solo album. Which was far far from the rock alternative sound into synth sounding album that’s not worth the time or the money. Nothing memorable. Truly awful album.,z +"This album was like having the Abba and Jesus speak over me thru the Holy Spirit, who I am in Him and who He is and His heart for me. Meeting me in the depths of my struggle. The lyrics to every song touch my heart and the music takes it to a place of soaking and praising. At least that was my experience. Very thankful for this album. It was a Godsend for me!",D +Steffany is one of my all time favorite praise singers. Her music is packed with truth and heart and talent. Such a pleasure to listen to these tracks. It often takes right into Abba’s embrace.,K +Heard the re-done tracks and said "hey that's not bad". Heard the originals and said "there's the Michael Jackson I'm looking for". The originals have a vintage Michael Jackson feel to it and I like that vibe. I ended up buying only the individual original tracks only and paid less than $12.00 on the Amazon app.,E +"One of her last recordings (2005) before retiring to Canada, when it comes to female guitarists, Ellen Mcilwaine is legendary. The sound is crisp. recorded live at Japan's Yellow. Backed by bass & drums, Ellen chooses to go the electric route on this offering. Includes a 17 minute funky 70's medley. If you can find a copy of this, enjoy!",R +"I have been unhappy with the 'Motown Greatest Hits' Series. I found their sound quality to be muddy, distorted, or in mono. Volume 1 of this 'Promo Only' EMI Box Set, had superb sound, with most tracks being in their original stereo mixes. Volume 2 here, has quite a few more tracks in mono, but once again, the stereo tracks are the best I have heard. If you love Motown, but are picky about sound quality, these box sets are a worthwhile investment.",p +"I have always been disappointed with the Motown Greatest Hits Series. Most of the selections sound 'muddy' or distorted, or in mono. The tracks here in this 'Promo' 4 CD Box Set from EMI Publishing, has the most pristine sounding Motown I have heard to date. In Stereo! There is a Volume 2, so I just ordered that. Highly recommended, if you are fussy about sound quality.",v +"After seeing and hearing Tyler in the documentary, 'ROCK PROPHECIES', I am thinking to myself, this kid could be the hope, for the future of the blues.So much for that idea, after hearing these tunes.Thank God we got Bonamassa.I wish Tyler all the best, as this is obviously the path he wants to pursue.But, I still think, he should be down in Chicago, kickin' butt with Buddy Guy...",p +Fond memories on a good soundtrack,n +LOVE THEM..that man sure can play piano and sing for Jesus,b +"Excerpts from Dylan's iconic mid-1970s Rolling Thunder Revue, featuring live performances of "Shelter from the Storm," and a 10-minute blistering version of "Idiot Wind." Much of this material is also available on "Hard Rain," another live CD from the iconic tour. Dylan -- in a burnoose -- is throaty and hard-charging. The sound mixing was widely criticized on these recordings, and you'll hear why.",M +I have loved Brook Benton's music for a long time and I look forward to hearing more of it.,F +I bought as a gift and she loves it. Nice selection of songs.,b +I used to see him at the Renaissance Faire in Bristol Wi. Just finding this CD on Amazon is Amazing . This CD is Absolutely Beautiful. Fall In Love with Billy Miller ! I have Again ! !!!,J +cd skips,O +"Wow.First off, the writing is incredible. Next, the story is better than good. This is not a formula novel. I think that anyone could read (or listen) to this and connect with the main character on some level. If you've been up in the world; if you've been lonely or feeling empty; if you have loved; if you have hurt people - it's all there. I almost felt like the narrator was telling the story that a lot of us would tell some part of if we were honest enough. This is one you will want to go back to again and again. I listened to the audiobook, but am going to be getting the hardback (for my bookshelf) *and* a paperback (to read and re-read).It will do no good to tell you anything else about this book because it's one that will mean something different to each individual reader.",s +"I own this CD and have played it for many of my friends who are Veterans from Vietnam, Desert Storm, OEF, and OIF. The response I get is ""Where can I buy this?"". I've even been told one Vet wants the song played at his funeral. It is a very moving song. Well worth the money for a single song!",u +"I purchased this because I really enjoy Simon and Garfunkel and I wanted to hear what they sounded like before they became famous. The music on this CD is more 50's light rock than folk. If I had not been listening to a CD by Simon and Garfunkel I would not have recognized either of them in many of the songs. I did hear elements of what they were to become in a few of the songs and that was nice.The Songs on this CD are from 1957 to 1960.The song list is as follows:1. Dream Alone2. Beat Love3. Just A Boy4. Play Me A Sad Song5. It Means A Lot To Them6. Flame7. Shy8. The Lone Teen Ranger9. Hey School Girl10. Our Song11. Teenage Fool12. Dancin' Wild13. Don't Say Goodbye14. Two Teenagers15. True Or False16. Just To Be With You17. Dreams Can Come True18. Pretty WordsThe CD does not come with any extras explaining when or where the songs were recorded. It would have been nice to have a paragraph or two talking about the early years of Simon and Garfunkel.This music is nothing like the Simon and Garfunkel of the 60's. Still, if you are a Simon and Garfunkel fan it might be worth hearing what they sounded like before they matured and developed into the greats that they became and still are. Overall I am glad I purchased this.",e +"We love all of Tim Janis' music, but this one is a new favorite because of our Irish-Scottish heritage. Truly beautiful, relaxing & with a libation, it will carry you away to the beautiful British Isles.",E +good product,N +We heard this artist on Sirius Radio and decided to try something by him. We really enjoy the cd and are glad to support the artist.,a +"This import from the UK is a must own for anyone who loves the old train songs. The collection is a crazy bargain, 200 somngs for just $20. The collection includes legends like Elvis Presley, Johnny Cash & Hank Williams right along side the obscure like Hawkshaw Hawkins, Timy Bradshaw & Lonnie Glosson.",] +"The March Violets are probably best known from the Some Kind Of Wonderful soundtrack. This 1985 release is a decent collection of 80's English goth rock with ""Walk Into The Sun"" and ""Eldorado"" standing out as the best tracks.",y +"Franke & The Knockouts achieved a top ten single in 1981 with the infectious ""Sweetheart"". Hailing from the New Brunswick, NJ club scene, their music styling is rock but with an ear towards the pop side. ""Sweetheart"" is the catchiest and best song they released, but other solid efforts include ""Without You"" and ""You're My Girl"".",R +Great music.,D +It doesn't sound as good as the original mp3s I got off the old real radio web board it does have cleaned up audio,a +a,C +Heard several of these in person while at youth camp in Colorado. I was reading my scriptures and the word bridegroom appeared which reminded me of a song in this CD. I'm in my 30s now and wondered if I could find the CD again. Hadn't crossed my mind in years. I was surprised when the singers name popped into my head. I wasn't even sure I had it right. Wish I had $20 to buy it again.,n +Great music from one of the greats. If you like Chet Baker this is a must-have.,c +"Good value, Some great songs",` +"I've loved Morgana's singing ever since I heard her, sometime in the early 60's. Two of my all time favorite tunes of hers are "Bluesette" and "All Blues." Bluesette was "supposed" to be on her CD "With a Taste of Honey," but wasn't. After some research, I found out that my two favorite (but missing) tunes were on an identical CD but with slightly different printing on the CD cover. So, I had to pay for a new CD which did have those two extra tunes I so much wanted. Why they did that, I have no idea.",m +"Very nice recording with just a few exceptions. The level of the live cannons used seems to be set way too high! Now, I know what cannons (and artillery) sound like but WOW, even the grill cloth on my speakers vibrate outward about an inch or so and that's just using medium volume. I would play that particular passage at lower than normal volume to be certain you don't blow a woofer on this CD. Aside from that, I love the arrangement and sound of the Cincinnati Symphony Orchestra.",n +This record arrived shattered... so who knows if its any good. I do know that this sell does not know how to package or ship !!!,` +😳,E +😗,s +I bought this in the form of a cassette tape about 40 years ago! It has been without a doubt the most inspirational album that I have ever owned. I wonder if Judy is still with us? She has no idea what a blessing this album has been to so many. It's a wonderful musical attribute to the Pilgrim's Progress!,b +It took a little bit to come cause it comes from korea. It's really good even tho it's only 4 songs but it's a mini album .. very happy with it,x +The Lady at her Best !,D +"I've seen this movie many years ago and so much of it kept creeping into my head in the weirdest of situations.All I could do to make it stop was to watch it again, it's like that song that gets stuck in your head.",C +"Wet Willie, sadly overlooked band from the Altman Brothers period. Super soulful, very unique band. Pickup their greatest hits if nothing else..",[ +Josh Garrels is so gifted that any of his projects are beautifully crafted. A joy to listen to.,_ +A great album with awesome swing tunes. Our whole family enjoyed it!,g +Good cd,G +I couldn't believe I found this CD! Love it! No scratches no lag!,g +Thomas Ranja is simply the best interpreter of Granados I've ever heard.,s +"I heard them in So California in the 1950's live at a Sing-Spiration. The lead singer's wife played piano for them. My family, and some of our church members went to hear them back then, and we bought an album when the concert was over. I have listened and loved each song all these years! Reign Master Jesus Reign is maybe my favorite, but they are all great. I made an amature copy from my vinyl album to a CD-writable, and you can hear all the pops and scratches on it, but that's okay! I wish it could be done professionally and released to modern day fans!",G +Exactly as described and perfect at a great price. It is always a pleasure shopping on Amazon.com! Thank you kindly.,_ +"Nat King Cole, what a wonderful singer and this album is great for fans like I am of him.",y +The best word to describe this is JUNK. There are all kinds of technical problems. It is in the shop all the time. Zero help from the manufacturer. They do not respond to my requests for help.,Z +"This CD provides essential relaxing music, for setting the mood for relaxing prior to going to bed.What more could a person ask for?",J +"I also have the "The Most Relaxing Classical Music in the Universe", but this Mozart for Meditation CD is probably best!",c +Thank you amazon my item was very good 👍,e +I love this cd. I had one years ago and lost it. So happy to have found another.,o +"This lesser known and still wonderful Donizetti opera tells the story of a young woman Marie who was abandoned as a baby in the Tyrolese mountains and reared by a regiment of French Grenadier soldiers. She falls in love with a young man and finds her long-lost mother Marquise de Birkenfeld who bore Marie out of wedlock. Marie is pressured into an arranged marriage, but seeing her despair the Marquise finally relents thus allowing her to marry her true love. The performance displays an incredibly talented and vocally perfect Anna Moffo. This is live recording from 12/60 and the sound quality is very good++ - much better than most live recordings from this period. It's a wonderful chance to hear the great Moffo early in her career. Plus, opening your ears to a beautiful and touching opera by Donizetti. The only complaint that I have is this cd is highlights only. It would be terrific if only they would release the entire opera considering the sound quality being so good.",Y +"As far as the music for this set, it's great! Packaging appears to have mixed up the CDs with the 3 disc set. I received my order as requested, 2 CD Version, however, instead of the remastered original album (Disc 1), I received disc 3 - The Homecoming Concert. Disc 2 (More Money Maker) was appropriately included. Perhaps this is a problem with having so many alternate sets to satisfy everyone's pocketbook. Will need to request a return.Anybody else experience this?",G +A great addition to my Elton John collection. I love it.,e +Amazing job Totally worth the money. Great sound effects and story.,n +"For me, this is one of those gems you fine without actually looking for it. I found this group while watching the movie "The Shunning".I bought the album just for that one song and didn't get my hopes up for the rest of it. Turned out to be one of those few albums that I will listen to the whole thing and not have to skip a few tracks I can't stand. Looking forward to hearing more from Graylit.",] +This is a great concert and a must have for Dire Strait fans. This is however not a DVD Audio (will not play in Acura's DVD Audio player) but will play in a DVD video player.,O +Excellent singer and music. If you love her and who doesn't? You will love her music!,a +It's my second time purchasing this album.......I love it!!!,l +"Very well, I thought it was a little thicker but I still liked them.",a +"Product arrived in good shape, containing 1 hit after another. It is a joy to listen 🎶",g +"I heard Psalm 98 on my local Christian radio station years ago and was so moved by the song that I found the nearest payphone and called the station to find out who sang it, and the name of the album. I want to a Christian bookstore and had them order the album. It is one of my favorites to this day and Psalm 98 is one of my all-time favorites!!……",N +I have a copy of this I bought it for an ace on my team! HE should love it like I do.,l +Good music but no where near the quality of her previous albums,R +"Sarah Vaughn owned the song. ""Send in the Clowns."" I remember Johnny DanKworth introducing his Wife as ""The Greatest Jaazz Singer in the World."" I won't mention her name. She opened with ""Send in..."" Two weeks before we'd been seranaded by Sassy with the same song. Sorry Johnny, your wife wasn't.",p +An informative and entertaining program. Obtaining the CDs makes it easier to listen to the music Professor Greenberg mentions in his lectures. I highly recommend the whole program.,\ +"Jewel case was okay, but the CD itself played perfect. Great value for this album.",_ +I really like her voice.,e +He loves them. They arrived on time and as advertised,P +"Sadly I lost this CD. However, before I lost it I was bumpin this all day!! How I had not heard this before is just a crime in itself! The sounds these dudes pull off here is a sound that you don't hear too often. It's raw, it's smooth and it has a vibe that can uplift you, break your neck or make you feel like you just got a massage or something. Seriously these cats have some mad skills and I definitely recommend copping this while you can.",d +"I was pretty excited about ordering this CD. I own a Scribbling Idiots CD and thought this one was going to be just as good as that one. I was a little disappointed. A few of the songs sound cheesy, either because of a not so great beat or not so great lyrics. I know Cas Metah and Just Me got skills but this CD wasn't all that great. I either set the bar too high for this CD or they were early into their careers and came off a little weak on this one. I did like a few of the songs so I won't give it anything lower than a 3.",[ +"I have this album, purchased for six dollars heaven knows when, and probably decades ago. There are many interpretations of Kreisleriana; I have two: this one and Cynthia Raim's performance for Connoisseur. Arrau's performance sounds a bit deliberate, especially the first track: Ausserst bewegt. Ausserst bewegt is half-etude and half caprice. It plays with meter much in the same way that Schumann did for quartet 2, opus 41. The phrases overlap each measure by one triplet sixteenth note, except when they seem to catch their footing, except that they don't. Great fun. I prefer Raim's interpretation, because it seems more fluent and sustains clash of lopsided phrasing and the seeming regularity of triplets. And her interpretation seems to capture the drama. Arrau does much better with "Sehr innig und nicht zu rasch." He takes his time and makes this rather long movement serenely beautiful. Where he lacks the aggressive energy of Raim's performance on side one, he pulls more emotion out of the music. On track seven, Sehr rasch, he shows his ability to match anyone's tempo and dynamics. Amazing.And while this isn't an audiophile-quality performance, and it is close-mic'd, there is plenty of detail in the sustains and mechanics of the piano. If you like vinyl, this is a pretty decent deal. But you may find it for a lot less at a local used records store.",F +"This LP was purchased in 2011 as new old stock: a sealed lp from 1972. I remember hearing this LP at the local public library as a junior high student and wanting to buy it. 30 years later I have it and it's not all that great. The LP pressing is noisy and the levels exceed the dynamic range of the LP, so the sound is distorted. Furthermore, the Partita for Brass Quintet and Tape is all brass and no tape... they actually didn't perform it with the tape accompaniment (unless Lazarof recorded other brass players for playback). I don't think that it is his best work, either. In summary, I recommend avoiding this LP, which turns up online from time to time as an overpriced item. Instead...Readers of this review should sample the amazing Naxos CD of Tableaux/Symphony #2/Violin concerto. Lazarof writes great music. The quality of the recording is top-notch, too. For that reason, if his music is to your tastes, I would forgo the MP3 format and buy the CD.",y +I love Taemin so I had to buy it. It is definitely new. However it came extremely late. I forgot I even ordered it.,F +love it all,v +"Every song on this album is excellent. I discovered Bacon's voice for the first time while Jopping with SuperM. His voice really surprised me, so I decided to do some research about him. I was thrilled to find out that he had a solo album. This album is a wonderful arrangement of mature R&B. It has that old school '90s R&B vibe and it is refreshing to hear. I received the Night Version and it came with everything listed (Photo Book, Lyric Booklet, CD, Photo Card, Folded Poster).",D +"This album has a nice collection of songs that give off an old school R&B vibe. Yugyeom and JB's voices mend together perfectly. Their song writing skills are fantastic. Focus On Me and Long Black make you go, wow! I wasn't even familiar with the group they came from originally. Jus2 has a totally different vibe from GOT7, so I was surprised these two individuals had so much soul. JB's soundcloud showcases even more of this. They are super talented! Many of these kpop groups have hidden gems of talent that I feel can't really showcase their individual talents. Jus2 is the best sub-unit collab I have heard in kpop. This is a wonderful album and I hope they make another.This album collection has many nice photos. Yugyeom and JB are stunning! The whole concept is very classy and elegant. The poster is much larger than I expected. I loved the extra photocards that came with this. I had to take off one star because the most important thing in this, the cd, had scratches. These scratches caused the songs to skip. I wiped it down and that seems to have remedied the problem. The album was sealed, so I don't know why it had scratches.",H +Nice gospel cd. Love the old school gospel style & flavor. Adding this cd to my workout favs. 😍,c +Beeth --- got the notes so good. Alfred plays them well. Con brio.,e +I am enjoying my Pottery Barn CD. I have collected most all of them. Quick shipping,T +"Another introspective, jazz filled gem from this west coast duo. It's right up there w/""below the heavens"". A hip hop classic.",z +love this album,P +love it,H +love it,S +works great!,` +"Not too impressed with this product as it is difficult to handle when cutting the greens. Finally found out if I cut the greens into small pieces before I use this toss & chop scissors it worked better, but I will not be using this like I would have hoped to do.",f +"Almost needless to say, but I'll just say it: this is for the hardcore Bjork fan. For anyone who knows her history, Tappi Tikarrass was a phenomenal era for her, where with Eyþór Árnalds, Jakob Smári Magnússon, Eyjólfur Jóhannsson, and Oddur F. Sigurbjörnsson, pop and punk was mixed into one stunning, exciting, and stimulating experience. Miranda is their first and only album. A rarity, a one-of-a-kind collector's item, and a true masterpiece of what it was and still is for its time: simply something special.",J +"Of all the limited edition colored vinyls, I'd say that Medulla and Vespertine is the best in quality, sound, art work, and beauty. These albums are already amazing in their own right, but on LP, it's transformed into something out of this world. I LOVE the color for Medulla. It looks prettier in person than it does on the image here. If you love the album, you'll love it even more on vinyl.",S +"The best part of ordering this was the nostalgia that came with it. The case was damaged a little but it was made of cardboard/paper material. As far as the cd it was in great condition. Album is amazing, the remake is out now and i have to get that now as well!!!",r +"Moody blues💕💕💕 Great CD. Packaging was good, and fast delivery.Windows down, cool breeze and the Moody Blues “sounds”. Heaven",R +"I really do like the Hypnosis Network Series but I think they are too expensive compared to other cd's readily available. But if you have the money to spend, these cd's are worth the price. This series, Core Inner Strength, is focused on working on your self-esteem and and self-confidence. These sessions reminded me very much of the type of session you would undergo if you went to see a psychiatrist that utilized hypnosis in their practice. There is no background music and no wave sounds. All you hear is Dr. McNeal's calm soothing voice. She first puts you into hypnosis and then once you are ""under"" and in a safe place she talks to you about trusting and finding your inner voice, self-confidence, self-esteem, and feeling safe in your life. Again, no bells and whistles or new age stuff but just the type of session that you might get if you went to seek help from professional western doctor to help you regain your self-esteem after some sort of emotional trauma from childhood, a divorce, or mild depression. I think it would be very good for someone who is healing from an verbally or emotionally abusive relationship and is trying to reconnect to their inner sense of self worth.",y +"Met expectations for relaxing music to add to my collection of stress busting CDs. No virtuoso classical performances here, but worth the very reasonable price for an occasional listen to help you unwind.",x +I bought this cd when Mark sang at my church - and then lost it. I was so happy to find a replacement because I love his voice and his song-writing.,S +"Streisand at 19 singing, Lillian Roth, a master of singing on the stage, Elliot Gould excellent voice, all bringing down the house.",D +I am PISSED. This says Green Marble. The packaging says green marble. It came as a plain black 180 gram vinyl. This is NOT what I paid for!,C +The CD was defective,W +"Bought first years ago for my granddaughter on DVD, now at age 17 she asked for a cd of it for her birthday! She still loves it, and they are all from scripture, hence the title. Some of them caused me to open the Word to look at myself. Excellent tool for teaching scripture to anyone!! Love songs and music!!",z +A great remix of classic Herb Alpert music.,Y +Jim Ahrend is a wonderful musician. I enjoy every song on this CD! I wish I could find more by him.,J +"just about anything Mojo does is done well, and this is no exception. It's not a Tour De Force, but it's dam good and interesting, too. Well worth the time it took to buy it and the money was well spent.",M +"Wow, I just can't believe this C.D. I mean I'm just shocked how good this guy is, and I have never heard him before. I don't know much, if anything about him, so I didn't expect it would be this good. It appears he's still alive, 91, and got a handful of Cd's out there I've found on the internet, the most one goes for was 19.99, and most are five buck or less. I guess the price of a c.d. or how commercial a musician is, or even how many Cd's they have produced does not determine the worth of a musician.We listen to what people promote, what we know of, what's marketed to us, and there are these people out there with amazing talent. Well I'm probably not going to be a savvy namedropping blue musician reviewer, here. I do know good music though, even though a matter of taste, this cd. is impeccably recorded and done.I guess I am just wondering how it is that he's not more commercially successful?I love R.L. Burnside, too, but he's prolific with the c.d.'s and even with hooking up with a younger set of commercially successful artists or I'd probably not have found him either.",` +"It's been a long time since I heard a compact disc that was truly unique and somewhat unable to be categorized. First, it's amazing. The voices are solid, like a rock opera style in a way, much like Jesus Christ Superstar, the rock opera. Not exactly, but it reminded me of that. It's somehow powerful, and mellow at the same time, exciting and full of passion. I would not normally think a cd was worth this price, but I do with this one. I wish I knew more about it's origin, where the music comes from, what it's all about, etc. I do think it has a spiritual quality, and don't know if they are singing about their relationship with God, or a person, or both?I say it is worth it because I would pay that to see them one time, and with the cd, I can hear it over and over again. Also, because it's unusual, it has grown on me. At first, the verdict was out. I was not unimpressed, but it did not knock my socks off. As each song played, I began to say wow, pretty impressive, and then listening to it again, it was a lovely experience. Worth a try.It looks, from the cover, like your typical "new agey" cd, but it's way beyond that.Also, there is some "techno" like sound in there in certain songs, and usually that turns me off, but it's good music.I mean I do like techno to an extent, but often I find it cheapens music, lowers the class of a piece, but that doesn't seem to happen here. Hope that helps.",W +"I don't know who mixed this c.d., who chose the artists for each song but its a treasure chest of rare, raw, really authentic and soulful musicians that are like best kept secrets in this age of "pop" where many of us want more than banality, whats marketed to us. Crazy interesting and refreshing because of that. Otherwise its a bit dark..but in a good way.",m +"I did not really like this and cannot recommend it. I'm not a huge fan of this type of music, but I do like it somewhat, esp. songs like Ave Maria. How could anyone do that one really bad. It's a great song. So I liked the style and the songs for the most part, but it was not well done. I was thinking, maybe it's not supposed to be, like your Uncle George singing at your wedding. I think some of it was kind of sad it was so "roughly," done, because I can accept it and rejoice when my Uncle sings. He doesn't have to be a pro. But, I would never purchase this, and that's about it.",j +I loved all songs but there is a certain song "I belong to you" that I literally adore. Beautiful song and it is very nicely performed.,F +My husband and I still buy CDs and this is one we added to the collection,W +I love Cherry Ghost. This is just as good as the other albums they made. Beautiful and melodic. Simon Aldred is a great singer and songwriter.,D +"We used it to cook Laing and it’s very good , one pack can cook a lot since it expands once you add liquid to it , will buy it again",t +"Eagles made wonderful music! The harmonies and accoustic style makes this album a must have for any Eagles fan. Too bad all things must end, but we have some great songs by them",p +Beautiful blue vinyl. Purchased as a gift for my son and he loves it.,S +"I really enjoy this CD, when I am needing some uplifting or just want to listened to worship, I put this CD on. Better than I expected, no regrets on this purchase.",d +Absolutely beautiful collection. My 4 years old loves it!!,r +"What a fun album - it starts out with the version of I Can Only Imagine from the movie!!! Love the original, and the beat of this version is amazing!! Sooo glad I bought the album. Lots of other greats on the album as well.",F +"The songs are truly an inspiration and a soothing element on anyone's heart.I found ""One Day"" and ""Don't Have to Speak"" two songs which bolsters my heart for days ahead.""Lord of the Harvest"" is a wonderful praise song, glorifying the awesomeness of our Lord.""Miss Me"" is the tender and heartfelt song of a lost soul reaching out to God.The rest of the songs are just as powerful and fun, each praising and glorying God and His awesome love.",d +"This is one of the early CDs from the ORIGINAL Eden Regained Christian rock band, based in Newark, Ohio, and formed in 1995 with a group of Newark High School students. The band's skilled instrumentations, heavy guitar sound (formed by three guitars plus bass) and beautiful vocal harmonies, along with their good looks, stage presence and message, made them a very popular band in the MIdwest in the mid- to late 1990s. There is another Christian rock band floating around YouTube who has stolen this band's name, but this CD, Once You Move, belongs to the original Eden Regained, who also crafted The Clean EP and Ascension before the band's breakup in 2000. It must be noted that they toured extensively in the Ohio, Michigan, Pennsylvania, and Indiana regions and were offered a band showcase in Nashville, where they were offered a contract, but never signed on to tour. On this album, the 6:48 minute ""Shaded"" offers the best on the Once You Move CD, with outstanding guitar and vocal solos. Their all-original songs are all copyrighted and owned by the REAL Eden Regained band members: Nick Bain, Shawn Barrett, Scott Berry, Donny Dixon, and Jason Grady.",F +thanks for the awesome comic!,o +Nothing better than Swedish Candy :),R +"If you long to hear some fresh new arrangements of holiday favorites, this is the album for you. This album by the Young People's Chorus of New York City also features some new holiday songs, like the gorgeous ""How Many Christmases"" featuring Rosanne Cash, which contains a beautiful message you will want to listen to over and over again.",r +I have looked for this artist for a very long time. I am glad that I found it and enjoy listening to all of the songs on it especially Private Affair!,E +"Cathrine Iversen truly has the voice of an angel. Pure, sweet, without being saccharine. Truly uplifting. It’s April and I only just found this CD, but who cares! It’s gorgeous and uplifting. The Norwegian lyrics in particular are exquisite. It’s a beautiful language, captivating, never harsh. This is my new favorite CD to get my day started while I sip my latte!",p +"Sonny Boy Williamson (Rice Miller) The Chess Years - Chess CD Red Box 1All the Chess sides recorded by Sonny Boy in chronological order on four CD's. An exemplary reissue.This is prime Sonny Boy, all of his best recordings for Chess in clean recording quality. This is all the classic Sonny Boy you'll ever need!",B +"I used to listen to Robert, live here in Union, wonderful singer and guitarist, always had stories about the Yardbirds to tell. sadly he moved down south. Bought this little cd and loved it very much. I especially liked Song to Churches, very, very nice.",` +I purchased this in audiocassette many years ago at Target and I still consider it my best workout tape. So happy to have found it in CD version.,P +"I bought this CD in Seattle as a gift for my 4-year-old son. He's 5 now and he still adores these stories. They are all very gentle with positive messages, and wonderful for car trips both short and long. We love to sing Mindy Mousekins' song, "The thread follows the needle..."We have the Autumn-Winter CD but are about to purchase the spring-summer one to complete the collection.",H +"Of the 3 volumes of this series that exist, this is by far the easiest session to skip. WHFS at this stage in their lifespan had lost some of it's indie-cred and that is reflected in the generic artists and song selections for this collection of songs performed in the radio studio. Recording quality is good...it is sadly the songs that lack any substance.",_ +Great CD even if it does not represent their best songs. Strong recording quality and a very intimate feel. Great for those that have not seen them live or experienced their power on stage.,\ +"This Fast Tracks For Men is the first cologne by Watermark Beauty that I really did not like 100%. Don't get me wrong, its okay. But it kind of reminds me of a mix between a manly scent and a feminine scent together. Its not strong, but not sweet either. The scent did not seem to stay on very well and should probably be reapplied at least once in a day, but the 2.4 Fl. Oz. / 70 ml is fairly large and should last.It is supposed to be Inspired By Ferrari Cologne but I have never tried it before so I cant tell you how close it actually is. Its okay, but by far not the best scent by watermark beauty I have smelled so far. I received this product at a discounted rateThanks <3",Q +"I can't believe I found this original recording on Amazon! My brother and I would sit at the phonograph time and time again when we were little, with our Dad at our side, listening to the wonderful story of spunky little Lentil! Oh how we loved it! What do you give your 73 year old brother for Christmas . . . that's better than a sweet childhood memory, eh?",q +Just plain good stuff! Makes you feel good. Mellow and thoughtful. We cant wait for more discs from Eliezer and Yonason!,c +it is here,R +"Having heard this years ago, we purchased this for a relative who needs lifting up on a daily basis. This worked!",w +"We met Ronnie Freeman who sang at our church some time ago, and though we rarely purchase CD's (we have SO many), we chose to buy this one. It was then, and still is today, one of our favorite Christian CD's. Let's just say that it doesn't get dusty with many of the others. Uplifting to the heart!",P +Great Music. But this bundle is more expensive than buying each box set separately.,N +"Early in their career, the Alan Parsons Project didn't play live. Alan Parsons knew he would be unable to produce in a live show the excellent production quality he was able to perform in the studio.As technology advanced, that limitation was lifted, and the musical world is a better place for Alan Parsons' frequent concert performances. I attended one and noticed no dropoff in quality from the cds. If you have a chance to see him, do so. He's a unique talent. Sadly, with the passing of Eric Woolfson, there will not be any new APP music.As for this CD, it's excellent. That said, if you have all of the APP cds, this is not a must-have. But, if you want to hear slightly different versions of their numerous great tracks, this provides that, with high quality.",V +"This is Yannis best CD by far. After seeing the concert on TV many years back, I've been hooked. Since then, I've attended six concerts.",v +This entire project is timely and much needed. We can't wait until this songstress is released to spread her wings again.,D +I bought this as a birthday gift for my niece who enjoys vintage albums. She is still raving about this one. It is still a classic.,a +"I really enjoy this hypnosis CD. The women voicing it has a very soothing, British accent. It is so important that the listener enjoys the voice hypnotizing them. What she says is very clear, and powerful. I use this CD regularly.",N +Great,R +Brandon lake has an amzing voice both in his own music and with elevation worship,E +As described,j +Bought it for the Rooster song...the rest are OK,M +"Good, timely service.",L +This movie didn't work in my Xbox One it wouldn't play bad experience,b +The vinyl and cover are well done. I own both this one and the blue emblem. Great for framing and displaying. A great add to any DC Comics/Superman collector.,] +"This is an excellent album, yes vinyl not CD as we know tends to be flat. Finished playing side one and the sound was fantastic, turned up the volume and the sound was clear not distorted. My only gripe with this album is that it's missing "Lodi" so unfortunate as this is a great song, very much missed here. So I really have 19 songs vice the 20 stated on the cover. 4 Stars due to the missing song.",` +It says Twiggy's real name was Lawson.Her maiden name is Hornby; Lawson was from hr st marriage,T +great music,L +This album is absolutely fantastic!,j +Great Purchase,J +Great Purchase,d +Great Purchase,a +"Another great album by Asbury about acknowledging our sinful shortcomings, begging for God's forgiveness, and resting safely in the promise of His mercy and forgiveness because of Jesus' work. Favorite tracks include: Canyons, Unraveling, and The Father's House.",N +Well,G +I like the CD. It’s okay music. I like listening in my car disc player. Thanks.,[ +Its very good cd except I think to many songs for the price. I enjoyed it. Thanks,T +It’s very good CD. Enjoyed listening in the car. Fast shipping. Thanks,u +CD was as described,h +Very good listening of old church sounds.,Y +I love this musical has a lot of good songs and artists. It's nice seeing artists perform live especially those who have passed away.,q +I love this musical has a lot of good songs and artists. It's nice seeing artists perform live especially those who have passed away.,Q +I love this musical has a lot of good songs and artists. It's nice seeing artists perform live especially those who have passed away.,j +Good music,\ +"For many people who don't know who she is her music has been played in anime like Bleach. Some of her music is well known in Korea, USA, Japan, China, Taiwan and Philippines. This cd is a compiled version of her music selection from 2002-2010. Her music has won several J-Pop awards. Now about the cd is awesome. I bought this because she is a really great singer and her band makes their own music. For an artist this group of musicians are very talented. I would highly recommend this cd to anyone who is interested in starting off to gain interest in knowing what J-pop is about.",v +This is an awesome singer very skilled in singing various types of music. Her music reached the top ten charts in Japan. She does a lot of pop rock. This cd was very well worth buying. The video is a DVD very clear.,H +"The music from her is very playful. She is dressed on th album very cutely. Also her music sounds pretty innocent. That is the best to discribe her music. If anyone would hear this music they would just melt. I know it is odd coming from a guy but, I show this album to some other people and they were hooked on it.",R +The music fits very well with the tv series. Aoi Eir has great experience performing her songs. Each of thing have a different theme that give it a special uniqueness. My favorite was on track 1 and 5. It is very well worth listening.,v +Kalafina is very famous all over asia. This cd was very special for the anime series. The music is very dynamic and give alot of feelings about the anime. If you are looking into fate zero the music will give you a great taste of what great anime music should be like.,[ +"love the songs, played well, no problems with shipping, got it quick and fast",S +"Coming from the UK, this cd does have a different feel to it. That said, the tunes here that aren't on the US version are great. Struts fans should def have this in their collection...",X +Good music,Y +"Very nice but liked more to Russia, something later like that.",Y +A great cd and great service.,a +A must have for your collection.,a +Great CDs,I +Love Yanni music this is one of his best albums,b +Love this CD. It only has 12 songs. Very worthy it.,c +Love love love this CD.,T +loved took me back to my younger years.,j +Love it.,A +"The singers & orchestra do a good job. The audience is quiet except for some applause and there's very little stage noise. The booklet, in Italian and English includes the complete libretto and has a good ammount of information about the composer, the libretto, the original cast & performance history. It does not have any information about the current cast nor does it have a synopsis of the plot & I was not ready to read the 47 page libretto to learn the plot. Also, there are only 8 tracks per CD which means some of the tracks combine a scene, aria, duet and or chorus all together instead of breaking them up into separate tracks. Even worse, there are no timings on the tracks. Each CD is approx 64 min and after doing some searching I found the track timings at Presto Music. CD 1 track 1 - 8:12; 2 -2:45; 3 - 1:51; 4 - 5:59; 5 - 11:49; 6 - 12:46; 7 - 4:09; 8 - 17:06. CD 2 track 1 - 12:46; 2 - 6:36; 3 - 6:20; 4 - 7:21; 5 - 4:09; 6 - 9:30; 7 - 10:00; 8 - 6:38. As for Persiani's music it's pleasant with some interesting arias & duets. If you've heard any of the music of his contemporaries such as Morcacchi, Vaccaj, Ricci or Mercadante or one of Donizetti's lesser more slap dash operas you'll have a pretty good idea of what this opera sounds like. If you're looking for something really great from this era & you're tired of the same old Bellini & Donizetti try Meyerbeer's Italian opera & even better the operas of Giovanni Pacini.",y +"This CD includes Franck's three most popular pieces. Recorded in 1959 & 1962 & remastered in 1994-95. Sound quality is very good. The Chassseur & Variations performances are very good. The Symphony, performed by the Philharmonic Orchestra and Constantin Silvestri is a bit slow and lackluster. The 1st & 2nd movements are both about one minuet each slower than another recording I have. That may not sound like much but it does make a difference. If you're going to own only one recording of this symphony, this is not the one to have. I checked & my other, more recent recording is also by the Philharmonic orchestra, but a different conductor (Francesco D'Avalos). It's much livelier and more satisfying.",w +"Pinto (1785-1806) died at age 20 of TB. These teenage works are so attractive that one wonders what he might have achieved had he lived a full life. Played on an 1807 Broadwood fortepiano (restored in 1991). Unlike some fortepianos which have an unpleasant sound, the sound of this instrument is very pleasing.",Z +"I owned this recording and having listened to it a couple of times I was always left with the question, why was this opera so famous? It was dull & plodding. Then I bought the 1954 performance by Callas & Corelli and I had an ah ha moment. Listening to that '54 performance compared to this modern performance made me realize what a difference great singers can make. The '54 performance was so great that I now understood why this opera was so famous & highly thought of in it's day. The quality of the recording here is obviously better than the '54 recording, but that's the only good thing I can say about this recording when comparing the two.",J +"A recording of a 1960 performance from La Scala. While not as good as a modern studio recording, the sound is perfectly acceptable. No noise from the stage or audience except very enthusiastic applause. The chorus, orchestra and singers are excellent. Callas is Callas whether you love her voice or hate it (this audience loved her). Corelli, at the height of his career, is the stand out. Incredible, voice, high notes & emotion. I've read that someone thought this opera contained some of Donizetti's best music. I'm not too sure of that. I'm not a big fan of Donizetti though I've heard most of the recordings of his operas. To me, this one falls somewhere in the middle, not the worst, but not up there with his hand full of best operas. There are some pretty exciting arias and duets for Callas & Corelli and some good choruses & act finales. With lesser soloists this opera probably wouldn't sound as good as it does here.",K +"A live recording from Verona, Italy. Very little stage noise & the audience is quiet except for applause after nearly every number. The sound quality is good. The singers, chorus & orchestra all do a fine job. As for the music, while there's nothing here that will keep you ""humming a tune"" ever after, there are many very melodic, pleasant & quite pretty arias & duets to enjoy. This opera wasn't one of Salieri's succeses. Apparently it wasn't the music that was to blame but the story which was about women characters taking on male roles & vice versa & even a gay character. The 1795 audience was not ready & rejected to opera. If you're a Salieri fan like me this is a must have, if you're just looking to sample one Salieri opera, then the ""Grotto of Trifonio"" or the ""Falstaff"" might be a better choice.",y +"These overtures are better than I thought they'd be. They've inspired me to explore some on Spohr's dramatic work, not just his orchestral & chamber music.",S +"Spontini's fame & reputation rested on his last four operas (La Vestale, Fernan Cortez, Olympie & Agnes de Hohenstaufen). Today only La Vestale keeps his name alive, if just barely. In my opinion, Olympie is the least interesting of the four. Originally written in 1819 for Paris it was a modest success. It was then rewritten in a new German edition for Berlin in 1822 & was much more successful. Then in 1826 the German edition was translated back into French & given in Paris quite successfully. It's the 1826 edition that's recorded here. The singing & orchestra are fine & the music is OK but nowhere as interesting as the music in Fernan Cortez or Agnes de Hohenstaufen. These last two operas however do not exist in modern recordings which is a great pity because in my opinion both are much better than either Vestale or Olympie.",y +"These are the first recordings of these two works. Steinberg (1883-1946) was the son in law of Rimsky Korsikov, the teacher of Shostakovich, & a fellow student with Stravinsky (who he beat out in a competition much to Stravinsky's chagrin over many decades). While Stravinsky went to the West, Steinberg stayed in Russia & remained very little known outside Russia, The two earlier symphonies recorded by Neemi Jarvi are very good in the style of Tchaikovsky & Glauzunov.. These are two later pieces. The symphony is a piece of Soviet ear music commemorating the opening of a railway. Programatic music with folk melodies. Well orchestrated. Worth a listen but not going to be a piece I'll listen to again for a long while. The violin concerto (from 1946) is much better, Very mush in the Romantic idiom, it's a very appealing work.",h +See may review of Bennett's Concerto #3 for my opinion of his 4th Concerto.,r +"A live recording from Turin, Italy in 1974. Audience was very quite. Sung in Italian rather than the original French. Based on the 1817 revision rather than the 1809 original. Sound quality is very good (vastly better than the 1951 Naples recording). Bruno Pevedi (a world class tenor in his day), singing the role of Cortez is outstanding. Chorus & orchestra are also very good, the soprano is the real weak spot for me (especially compared with Renata Tebaldi in the 1951 recording). The opera is wonderful. Some very fine arias & duets & especially outstanding & dramatic writing for the chorus. Why this opera has not had a really good, modern, studio recording is a mystery to me. So many much poorer opera have been recorded over & over I wonder why this very exciting opera has been ignored. To my ear, this opera is much more entertaining than Spontini's vastly better known opera ""La Vestale"". What may have held this opera back is that it has an all male cast except for one soprano & the lead character (Cortez) does not make an appearance until the second act. Highly recommended to anyone who enjoys French Grand Opera, Meyerbeer, Halevy's La Juive, etc.",f +Dad was happy thanks,n +Loved it. Thank you,C +"This is one of the finest recordings of cowboy and western songs I've ever heard. I saw them perform at their ranch in Wyoming, and I was captivated by their sound. They have perfect harmonies. It is authentic in its presentation yet contemporary in recording quality. It isn't overflowing with twangy sound, neither is it revved up with modern synthetic background. If you like cowboy and western music, you will love this album!",u +"The music from the Eloise films is happy, uplifting and beautiful. Worth every penny if you are a film score collector.",r +i enjoyed it but it's not as catchy as some of the earlier albums but Amr always delivers something fun to listen to.,t +ZBB AND Dave Grohl? Are you kidding me? Love love love this CD and the DVD! So fun to listen and watch. Great condition and was shipped quickly. Thank you so much,z +The movie is awesome.,[ +as advertised,i +"This band is the best! And this CD captures (almost) the best of Indian Ocean....although I think the compilation has omitted some very good songs of theirs. Still it is a good start for anyone just trying out Indian Ocean's music and sound.The recording is very, very good. There is an amazing amount of low level resolution and imaging is spot on. Drums, gongs cymbals, all come out beautifully! I was pleasantly surprised to find how good the recording is. I am listening to them through a Meridian 588 + Anthem preamp + Bryston monoblocks + Magnepan 3.6Rs. Indian Ocean sounds simply magical. Reminds me of Sheffield Labs recordings ( I have many of their CDs). VERY high resolution.",^ +"This CD dropped in 1996 and the music is timeless. Frank and I had just 5K copies made and here we find a few surviving in the Amazon jungle. A+ musicians, experienced engineers and producers assembled the masterful music impressions of 'Raw Sweat'.",g +"""I was walking through Milwaukees' Irishfest '05' with wife and college aged daughter. In the background was Off Kilter rocking away like no other at the fest. Like a bug to light we wandered over. My daughter said ""they are lil to wild for my tast in Irish"". I bought all their cd's that night. Pure fun and truly unmatchable. Watch for them live, They are great for the whole clan. They are very collectable. More bands should follow there style. Enjoy",D +"Not the same ol' greatest hits package, this have a few raritys on it, like, je tem- I Feel Love Patrick Cowley LONG remix! So, id get the before it disappears. Recomended.",g +"I first discovered The Real Ambassadors in the summer of 1996, and I listened to it obsessively. Yes, it has a message. It's about racism, nationalism and all the other "isms," but at the end of the day, it's just a great album, and I'm glad to own it on vinyl finally. Highly recommended. *****",L +"Had the luck to hear Dalglish and Larsen live a couple times in the SF bay area, late 1970s/early 1980s. Transcendent. The first track on the album--Paddy Concannon's Reel/The Wise Maid--is just wonderful. I have the vinyl, left over from those days. If you have a turntable, and get the chance, do pick this up. It is truly wonderful.",_ +"I was shocked to learn a few minutes ago that Gerry Niewood was among those killed in the Buffalo plane crash. His album as leader of the group Timepiece has been one of my favorites since college days in the 70s. I've always considered Niewood to be a pioneer of what became known as ""smooth jazz,"" although Timepiece is several steps above what passes for jazz on commercial FM radio these days. This is wonderful, engaging relaxing music, and while I'm shocked over Niewood's death, I hope his work will now find a wider audience.",v +"Combining all the bombast and pretension of Coldplay and U2 with songs that rely more on gimmickry than craft (several are more loops than structured songs, with lots of abrupt samples thrown in for, um, who knows why) and kitchen-sink production that strangely makes it sound like speakers are blown on your system on the more bombastic tunes, Imagine Dragons reference everyone from Arctic Monkeys ("I'm So Sorry") to Ladysmith Black Mambazo (the ubiquitous "I Bet My Life"). They do it fairly well but blaze no new trails of their own, almost always sounding like bonus tracks from other bands' albums. Some bands should be flattered; the aformentioned Coldplay and U2 should sue.",O +Awesome songs,^ +I love this album.,u +One of our favorite CDs.,X +Just what I expected if not more.,d +"I bought this for my father whose music collection included this album set in cassette and his new car had only a CD player.He was thrilled.Good remastering to digital, sounds better than the cassettes or the vinyl for that matter, and he is a happy man.Says this is 'his and mom's music'.A great gift for Older Americans who grew up on the music of America's Heartland.",r +excellent purchase,e +excellent purchase,I +"It's Larry Young.'Nuff said.Listening to Larry makes you feel so good, you might feel as if you were a sick man waiting online. And you just felt the first invigorating jolt, from your new "organ" transplant.",c +Bucky's the best.This is TRULY a SOLO masterpiece.Viva La Buck!,G +Price,U +Price is great,C +"It would have been icing on the cake if they had included the lyrics. None the less, he's King for a reason.",C +"This is, of course, an unauthorized recording. Every version of this concert I've heard has audible distortion, and judging from the other reviews, this is no exception. Thing is, in late October an official version will be released as part of the Joni Archives Vol 2.",t +"The sound quality is only so-so, and the speed is off. Readily available for cheap on eeb-ay for cheap.",c +"This 3CD boot contains: CD1 Newport Folk Festival 1969 & other random early cuts. Sound VG+ CD 2 Live FM b'cast Netherlands 1983 tour. Full band, great sound. Worth it just for this (also on Youtube) CD 3 Solo Joni 1995 Gene Autry Museum, LA, Excellent sound FM b'cast-also widely available online.",w +"This is one of Air Supply's best cd's and has so many of all the old songs they sang for the past 30 years. I love Air Supply, and have seen them in concert, and have several of their cd's. Just the right kind of music to mellow out and enjoy after a long day. One of the best cd's they have.",B +Enjoyed the CD very much.,Q +"When Buffalo Springfield was on the radios, I was not worried about buying the music, but now, I want to hear the music that I loved then. Just bought all three of their original albums.The music is just as great now as it was then and the clarity is much better.",c +Great Christian Music,R +Nice,\ +Nice,g +A great group of songs from my childhood.,k +Great album. If you are a fan of Dave's I am sure you will love this Lp. Some great tunes on this album my favorite being Show me Some Affection.,I +Childhood,n +I wish the song JEsus Christ IS The Way.,x +"Michael O'Brien's new CD ""Something About Us"" is one of my new favorites! Meaningful songs, romantic melodies, and his completely beautiful voice makes this CD one that will stay in my collection for years to come.",e +GREAT worship cd! This is a very anointed cd I love the worship!I would recommend. ITs GREAT to listen to!,q +"I was fortunate enough to be present for this live recording. It was and still is a beautiful set, I love her use of strings in this recording. The musical element is powerful and stirring just as much as the words. The songs are amazing, and as always, Kim brings the Presence!",f +beautiful mariachi & synfonic orchastra collabaration,^ +Great as expected,z +"I remember these songs on LP's from back in the 50's & 60's, always good to find CD's that sound this good. "El Tariacuri" at his best.",H +"This book is packed with interesting facts and social commentary, and I enjoyed it enough to immediately start another Picoult book. That said, two crucial characters keep withholding information from a murder case — with other characters even imploring them “Don’t tell me” — in a way that feels a bit contrived. Sure, there are reasons, but I think the book could've been even better if those secrets weren’t saved for the end. Still, Picoult makes an important point entertaining when it could've easily been dull or depressing: how poorly set up our judicial and prison systems are for handling anyone who’s even a little bit different. Grade: B+",W +"Kind of a young adult version of "Oryx & Crake" where teens navigate what they think is an awesome world but is really a dystopian future dominated by corporate advertising and the protection of Americans from knowing about the damage caused elsewhere in the world by their habits. People have chat feeds and shopping recommendations plugged directly into their bodies from birth. It took a bit to get used to the language because nearly every sentence is in teenspeak but you soon realize the entire society is this way — even the parents are, like, "Dude" and "Whatev." Their limited vocabulary weakens their ability to express their emotions when things start to go wrong. I'm not sure how the book will "read" but the audio version was fantastic, with fully produced commercials between chapters. The book is better the more I think about it. Bechdel test: pass. Grade: A-",S +"good CD. has some great one hit wonders by the original artist, and seems to be a good recording.",Z +good CD. good recording. original hits and by the original artists. sounds great.,s +good CD. too bad it don't have it might as well rain until September on it.,I +"I like this, but it is a bit weak, it don't have all the recordings from the EP, Follow that Dream. I don't understand what is what on the track listings, but some of the songs were OK, even though I did not quite place them.",K +"I love this CD. 1 bestknown single is on it. Playgirl sounds just like the hit 45 single.FYI; if you must ask, this sounds great sober, or drunk.",r +"1 song on this CD is very well known, at one time. Chirpy Chirpy, Cheep, cheep.I apreciate the full unedited hit version on this CD.",F +"this CD has Donnies 3 biggest hits on it. they all seem to be of excellent quality and are the original top 40 hit versions. this CD can be a little hard to find anymore. these CDs can be scalped on the prices, sometimes, so I was glad to find a good price. there is a seller or2 or more that may say the scalped prices are the best I can find, but the price I paid, has undersold them by a large amount.",x +"this cd was at 1 time misunderstood by me. after listening thru a few times, I find that it indeed has the original hit of breaking up is hard to do by Jivin Genethis is realy a good CD. I love it.",s +"there are some good tracks on this CD. I would want to be sure they are good recordings. if they are good recordings, that would make this a very good CD.",U +Beautiful songs and singing by this talented Taiwan singer. It is out of print almost everywhere. I am glad I could purchase it here at such a low price.,i +"Thanks to this recording, we finally get to know how a Kleiber's Mahler would sound like. Unless you are paying some offshore sellers near $100 for this CD ... the experience alone is well worth the price (near double of other classical music CDs in Amazon)Like one of the reviewers has mentioned, the tempo for 'Abschied' is indeed curiously on the fast side. The pace is more curious than the fact why Kleiber choose to conduct a Mahler piece at all. Perhaps the Mahler revival in the 60's had motivated him enough? But why the fast tempo?Nevertheless, IMHO, Das Lied is Mahler's farewell piece, not a death bed piece like the 9th or the 10th. It is like how we get used to numerous versions of comatose Faun (Prélude à l'après-midi d'un faune) or Mahler 9th. Kleiber's choice of tempo seems right to me.I am so very grateful to own this CD",a +"I believe this is one of Jim's best! I can say after many many personal development cd's I've used over the past 25 years, this is in my top 5 of programs. What a great man! If you only retain or learn a fraction of this material, you will benefit immensely.",k +I love it! Thank you!! So cuddly and soft.,G +"Listening to this album reminded me of being at a John Denver concert. Bevan does an awesome job at singing John Denver songs. It would be great if he went on tour here in the US, I would be there.",K +Okay,V +My children love this story and often act it out. The recording is well done and the songs catchy for children. The story of children “becoming” brownies to help with chores is charming and a cute twist.,J +I like it.,V +I bought the album to support NU'EST and it's one of my favorite albums! Amazing songs and beautiful photos are included. Please support them!,\ +Love the CD/DVD Combo! Fantastic Performance!,x +"This is really junk junk. Normally I read reviews, but I was in a rush and only looked at stars. Now after I got the hose and tried it out and saw how bad it was I went back to read the reviews. The only good reviews were written by people who are from the seller and who have been given a "discounted" (I assume free) hose to test and give an "unbiased" report on it. All the bad reports were given by innocent people like me that trusted the stars. I thought Amazon had solved this problem of fake reviews.Anyway, the nozzle did not attach properly to the hose because the threads don't match up so it leaked badly. This problem is compounded by the fact that the nozzle once attached cannot be removed because the nozzle threads onto a fitting that has a separate set of threads in the back which thread directly to the hose. So when you try to remove the nozzle you end up unthreading the valve that the nozzle threads into. There is no place to put a wrench on to keep everything from spinning. It is hard to describe, but trust me you can't remove the nozzle.The holder is a thin piece of metal with no holes to put screws in so it can be mounted.So I am returning it as you will too if you buy it. I never actually got to see how long the hose lasts. I refer you to the other one star reviews for that information. Good luck.",m +"2 CDs for next to nothing, how can you beat that? The music is VERY relaxing--great background music, fine for meditation too. Versatile Telemann.",H +quite good,E +"interesting, but passed it on",^ diff --git a/src/test/scripts/functions/transform/TransformFrameEncodeBagOfWords.dml b/src/test/scripts/functions/transform/TransformFrameEncodeBagOfWords.dml new file mode 100644 index 00000000000..49231c97c0b --- /dev/null +++ b/src/test/scripts/functions/transform/TransformFrameEncodeBagOfWords.dml @@ -0,0 +1,78 @@ +#------------------------------------------------------------- +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF 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. +# +#------------------------------------------------------------- + +# Read the token sequence (1K) w/ 100 distinct tokens +Data = read($1, data_type="frame", format="csv"); + +if(!as.boolean($4) & as.boolean($6)){ + Data = Data[,1] +} +if(as.boolean($5) & as.boolean($6)){ + Data = cbind(Data,Data) +} +while(FALSE){} + +if (as.boolean($4)) { + if (as.boolean($5)) { + jspec = "{ids: true, bag_of_words: [1,3], recode : [2,4]}"; + } else { + jspec = "{ids: true, bag_of_words: [1], recode : [2]}"; + } +} else { + if (as.boolean($5)) { + jspec = "{ids: true, bag_of_words: [1,2]}"; + } else { + jspec = "{ids: true, bag_of_words: [1]}"; + } +} +if(as.integer($7) == 1){ + jspec = "{ids: true, bag_of_words: [1], recode : [1]}"; +} +if(as.integer($7) == 2){ + jspec = "{ids: true, bag_of_words: [1], dummycode : [2]}"; +} +if(as.integer($7) == 3){ + ones = as.frame(matrix(1, nrow(Data), 1)) + Data = cbind(Data, ones) + jspec = "{ids: true, bag_of_words: [1]}"; +} + +i = 0 +total = 0 +j = 0 +# set to 20 for benchmarking +while(i < 1){ + t0 = time() + [Data_enc, Meta] = transformencode(target=Data, spec=jspec); + if(i > 10){ + total = total + time() - t0 + j = j + 1 + } + i = i + 1 +} +print(total/1000000000 / j) +print(nrow(Data_enc) + " x " + ncol(Data_enc)) +#reduce nr rows for large input tests +if(nrow(Data_enc) > 100){ + Data_enc = Data_enc[1:100,] +} +write(Data_enc, $2, format="text"); +write(Meta, $3, format="csv");