Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
Remove <master> from examples + update usages
  • Loading branch information
andrewor14 committed May 16, 2014
commit d32072c0bd60b1256a1c95bffcd1264bc80a4594
18 changes: 9 additions & 9 deletions examples/src/main/python/als.py
Original file line number Diff line number Diff line change
Expand Up @@ -46,15 +46,15 @@ def update(i, vec, mat, ratings):
return np.linalg.solve(XtX, Xty)

if __name__ == "__main__":
if len(sys.argv) < 2:
print >> sys.stderr, "Usage: als <master> <M> <U> <F> <iters> <slices>"
exit(-1)
sc = SparkContext(sys.argv[1], "PythonALS", pyFiles=[realpath(__file__)])
M = int(sys.argv[2]) if len(sys.argv) > 2 else 100
U = int(sys.argv[3]) if len(sys.argv) > 3 else 500
F = int(sys.argv[4]) if len(sys.argv) > 4 else 10
ITERATIONS = int(sys.argv[5]) if len(sys.argv) > 5 else 5
slices = int(sys.argv[6]) if len(sys.argv) > 6 else 2
"""
Usage: als [M] [U] [F] [iterations] [slices]"
"""
sc = SparkContext(appName="PythonALS", pyFiles=[realpath(__file__)])
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we need to pass the path through pyFiles if the user launches this with spark-submit?

M = int(sys.argv[1]) if len(sys.argv) > 1 else 100
U = int(sys.argv[2]) if len(sys.argv) > 2 else 500
F = int(sys.argv[3]) if len(sys.argv) > 3 else 10
ITERATIONS = int(sys.argv[4]) if len(sys.argv) > 4 else 5
slices = int(sys.argv[5]) if len(sys.argv) > 5 else 2

print "Running ALS with M=%d, U=%d, F=%d, iters=%d, slices=%d\n" % \
(M, U, F, ITERATIONS, slices)
Expand Down
12 changes: 6 additions & 6 deletions examples/src/main/python/kmeans.py
Original file line number Diff line number Diff line change
Expand Up @@ -45,14 +45,14 @@ def closestPoint(p, centers):


if __name__ == "__main__":
if len(sys.argv) < 5:
print >> sys.stderr, "Usage: kmeans <master> <file> <k> <convergeDist>"
if len(sys.argv) != 4:
print >> sys.stderr, "Usage: kmeans <file> <k> <convergeDist>"
exit(-1)
sc = SparkContext(sys.argv[1], "PythonKMeans")
lines = sc.textFile(sys.argv[2])
sc = SparkContext(appName="PythonKMeans")
lines = sc.textFile(sys.argv[1])
data = lines.map(parseVector).cache()
K = int(sys.argv[3])
convergeDist = float(sys.argv[4])
K = int(sys.argv[2])
convergeDist = float(sys.argv[3])

kPoints = data.takeSample(False, K, 1)
tempDist = 1.0
Expand Down
10 changes: 5 additions & 5 deletions examples/src/main/python/logistic_regression.py
Original file line number Diff line number Diff line change
Expand Up @@ -47,12 +47,12 @@ def readPointBatch(iterator):
return [matrix]

if __name__ == "__main__":
if len(sys.argv) != 4:
print >> sys.stderr, "Usage: logistic_regression <master> <file> <iters>"
if len(sys.argv) != 3:
print >> sys.stderr, "Usage: logistic_regression <file> <iterations>"
exit(-1)
sc = SparkContext(sys.argv[1], "PythonLR", pyFiles=[realpath(__file__)])
points = sc.textFile(sys.argv[2]).mapPartitions(readPointBatch).cache()
iterations = int(sys.argv[3])
sc = SparkContext(appName="PythonLR", pyFiles=[realpath(__file__)])
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same here, don't need pyFiles

points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache()
iterations = int(sys.argv[2])

# Initialize w to a random value
w = 2 * np.random.ranf(size=D) - 1
Expand Down
10 changes: 5 additions & 5 deletions examples/src/main/python/mllib/kmeans.py
Original file line number Diff line number Diff line change
Expand Up @@ -33,12 +33,12 @@ def parseVector(line):


if __name__ == "__main__":
if len(sys.argv) < 4:
print >> sys.stderr, "Usage: kmeans <master> <file> <k>"
if len(sys.argv) != 3:
print >> sys.stderr, "Usage: kmeans <file> <k>"
exit(-1)
sc = SparkContext(sys.argv[1], "KMeans")
lines = sc.textFile(sys.argv[2])
sc = SparkContext(appName="KMeans")
lines = sc.textFile(sys.argv[1])
data = lines.map(parseVector)
k = int(sys.argv[3])
k = int(sys.argv[2])
model = KMeans.train(data, k)
print "Final centers: " + str(model.clusterCenters)
10 changes: 5 additions & 5 deletions examples/src/main/python/mllib/logistic_regression.py
Original file line number Diff line number Diff line change
Expand Up @@ -39,12 +39,12 @@ def parsePoint(line):


if __name__ == "__main__":
if len(sys.argv) != 4:
print >> sys.stderr, "Usage: logistic_regression <master> <file> <iters>"
if len(sys.argv) != 3:
print >> sys.stderr, "Usage: logistic_regression <file> <iterations>"
exit(-1)
sc = SparkContext(sys.argv[1], "PythonLR")
points = sc.textFile(sys.argv[2]).map(parsePoint)
iterations = int(sys.argv[3])
sc = SparkContext(appName="PythonLR")
points = sc.textFile(sys.argv[1]).map(parsePoint)
iterations = int(sys.argv[2])
model = LogisticRegressionWithSGD.train(points, iterations)
print "Final weights: " + str(model.weights)
print "Final intercept: " + str(model.intercept)
10 changes: 5 additions & 5 deletions examples/src/main/python/pagerank.py
Original file line number Diff line number Diff line change
Expand Up @@ -36,19 +36,19 @@ def parseNeighbors(urls):


if __name__ == "__main__":
if len(sys.argv) < 3:
print >> sys.stderr, "Usage: pagerank <master> <file> <number_of_iterations>"
if len(sys.argv) != 3:
print >> sys.stderr, "Usage: pagerank <file> <iterations>"
exit(-1)

# Initialize the spark context.
sc = SparkContext(sys.argv[1], "PythonPageRank")
sc = SparkContext(appName="PythonPageRank")

# Loads in input file. It should be in format of:
# URL neighbor URL
# URL neighbor URL
# URL neighbor URL
# ...
lines = sc.textFile(sys.argv[2], 1)
lines = sc.textFile(sys.argv[1], 1)

# Loads all URLs from input file and initialize their neighbors.
links = lines.map(lambda urls: parseNeighbors(urls)).distinct().groupByKey().cache()
Expand All @@ -57,7 +57,7 @@ def parseNeighbors(urls):
ranks = links.map(lambda (url, neighbors): (url, 1.0))

# Calculates and updates URL ranks continuously using PageRank algorithm.
for iteration in xrange(int(sys.argv[3])):
for iteration in xrange(int(sys.argv[2])):
# Calculates URL contributions to the rank of other URLs.
contribs = links.join(ranks).flatMap(lambda (url, (urls, rank)):
computeContribs(urls, rank))
Expand Down
10 changes: 5 additions & 5 deletions examples/src/main/python/pi.py
Original file line number Diff line number Diff line change
Expand Up @@ -23,11 +23,11 @@


if __name__ == "__main__":
if len(sys.argv) == 1:
print >> sys.stderr, "Usage: pi <master> [<slices>]"
exit(-1)
sc = SparkContext(sys.argv[1], "PythonPi")
slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2
"""
Usage: pi [slices]
"""
sc = SparkContext(appName="PythonPi")
slices = int(sys.argv[1]) if len(sys.argv) > 1 else 2
n = 100000 * slices
def f(_):
x = random() * 2 - 1
Expand Down
8 changes: 4 additions & 4 deletions examples/src/main/python/sort.py
Original file line number Diff line number Diff line change
Expand Up @@ -21,11 +21,11 @@


if __name__ == "__main__":
if len(sys.argv) < 3:
print >> sys.stderr, "Usage: sort <master> <file>"
if len(sys.argv) != 2:
print >> sys.stderr, "Usage: sort <file>"
exit(-1)
sc = SparkContext(sys.argv[1], "PythonSort")
lines = sc.textFile(sys.argv[2], 1)
sc = SparkContext(appName="PythonSort")
lines = sc.textFile(sys.argv[1], 1)
sortedCount = lines.flatMap(lambda x: x.split(' ')) \
.map(lambda x: (int(x), 1)) \
.sortByKey(lambda x: x)
Expand Down
10 changes: 5 additions & 5 deletions examples/src/main/python/transitive_closure.py
Original file line number Diff line number Diff line change
Expand Up @@ -36,11 +36,11 @@ def generateGraph():


if __name__ == "__main__":
if len(sys.argv) == 1:
print >> sys.stderr, "Usage: transitive_closure <master> [<slices>]"
exit(-1)
sc = SparkContext(sys.argv[1], "PythonTransitiveClosure")
slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2
"""
Usage: transitive_closure [slices]
"""
sc = SparkContext(appName="PythonTransitiveClosure")
slices = int(sys.argv[1]) if len(sys.argv) > 1 else 2
tc = sc.parallelize(generateGraph(), slices).cache()

# Linear transitive closure: each round grows paths by one edge,
Expand Down
8 changes: 4 additions & 4 deletions examples/src/main/python/wordcount.py
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,11 @@


if __name__ == "__main__":
if len(sys.argv) < 3:
print >> sys.stderr, "Usage: wordcount <master> <file>"
if len(sys.argv) != 2:
print >> sys.stderr, "Usage: wordcount <file>"
exit(-1)
sc = SparkContext(sys.argv[1], "PythonWordCount")
lines = sc.textFile(sys.argv[2], 1)
sc = SparkContext(appName="PythonWordCount")
lines = sc.textFile(sys.argv[1], 1)
counts = lines.flatMap(lambda x: x.split(' ')) \
.map(lambda x: (x, 1)) \
.reduceByKey(add)
Expand Down