Skip to content

Commit d32072c

Browse files
committed
Remove <master> from examples + update usages
1 parent 94c5139 commit d32072c

File tree

10 files changed

+53
-53
lines changed

10 files changed

+53
-53
lines changed

examples/src/main/python/als.py

Lines changed: 9 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -46,15 +46,15 @@ def update(i, vec, mat, ratings):
4646
return np.linalg.solve(XtX, Xty)
4747

4848
if __name__ == "__main__":
49-
if len(sys.argv) < 2:
50-
print >> sys.stderr, "Usage: als <master> <M> <U> <F> <iters> <slices>"
51-
exit(-1)
52-
sc = SparkContext(sys.argv[1], "PythonALS", pyFiles=[realpath(__file__)])
53-
M = int(sys.argv[2]) if len(sys.argv) > 2 else 100
54-
U = int(sys.argv[3]) if len(sys.argv) > 3 else 500
55-
F = int(sys.argv[4]) if len(sys.argv) > 4 else 10
56-
ITERATIONS = int(sys.argv[5]) if len(sys.argv) > 5 else 5
57-
slices = int(sys.argv[6]) if len(sys.argv) > 6 else 2
49+
"""
50+
Usage: als [M] [U] [F] [iterations] [slices]"
51+
"""
52+
sc = SparkContext(appName="PythonALS", pyFiles=[realpath(__file__)])
53+
M = int(sys.argv[1]) if len(sys.argv) > 1 else 100
54+
U = int(sys.argv[2]) if len(sys.argv) > 2 else 500
55+
F = int(sys.argv[3]) if len(sys.argv) > 3 else 10
56+
ITERATIONS = int(sys.argv[4]) if len(sys.argv) > 4 else 5
57+
slices = int(sys.argv[5]) if len(sys.argv) > 5 else 2
5858

5959
print "Running ALS with M=%d, U=%d, F=%d, iters=%d, slices=%d\n" % \
6060
(M, U, F, ITERATIONS, slices)

examples/src/main/python/kmeans.py

Lines changed: 6 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -45,14 +45,14 @@ def closestPoint(p, centers):
4545

4646

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

5757
kPoints = data.takeSample(False, K, 1)
5858
tempDist = 1.0

examples/src/main/python/logistic_regression.py

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -47,12 +47,12 @@ def readPointBatch(iterator):
4747
return [matrix]
4848

4949
if __name__ == "__main__":
50-
if len(sys.argv) != 4:
51-
print >> sys.stderr, "Usage: logistic_regression <master> <file> <iters>"
50+
if len(sys.argv) != 3:
51+
print >> sys.stderr, "Usage: logistic_regression <file> <iterations>"
5252
exit(-1)
53-
sc = SparkContext(sys.argv[1], "PythonLR", pyFiles=[realpath(__file__)])
54-
points = sc.textFile(sys.argv[2]).mapPartitions(readPointBatch).cache()
55-
iterations = int(sys.argv[3])
53+
sc = SparkContext(appName="PythonLR", pyFiles=[realpath(__file__)])
54+
points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache()
55+
iterations = int(sys.argv[2])
5656

5757
# Initialize w to a random value
5858
w = 2 * np.random.ranf(size=D) - 1

examples/src/main/python/mllib/kmeans.py

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -33,12 +33,12 @@ def parseVector(line):
3333

3434

3535
if __name__ == "__main__":
36-
if len(sys.argv) < 4:
37-
print >> sys.stderr, "Usage: kmeans <master> <file> <k>"
36+
if len(sys.argv) != 3:
37+
print >> sys.stderr, "Usage: kmeans <file> <k>"
3838
exit(-1)
39-
sc = SparkContext(sys.argv[1], "KMeans")
40-
lines = sc.textFile(sys.argv[2])
39+
sc = SparkContext(appName="KMeans")
40+
lines = sc.textFile(sys.argv[1])
4141
data = lines.map(parseVector)
42-
k = int(sys.argv[3])
42+
k = int(sys.argv[2])
4343
model = KMeans.train(data, k)
4444
print "Final centers: " + str(model.clusterCenters)

examples/src/main/python/mllib/logistic_regression.py

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -39,12 +39,12 @@ def parsePoint(line):
3939

4040

4141
if __name__ == "__main__":
42-
if len(sys.argv) != 4:
43-
print >> sys.stderr, "Usage: logistic_regression <master> <file> <iters>"
42+
if len(sys.argv) != 3:
43+
print >> sys.stderr, "Usage: logistic_regression <file> <iterations>"
4444
exit(-1)
45-
sc = SparkContext(sys.argv[1], "PythonLR")
46-
points = sc.textFile(sys.argv[2]).map(parsePoint)
47-
iterations = int(sys.argv[3])
45+
sc = SparkContext(appName="PythonLR")
46+
points = sc.textFile(sys.argv[1]).map(parsePoint)
47+
iterations = int(sys.argv[2])
4848
model = LogisticRegressionWithSGD.train(points, iterations)
4949
print "Final weights: " + str(model.weights)
5050
print "Final intercept: " + str(model.intercept)

examples/src/main/python/pagerank.py

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -36,19 +36,19 @@ def parseNeighbors(urls):
3636

3737

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

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

4646
# Loads in input file. It should be in format of:
4747
# URL neighbor URL
4848
# URL neighbor URL
4949
# URL neighbor URL
5050
# ...
51-
lines = sc.textFile(sys.argv[2], 1)
51+
lines = sc.textFile(sys.argv[1], 1)
5252

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

5959
# Calculates and updates URL ranks continuously using PageRank algorithm.
60-
for iteration in xrange(int(sys.argv[3])):
60+
for iteration in xrange(int(sys.argv[2])):
6161
# Calculates URL contributions to the rank of other URLs.
6262
contribs = links.join(ranks).flatMap(lambda (url, (urls, rank)):
6363
computeContribs(urls, rank))

examples/src/main/python/pi.py

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -23,11 +23,11 @@
2323

2424

2525
if __name__ == "__main__":
26-
if len(sys.argv) == 1:
27-
print >> sys.stderr, "Usage: pi <master> [<slices>]"
28-
exit(-1)
29-
sc = SparkContext(sys.argv[1], "PythonPi")
30-
slices = int(sys.argv[2]) if len(sys.argv) > 2 else 2
26+
"""
27+
Usage: pi [slices]
28+
"""
29+
sc = SparkContext(appName="PythonPi")
30+
slices = int(sys.argv[1]) if len(sys.argv) > 1 else 2
3131
n = 100000 * slices
3232
def f(_):
3333
x = random() * 2 - 1

examples/src/main/python/sort.py

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -21,11 +21,11 @@
2121

2222

2323
if __name__ == "__main__":
24-
if len(sys.argv) < 3:
25-
print >> sys.stderr, "Usage: sort <master> <file>"
24+
if len(sys.argv) != 2:
25+
print >> sys.stderr, "Usage: sort <file>"
2626
exit(-1)
27-
sc = SparkContext(sys.argv[1], "PythonSort")
28-
lines = sc.textFile(sys.argv[2], 1)
27+
sc = SparkContext(appName="PythonSort")
28+
lines = sc.textFile(sys.argv[1], 1)
2929
sortedCount = lines.flatMap(lambda x: x.split(' ')) \
3030
.map(lambda x: (int(x), 1)) \
3131
.sortByKey(lambda x: x)

examples/src/main/python/transitive_closure.py

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -36,11 +36,11 @@ def generateGraph():
3636

3737

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

4646
# Linear transitive closure: each round grows paths by one edge,

examples/src/main/python/wordcount.py

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -22,11 +22,11 @@
2222

2323

2424
if __name__ == "__main__":
25-
if len(sys.argv) < 3:
26-
print >> sys.stderr, "Usage: wordcount <master> <file>"
25+
if len(sys.argv) != 2:
26+
print >> sys.stderr, "Usage: wordcount <file>"
2727
exit(-1)
28-
sc = SparkContext(sys.argv[1], "PythonWordCount")
29-
lines = sc.textFile(sys.argv[2], 1)
28+
sc = SparkContext(appName="PythonWordCount")
29+
lines = sc.textFile(sys.argv[1], 1)
3030
counts = lines.flatMap(lambda x: x.split(' ')) \
3131
.map(lambda x: (x, 1)) \
3232
.reduceByKey(add)

0 commit comments

Comments
 (0)